You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2016/09/05 08:11:40 UTC

[4/7] ignite git commit: IGNITE-3390: Added DSN configuration window.

IGNITE-3390: Added DSN configuration window.


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

Branch: refs/heads/master
Commit: 70e69cb7aa08c268b07920838add4a40e28fe25d
Parents: dabd86c
Author: isapego <is...@gridgain.com>
Authored: Wed Aug 31 16:47:11 2016 +0300
Committer: isapego <is...@gridgain.com>
Committed: Wed Aug 31 16:47:11 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcHandshakeRequest.java   |   8 +-
 .../processors/odbc/OdbcHandshakeResult.java    |  17 +-
 .../processors/odbc/OdbcMessageParser.java      |  22 +-
 .../processors/odbc/OdbcProtocolVersion.java    | 106 ++++++++
 .../processors/odbc/OdbcRequestHandler.java     |  17 +-
 .../cpp/common/include/ignite/common/utils.h    |  25 ++
 .../cpp/common/os/win/src/common/utils.cpp      |  20 ++
 modules/platforms/cpp/odbc-test/Makefile.am     |   1 +
 .../odbc-test/config/queries-test-noodbc.xml    | 103 +++++++
 .../cpp/odbc-test/config/queries-test.xml       |  31 ++-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   2 +
 .../project/vs/odbc-test.vcxproj.filters        |   6 +
 .../cpp/odbc-test/src/configuration_test.cpp    | 156 ++++++++---
 .../cpp/odbc-test/src/queries_test.cpp          | 122 ++++++---
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  14 +-
 modules/platforms/cpp/odbc/Makefile.am          |   2 +
 modules/platforms/cpp/odbc/include/Makefile.am  |   2 +
 .../cpp/odbc/include/ignite/odbc/common_types.h |   3 +
 .../include/ignite/odbc/config/configuration.h  | 207 +++++++++++++--
 .../cpp/odbc/include/ignite/odbc/connection.h   |  47 ++--
 .../ignite/odbc/diagnostic/diagnostic_record.h  |   2 +-
 .../cpp/odbc/include/ignite/odbc/dsn_config.h   |  61 +++++
 .../cpp/odbc/include/ignite/odbc/parser.h       |   3 -
 .../odbc/include/ignite/odbc/protocol_version.h | 168 ++++++++++++
 .../include/ignite/odbc/system/odbc_constants.h |   4 -
 .../odbc/system/ui/dsn_configuration_window.h   | 136 ++++++++++
 .../ignite/odbc/system/ui/custom_window.h       | 189 +++++++++++++
 .../win/include/ignite/odbc/system/ui/window.h  | 201 ++++++++++++++
 .../odbc/os/win/src/system/ui/custom_window.cpp | 184 +++++++++++++
 .../src/system/ui/dsn_configuration_window.cpp  | 212 +++++++++++++++
 .../cpp/odbc/os/win/src/system/ui/window.cpp    | 192 +++++++++++++
 .../cpp/odbc/os/win/src/system_dsn.cpp          | 218 +++++++++++++++
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |  27 +-
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |  36 +++
 .../cpp/odbc/src/config/configuration.cpp       | 266 +++++++++++--------
 modules/platforms/cpp/odbc/src/connection.cpp   |  66 +++--
 .../odbc/src/diagnostic/diagnostic_record.cpp   |   8 +-
 .../diagnostic/diagnostic_record_storage.cpp    |   2 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   | 111 ++++++++
 modules/platforms/cpp/odbc/src/entry_points.cpp |   8 -
 modules/platforms/cpp/odbc/src/odbc.cpp         |  83 ++----
 .../platforms/cpp/odbc/src/protocol_version.cpp | 131 +++++++++
 42 files changed, 2822 insertions(+), 397 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java
index 5e09041..2ffd8cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java
@@ -24,21 +24,21 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  */
 public class OdbcHandshakeRequest extends OdbcRequest {
     /** Protocol version. */
-    private final long ver;
+    private final OdbcProtocolVersion ver;
 
     /**
-     * @param ver Protocol version.
+     * @param ver Long value for protocol version.
      */
     public OdbcHandshakeRequest(long ver) {
         super(HANDSHAKE);
 
-        this.ver = ver;
+        this.ver = OdbcProtocolVersion.fromLong(ver);
     }
 
     /**
      * @return Protocol version.
      */
-    public long version() {
+    public OdbcProtocolVersion version() {
         return ver;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java
index bf1c61e..74c5bd4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.odbc;
 
-import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * ODBC handshake result.
@@ -33,11 +33,13 @@ public class OdbcHandshakeResult {
     private final String curVer;
 
     /**
-     * @param accepted Handshake accepted.
+     * Constructor.
+     *
+     * @param accepted Indicates whether handshake accepted or not.
      * @param protoVerSince Apache Ignite version when protocol version has been introduced.
      * @param curVer Current Apache Ignite version.
      */
-    public OdbcHandshakeResult(boolean accepted, @Nullable String protoVerSince, @Nullable String curVer) {
+    public OdbcHandshakeResult(boolean accepted, String protoVerSince, String curVer) {
         this.accepted = accepted;
         this.protoVerSince = protoVerSince;
         this.curVer = curVer;
@@ -53,14 +55,19 @@ public class OdbcHandshakeResult {
     /**
      * @return Apache Ignite version when protocol version has been introduced.
      */
-    @Nullable public String protoVerSince() {
+    public String protocolVersionSince() {
         return protoVerSince;
     }
 
     /**
      * @return Current Apache Ignite version.
      */
-    @Nullable public String currentVer() {
+    public String currentVersion() {
         return curVer;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OdbcHandshakeResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
index fce8b1b..a751eb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
@@ -34,12 +34,6 @@ import java.util.Collection;
  * ODBC message parser.
  */
 public class OdbcMessageParser {
-    /** Current ODBC communication protocol version. */
-    public static final long PROTO_VER = 1;
-
-    /** Apache Ignite version when ODBC communication protocol version has been introduced. */
-    public static final String PROTO_VER_SINCE = "1.6.0";
-
     /** Initial output stream capacity. */
     private static final int INIT_CAP = 1024;
 
@@ -82,10 +76,14 @@ public class OdbcMessageParser {
         // we has not confirmed that the remote client uses the same protocol version.
         if (!verConfirmed) {
             if (cmd == OdbcRequest.HANDSHAKE)
-                return new OdbcHandshakeRequest(reader.readLong());
+            {
+                long longVersion = reader.readLong();
+
+                return new OdbcHandshakeRequest(longVersion);
+            }
             else
-                throw new IgniteException("Unexpected ODBC command (first message is not a handshake request): [cmd=" +
-                    cmd + ']');
+                throw new IgniteException("Unexpected ODBC command " +
+                        "(first message is not a handshake request): [cmd=" + cmd + ']');
         }
 
         OdbcRequest res;
@@ -174,6 +172,8 @@ public class OdbcMessageParser {
 
         Object res0 = msg.response();
 
+        if (res0 == null)
+            return writer.array();
         if (res0 instanceof OdbcHandshakeResult) {
             OdbcHandshakeResult res = (OdbcHandshakeResult) res0;
 
@@ -189,8 +189,8 @@ public class OdbcMessageParser {
             }
             else {
                 writer.writeBoolean(false);
-                writer.writeString(res.protoVerSince());
-                writer.writeString(res.currentVer());
+                writer.writeString(res.protocolVersionSince());
+                writer.writeString(res.currentVersion());
             }
         }
         else if (res0 instanceof OdbcQueryExecuteResult) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java
new file mode 100644
index 0000000..97a1306
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java
@@ -0,0 +1,106 @@
+/*
+ * 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.processors.odbc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * ODBC protocol version.
+ */
+public enum OdbcProtocolVersion {
+    /** First version of the ODBC. Released with Ignite 1.6 */
+    VERSION_1_6_0(1),
+
+    /** Unknown version. */
+    VERSION_UNKNOWN(Long.MIN_VALUE);
+
+    /** Long value to enum map. */
+    private static final Map<Long, OdbcProtocolVersion> versions = new HashMap<>();
+
+    /** Enum value to Ignite version map */
+    private static final Map<OdbcProtocolVersion, String> since = new HashMap<>();
+
+    /**
+     * Map long values to version.
+     */
+    static {
+        for (OdbcProtocolVersion version : values())
+            versions.put(version.longValue(), version);
+
+        since.put(VERSION_1_6_0, "1.6.0");
+    }
+
+    /** Long value for version. */
+    private final long longVal;
+
+    /**
+     * @param longVal Long value.
+     */
+    OdbcProtocolVersion(long longVal) {
+        this.longVal = longVal;
+    }
+
+    /**
+     * @param longVal Long value.
+     * @return Protocol version.
+     */
+    public static OdbcProtocolVersion fromLong(long longVal) {
+        OdbcProtocolVersion res = versions.get(longVal);
+
+        return res == null ? VERSION_UNKNOWN : res;
+    }
+
+    /**
+     * @return Current version.
+     */
+    public static OdbcProtocolVersion current() {
+        return VERSION_1_6_0;
+    }
+
+    /**
+     * @return Long value.
+     */
+    public long longValue() {
+        return longVal;
+    }
+
+    /**
+     * @return {@code true} if this version is unknown.
+     */
+    public boolean isUnknown() {
+        return longVal == VERSION_UNKNOWN.longVal;
+    }
+
+    /**
+     * @return {@code true} if this version supports distributed joins.
+     */
+    public boolean isDistributedJoinsSupported() {
+        assert !isUnknown();
+
+        return longVal >= VERSION_1_6_0.longVal;
+    }
+
+    /**
+     * @return Ignite version when introduced.
+     */
+    public String since() {
+        assert !isUnknown();
+
+        return since.get(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index ce98720..3f7d505 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -91,7 +91,7 @@ public class OdbcRequestHandler {
         try {
             switch (req.command()) {
                 case HANDSHAKE:
-                    return performHandshake(reqId, (OdbcHandshakeRequest)req);
+                    return performHandshake((OdbcHandshakeRequest)req);
 
                 case EXECUTE_SQL_QUERY:
                     return executeQuery(reqId, (OdbcQueryExecuteRequest)req);
@@ -119,23 +119,24 @@ public class OdbcRequestHandler {
     /**
      * {@link OdbcHandshakeRequest} command handler.
      *
-     * @param reqId Request ID.
      * @param req Handshake request.
      * @return Response.
      */
-    private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest req) {
-        OdbcHandshakeResult res;
+    private OdbcResponse performHandshake(OdbcHandshakeRequest req) {
+        OdbcProtocolVersion version = req.version();
 
-        if (req.version() == OdbcMessageParser.PROTO_VER)
-            res = new OdbcHandshakeResult(true, null, null);
-        else {
+        if (version.isUnknown()) {
             IgniteProductVersion ver = ctx.grid().version();
 
             String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance();
 
-            res = new OdbcHandshakeResult(false, OdbcMessageParser.PROTO_VER_SINCE, verStr);
+            OdbcHandshakeResult res = new OdbcHandshakeResult(false, OdbcProtocolVersion.current().since(), verStr);
+
+            return new OdbcResponse(res);
         }
 
+        OdbcHandshakeResult res = new OdbcHandshakeResult(true, null, null);
+
         return new OdbcResponse(res);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/common/include/ignite/common/utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h
index c1046e2..f4d2a9f 100644
--- a/modules/platforms/cpp/common/include/ignite/common/utils.h
+++ b/modules/platforms/cpp/common/include/ignite/common/utils.h
@@ -60,6 +60,13 @@ namespace ignite
         }
 
         /**
+         * Strips leading and trailing whitespaces from string.
+         *
+         * @param str String to be transformed.
+         */
+        IGNITE_IMPORT_EXPORT void StripSurroundingWhitespaces(std::string& str);
+
+        /**
          * Get string representation of long in decimal form.
          *
          * @param val Long value to be converted to string.
@@ -179,6 +186,24 @@ namespace ignite
 
             return res;
         }
+
+        /**
+         * Check if the predicate returns true for all the elements of the
+         * sequence.
+         *
+         * @return True if the predicate returns true for all the elements
+         *     of the sequence and false otherwise.
+         */
+        template<typename Iter, typename Pred>
+        bool AllOf(Iter begin, Iter end, Pred pred)
+        {
+            Iter i = begin;
+
+            while (i != end && pred(*i))
+                ++i;
+
+            return i == end;
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/common/os/win/src/common/utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/os/win/src/common/utils.cpp b/modules/platforms/cpp/common/os/win/src/common/utils.cpp
index 47d7f43..77c90b8 100644
--- a/modules/platforms/cpp/common/os/win/src/common/utils.cpp
+++ b/modules/platforms/cpp/common/os/win/src/common/utils.cpp
@@ -40,6 +40,26 @@ namespace ignite
             return false;
         }
 
+        void StripSurroundingWhitespaces(std::string& str)
+        {
+            std::string::size_type newBegin = 0;
+            while (newBegin < str.size() && ::isspace(str[newBegin]))
+                ++newBegin;
+
+            if (newBegin == str.size())
+            {
+                str.clear();
+
+                return;
+            }
+
+            std::string::size_type newEnd = str.size() - 1;
+            while (::isspace(str[newEnd]))
+                --newEnd;
+
+            str.assign(str, newBegin, (newEnd - newBegin) + 1);
+        }
+
         time_t IgniteTimeGm(const tm& time)
         {
             tm tmc = time;

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index de8fb5d..1ca85a7 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -75,6 +75,7 @@ ignite_odbc_tests_SOURCES = \
     ../odbc/src/app/application_data_buffer.cpp \
     ../odbc/src/config/configuration.cpp \
     ../odbc/src/row.cpp \
+    ../odbc/src/protocol_version.cpp \
     ../odbc/src/column.cpp \
     ../odbc/src/utility.cpp \
     ../odbc/src/result_page.cpp

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
new file mode 100644
index 0000000..db19669
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
@@ -0,0 +1,103 @@
+<?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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+        <property name="odbcConfiguration"><null/></property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <!-- Configure type metadata to enable queries. -->
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Long"/>
+                                <property name="valueType" value="TestType"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="i8Field" value="java.lang.Byte"/>
+                                        <entry key="i16Field" value="java.lang.Short"/>
+                                        <entry key="i32Field" value="java.lang.Integer"/>
+                                        <entry key="i64Field" value="java.lang.Long"/>
+                                        <entry key="strField" value="java.lang.String"/>
+                                        <entry key="floatField" value="java.lang.Float"/>
+                                        <entry key="doubleField" value="java.lang.Double"/>
+                                        <entry key="boolField" value="java.lang.Boolean"/>
+                                        <entry key="guidField" value="java.util.UUID"/>
+                                        <entry key="dateField" value="java.util.Date"/>
+                                        <entry key="timestampField" value="java.sql.Timestamp"/>
+                                    </map>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i32Field"/>
+                                        </bean>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i64Field"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/config/queries-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml
index 67415fb..26e6341 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml
@@ -31,7 +31,9 @@
 
         <!-- Enabling ODBC. -->
         <property name="odbcConfiguration">
-            <bean class="org.apache.ignite.configuration.OdbcConfiguration"></bean>
+            <bean class="org.apache.ignite.configuration.OdbcConfiguration">
+                <property name="endpointAddress" value="127.0.0.1:11110"/>
+            </bean>
         </property>
 
         <property name="cacheConfiguration">
@@ -39,16 +41,17 @@
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="name" value="cache"/>
                     <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="writeSynchronizationMode" value="PRIMARY_SYNC"/>
-            
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
                     <!-- Configure type metadata to enable queries. -->
-                    <property name="typeMetadata">
+                    <property name="queryEntities">
                         <list>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
+                            <bean class="org.apache.ignite.cache.QueryEntity">
                                 <property name="keyType" value="java.lang.Long"/>
                                 <property name="valueType" value="TestType"/>
-                                <property name="queryFields">
+
+                                <property name="fields">
                                     <map>
                                         <entry key="i8Field" value="java.lang.Byte"/>
                                         <entry key="i16Field" value="java.lang.Short"/>
@@ -63,6 +66,17 @@
                                         <entry key="timestampField" value="java.sql.Timestamp"/>
                                     </map>
                                 </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i32Field"/>
+                                        </bean>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i64Field"/>
+                                        </bean>
+                                    </list>
+                                </property>
                             </bean>
                         </list>
                     </property>
@@ -79,8 +93,7 @@
                         instead os static IP based discovery.
                     -->
                     <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
                         <property name="addresses">
                             <list>
                                 <!-- In distributed environment, replace with actual host IP address. -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index a9423ca..cb5735f 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -157,6 +157,7 @@
     <ClCompile Include="..\..\..\odbc\src\config\configuration.cpp" />
     <ClCompile Include="..\..\..\odbc\src\config\connection_info.cpp" />
     <ClCompile Include="..\..\..\odbc\src\cursor.cpp" />
+    <ClCompile Include="..\..\..\odbc\src\protocol_version.cpp" />
     <ClCompile Include="..\..\..\odbc\src\result_page.cpp" />
     <ClCompile Include="..\..\..\odbc\src\row.cpp" />
     <ClCompile Include="..\..\..\odbc\src\utility.cpp" />
@@ -202,6 +203,7 @@
     </ProjectReference>
   </ItemGroup>
   <ItemGroup>
+    <None Include="..\..\config\queries-test-noodbc.xml" />
     <None Include="..\..\config\queries-test.xml" />
   </ItemGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 84f5a29..270bdd6 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -76,6 +76,9 @@
     <ClCompile Include="..\..\src\queries_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\..\odbc\src\protocol_version.cpp">
+      <Filter>Externals</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
@@ -122,5 +125,8 @@
     <None Include="..\..\config\queries-test.xml">
       <Filter>Configs</Filter>
     </None>
+    <None Include="..\..\config\queries-test-noodbc.xml">
+      <Filter>Configs</Filter>
+    </None>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
index 85aa3ff..0fd3277 100644
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@ -20,51 +20,93 @@
 #endif
 
 #include <iostream>
+#include <set>
 
 #include <boost/test/unit_test.hpp>
 
 #include <ignite/odbc/config/configuration.h>
+#include <ignite/ignite_error.h>
+#include <ignite/common/utils.h>
 
 using namespace ignite::odbc::config;
 
 namespace
 {
-    const char* testDriverName = "Ignite";
-    const char* testServerHost = "testhost.com";
+    const std::string testDriverName = "Ignite Driver";
+    const std::string testServerHost = "testhost.com";
     const uint16_t testServerPort = 4242;
-    const char* testCacheName = "TestCache";
-    const char* testDsn = "Ignite DSN";
+    const std::string testCacheName = "TestCache";
+    const std::string testDsn = "Ignite DSN";
+
+    const std::string testAddress = testServerHost + ':' + ignite::common::LexicalCast<std::string>(testServerPort);
 }
 
-BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite)
+void CheckValidAddress(const char* connectStr, uint16_t port)
+{
+    Configuration cfg;
+
+    BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr));
+
+    BOOST_CHECK_EQUAL(cfg.GetTcpPort(), port);
+}
+
+void CheckValidProtocolVersion(const char* connectStr, ignite::odbc::ProtocolVersion version)
+{
+    Configuration cfg;
+
+    BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr));
+
+    BOOST_CHECK(cfg.GetProtocolVersion() == version);
+}
+
+void CheckInvalidProtocolVersion(const char* connectStr)
+{
+    Configuration cfg;
+
+    cfg.FillFromConnectString(connectStr);
+
+    BOOST_CHECK_THROW(cfg.GetProtocolVersion(), ignite::IgniteError);
+}
 
 void CheckConnectionConfig(const Configuration& cfg)
 {
-    BOOST_REQUIRE(cfg.GetDriver() == testDriverName);
-    BOOST_REQUIRE(cfg.GetHost() == testServerHost);
-    BOOST_REQUIRE(cfg.GetPort() == testServerPort);
-    BOOST_REQUIRE(cfg.GetCache() == testCacheName);
-    BOOST_REQUIRE(cfg.GetDsn().empty());
+    BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName);
+    BOOST_CHECK_EQUAL(cfg.GetHost(), testServerHost);
+    BOOST_CHECK_EQUAL(cfg.GetTcpPort(), testServerPort);
+    BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress);
+    BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName);
+    BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string());
 
     std::stringstream constructor;
 
-    constructor << "driver={" << testDriverName << "};"
-                << "server=" << testServerHost << ";"
-                << "port=" << testServerPort << ";"
-                << "cache=" << testCacheName << ";";
+    constructor << "address=" << testAddress << ';'
+                << "cache=" << testCacheName << ';'
+                << "driver={" << testDriverName << "};";
 
     const std::string& expectedStr = constructor.str();
 
-    BOOST_REQUIRE(cfg.ToConnectString() == expectedStr);
+    BOOST_CHECK_EQUAL(ignite::common::ToLower(cfg.ToConnectString()), ignite::common::ToLower(expectedStr));
 }
 
 void CheckDsnConfig(const Configuration& cfg)
 {
-    BOOST_REQUIRE(cfg.GetDriver() == testDriverName);
-    BOOST_REQUIRE(cfg.GetDsn() == testDsn);
-    BOOST_REQUIRE(cfg.GetHost().empty());
-    BOOST_REQUIRE(cfg.GetCache().empty());
-    BOOST_REQUIRE(cfg.GetPort() == 0);
+    BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName);
+    BOOST_CHECK_EQUAL(cfg.GetDsn(), testDsn);
+    BOOST_CHECK_EQUAL(cfg.GetCache(), Configuration::DefaultValue::cache);
+    BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address);
+    BOOST_CHECK_EQUAL(cfg.GetHost(), std::string());
+    BOOST_CHECK_EQUAL(cfg.GetTcpPort(), Configuration::DefaultValue::port);
+}
+
+BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite)
+
+BOOST_AUTO_TEST_CASE(CheckTestValuesNotEquealDefault)
+{
+    BOOST_CHECK_NE(testDriverName, Configuration::DefaultValue::driver);
+    BOOST_CHECK_NE(testAddress, Configuration::DefaultValue::address);
+    BOOST_CHECK_NE(testServerPort, Configuration::DefaultValue::port);
+    BOOST_CHECK_NE(testCacheName, Configuration::DefaultValue::cache);
+    BOOST_CHECK_NE(testDsn, Configuration::DefaultValue::dsn);
 }
 
 BOOST_AUTO_TEST_CASE(TestConnectStringUppercase)
@@ -74,13 +116,12 @@ BOOST_AUTO_TEST_CASE(TestConnectStringUppercase)
     std::stringstream constructor;
 
     constructor << "DRIVER={" << testDriverName << "};"
-                << "SERVER=" << testServerHost <<";"
-                << "PORT=" << testServerPort << ";"
+                << "ADDRESS=" << testAddress << ';'
                 << "CACHE=" << testCacheName;
 
     const std::string& connectStr = constructor.str();
 
-    cfg.FillFromConnectString(connectStr.c_str(), connectStr.size());
+    cfg.FillFromConnectString(connectStr);
 
     CheckConnectionConfig(cfg);
 }
@@ -92,13 +133,12 @@ BOOST_AUTO_TEST_CASE(TestConnectStringLowercase)
     std::stringstream constructor;
 
     constructor << "driver={" << testDriverName << "};"
-                << "server=" << testServerHost << ";"
-                << "port=" << testServerPort << ";"
-                << "cache=" << testCacheName;
+                << "address=" << testAddress << ';'
+                << "cache=" << testCacheName << ';';
 
     const std::string& connectStr = constructor.str();
 
-    cfg.FillFromConnectString(connectStr.c_str(), connectStr.size());
+    cfg.FillFromConnectString(connectStr);
 
     CheckConnectionConfig(cfg);
 }
@@ -110,9 +150,8 @@ BOOST_AUTO_TEST_CASE(TestConnectStringZeroTerminated)
     std::stringstream constructor;
 
     constructor << "driver={" << testDriverName << "};"
-                << "server=" << testServerHost << ";"
-                << "port=" << testServerPort << ";"
-                << "cache=" << testCacheName;
+                << "address=" << testAddress << ';'
+                << "cache=" << testCacheName << ';';
 
     const std::string& connectStr = constructor.str();
 
@@ -128,13 +167,12 @@ BOOST_AUTO_TEST_CASE(TestConnectStringMixed)
     std::stringstream constructor;
 
     constructor << "Driver={" << testDriverName << "};"
-                << "Server=" << testServerHost << ";"
-                << "Port=" << testServerPort << ";"
-                << "Cache=" << testCacheName;
+                << "Address=" << testAddress << ';'
+                << "Cache=" << testCacheName << ';';
 
     const std::string& connectStr = constructor.str();
 
-    cfg.FillFromConnectString(connectStr.c_str(), connectStr.size());
+    cfg.FillFromConnectString(connectStr);
 
     CheckConnectionConfig(cfg);
 }
@@ -146,17 +184,53 @@ BOOST_AUTO_TEST_CASE(TestConnectStringWhitepaces)
     std::stringstream constructor;
 
     constructor << "DRIVER = {" << testDriverName << "} ;\n"
-                << " SERVER =" << testServerHost << " ; \n"
-                << "PORT= " << testServerPort << "; "
-                << "CACHE = \n\r" << testCacheName;
+                << " ADDRESS =" << testAddress << "; "
+                << "CACHE = \n\r" << testCacheName << ';';
 
     const std::string& connectStr = constructor.str();
 
-    cfg.FillFromConnectString(connectStr.c_str(), connectStr.size());
+    cfg.FillFromConnectString(connectStr);
 
     CheckConnectionConfig(cfg);
 }
 
+BOOST_AUTO_TEST_CASE(TestConnectStringInvalidAddress)
+{
+    Configuration cfg;
+
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:0;"), ignite::IgniteError);
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:00000;"), ignite::IgniteError);
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:fdsf;"), ignite::IgniteError);
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:123:1;"), ignite::IgniteError);
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:12322221;"), ignite::IgniteError);
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:12322a;"), ignite::IgniteError);
+    BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:;"), ignite::IgniteError);
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectStringValidAddress)
+{
+    CheckValidAddress("Address=example.com:1;", 1);
+    CheckValidAddress("Address=example.com:31242;", 31242);
+    CheckValidAddress("Address=example.com:55555;", 55555);
+    CheckValidAddress("Address=example.com:110;", 110);
+    CheckValidAddress("Address=example.com;", Configuration::DefaultValue::port);
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectStringInvalidVersion)
+{
+    CheckInvalidProtocolVersion("Protocol_Version=0;");
+    CheckInvalidProtocolVersion("Protocol_Version=1;");
+    CheckInvalidProtocolVersion("Protocol_Version=2;");
+    CheckInvalidProtocolVersion("Protocol_Version=1.6.1;");
+    CheckInvalidProtocolVersion("Protocol_Version=1.7.0;");
+    CheckInvalidProtocolVersion("Protocol_Version=1.8.1;");
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectStringValidVersion)
+{
+    CheckValidProtocolVersion("Protocol_Version=1.6.0;", ignite::odbc::ProtocolVersion::VERSION_1_6_0);
+}
+
 BOOST_AUTO_TEST_CASE(TestDsnStringUppercase)
 {
     Configuration cfg;
@@ -173,7 +247,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStringUppercase)
     CheckDsnConfig(cfg);
 }
 
-BOOST_AUTO_TEST_CASE(TestDsnStrinLowercase)
+BOOST_AUTO_TEST_CASE(TestDsnStringLowercase)
 {
     Configuration cfg;
 
@@ -189,7 +263,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStrinLowercase)
     CheckDsnConfig(cfg);
 }
 
-BOOST_AUTO_TEST_CASE(TestDsnStrinMixed)
+BOOST_AUTO_TEST_CASE(TestDsnStringMixed)
 {
     Configuration cfg;
 
@@ -205,7 +279,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStrinMixed)
     CheckDsnConfig(cfg);
 }
 
-BOOST_AUTO_TEST_CASE(TestDsnStrinWhitespaces)
+BOOST_AUTO_TEST_CASE(TestDsnStringWhitespaces)
 {
     Configuration cfg;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index 2d9bd58..4ba3a63 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -24,6 +24,7 @@
 
 #include <vector>
 #include <string>
+#include <algorithm>
 
 #ifndef _MSC_VER
 #   define BOOST_TEST_DYN_LINK
@@ -53,41 +54,12 @@ using ignite::impl::binary::BinaryUtils;
 struct QueriesTestSuiteFixture 
 {
     /**
-     * Constructor.
+     * Establish connection to node.
+     *
+     * @param connectStr Connection string.
      */
-    QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL)
+    void Connect(const std::string& connectStr)
     {
-        IgniteConfiguration cfg;
-
-        cfg.jvmOpts.push_back("-Xdebug");
-        cfg.jvmOpts.push_back("-Xnoagent");
-        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
-        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
-        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
-
-#ifdef IGNITE_TESTS_32
-        cfg.jvmInitMem = 256;
-        cfg.jvmMaxMem = 768;
-#else
-        cfg.jvmInitMem = 1024;
-        cfg.jvmMaxMem = 4096;
-#endif
-
-        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
-
-        BOOST_REQUIRE(cfgPath != 0);
-
-        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
-
-        IgniteError err;
-
-        grid = Ignition::Start(cfg, &err);
-
-        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-            BOOST_FAIL(err.GetText());
-
-        testCache = grid.GetCache<int64_t, TestType>("cache");
-
         // Allocate an environment handle
         SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
 
@@ -102,13 +74,16 @@ struct QueriesTestSuiteFixture
         BOOST_REQUIRE(dbc != NULL);
 
         // Connect string
-        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
+        std::vector<SQLCHAR> connectStr0;
+
+        connectStr0.reserve(connectStr.size() + 1);
+        std::copy(connectStr.begin(), connectStr.end(), std::back_inserter(connectStr0));
 
         SQLCHAR outstr[ODBC_BUFFER_SIZE];
         SQLSMALLINT outstrlen;
 
         // Connecting to ODBC server.
-        SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast<SQLSMALLINT>(sizeof(connectStr)),
+        SQLRETURN ret = SQLDriverConnect(dbc, NULL, &connectStr0[0], static_cast<SQLSMALLINT>(connectStr0.size()),
             outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
 
         if (!SQL_SUCCEEDED(ret))
@@ -124,10 +99,7 @@ struct QueriesTestSuiteFixture
         BOOST_REQUIRE(stmt != NULL);
     }
 
-    /**
-     * Destructor.
-     */
-    ~QueriesTestSuiteFixture()
+    void Disconnect()
     {
         // Releasing statement handle.
         SQLFreeHandle(SQL_HANDLE_STMT, stmt);
@@ -138,13 +110,67 @@ struct QueriesTestSuiteFixture
         // Releasing allocated handles.
         SQLFreeHandle(SQL_HANDLE_DBC, dbc);
         SQLFreeHandle(SQL_HANDLE_ENV, env);
+    }
+
+    static Ignite StartNode(const char* name, const char* config)
+    {
+        IgniteConfiguration cfg;
 
-        Ignition::Stop(grid.GetName(), true);
+        cfg.jvmOpts.push_back("-Xdebug");
+        cfg.jvmOpts.push_back("-Xnoagent");
+        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+
+#ifdef IGNITE_TESTS_32
+        cfg.jvmInitMem = 256;
+        cfg.jvmMaxMem = 768;
+#else
+        cfg.jvmInitMem = 1024;
+        cfg.jvmMaxMem = 4096;
+#endif
+
+        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
+
+        BOOST_REQUIRE(cfgPath != 0);
+
+        cfg.springCfgPath.assign(cfgPath).append("/").append(config);
+
+        IgniteError err;
+
+        return Ignition::Start(cfg, name);
+    }
+
+    static Ignite StartAdditionalNode(const char* name)
+    {
+        return StartNode(name, "queries-test-noodbc.xml");
+    }
+
+    /**
+     * Constructor.
+     */
+    QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL)
+    {
+        grid = StartNode("NodeMain", "queries-test.xml");
+
+        testCache = grid.GetCache<int64_t, TestType>("cache");
+    }
+
+    /**
+     * Destructor.
+     */
+    ~QueriesTestSuiteFixture()
+    {
+        Disconnect();
+
+        Ignition::StopAll(true);
     }
 
     template<typename T>
     void CheckTwoRowsInt(SQLSMALLINT type)
     {
+        Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
         SQLRETURN ret;
 
         TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456));
@@ -252,6 +278,16 @@ struct QueriesTestSuiteFixture
 
 BOOST_FIXTURE_TEST_SUITE(QueriesTestSuite, QueriesTestSuiteFixture)
 
+BOOST_AUTO_TEST_CASE(TestLegacyConnection)
+{
+    Connect("DRIVER={Apache Ignite};SERVER=127.0.0.1;PORT=11110;CACHE=cache");
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_1_6_0)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=1.6.0");
+}
+
 BOOST_AUTO_TEST_CASE(TestTwoRowsInt8)
 {
     CheckTwoRowsInt<int8_t>(SQL_C_STINYINT);
@@ -294,6 +330,8 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsUint64)
 
 BOOST_AUTO_TEST_CASE(TestTwoRowsString)
 {
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
     SQLRETURN ret;
 
     TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456));
@@ -387,6 +425,8 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString)
 
 BOOST_AUTO_TEST_CASE(TestOneRowString)
 {
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
     SQLRETURN ret;
 
     TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456));
@@ -448,6 +488,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowString)
 
 BOOST_AUTO_TEST_CASE(TestOneRowStringLen)
 {
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");
+
     SQLRETURN ret;
 
     TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456));

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
index 16e5ea0..69b4bfa 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
@@ -45,7 +45,7 @@ namespace ignite
 
         char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
 
-        BOOST_REQUIRE(cfgPath != 0) ;
+        BOOST_REQUIRE(cfgPath != 0);
 
         cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
 
@@ -54,14 +54,14 @@ namespace ignite
         grid = Ignition::Start(cfg, &err);
 
         if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-        BOOST_FAIL(err.GetText()) ;
+            BOOST_FAIL(err.GetText()) ;
 
         testCache = grid.GetCache<int64_t, TestType>("cache");
 
         // Allocate an environment handle
         SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
 
-        BOOST_REQUIRE(env != NULL) ;
+        BOOST_REQUIRE(env != NULL);
 
         // We want ODBC 3 support
         SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
@@ -69,10 +69,10 @@ namespace ignite
         // Allocate a connection handle
         SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
 
-        BOOST_REQUIRE(dbc != NULL) ;
+        BOOST_REQUIRE(dbc != NULL);
 
         // Connect string
-        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
+        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache";
 
         SQLCHAR outstr[ODBC_BUFFER_SIZE];
         SQLSMALLINT outstrlen;
@@ -85,13 +85,13 @@ namespace ignite
         {
             Ignition::Stop(grid.GetName(), true);
 
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ;
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc));
         }
 
         // Allocate a statement handle
         SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
 
-        BOOST_REQUIRE(stmt != NULL) ;
+        BOOST_REQUIRE(stmt != NULL);
     }
 
     SqlTestSuiteFixture::~SqlTestSuiteFixture()

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am
index 29f0ef4..1781bc0 100644
--- a/modules/platforms/cpp/odbc/Makefile.am
+++ b/modules/platforms/cpp/odbc/Makefile.am
@@ -62,6 +62,7 @@ libignite_odbc_la_SOURCES = \
     src/meta/table_meta.cpp \
     src/odbc.cpp \
     src/entry_points.cpp \
+    src/dsn_config.cpp \
     src/query/column_metadata_query.cpp \
     src/query/data_query.cpp \
     src/query/foreign_keys_query.cpp \
@@ -69,6 +70,7 @@ libignite_odbc_la_SOURCES = \
     src/query/table_metadata_query.cpp \
     src/query/type_info_query.cpp \
     src/query/special_columns_query.cpp \
+    src/protocol_version.cpp \
     src/result_page.cpp \
     src/row.cpp \
     src/column.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am
index 192021d..073dcaa 100644
--- a/modules/platforms/cpp/odbc/include/Makefile.am
+++ b/modules/platforms/cpp/odbc/include/Makefile.am
@@ -27,6 +27,7 @@ noinst_HEADERS = \
     ignite/odbc/query/column_metadata_query.h \
     ignite/odbc/query/query.h \
     ignite/odbc/query/primary_keys_query.h \
+    ignite/odbc/protocol_version.h \
     ignite/odbc/statement.h \
     ignite/odbc/config/configuration.h \
     ignite/odbc/config/connection_info.h \
@@ -37,6 +38,7 @@ noinst_HEADERS = \
     ignite/odbc/row.h \
     ignite/odbc/utility.h \
     ignite/odbc/environment.h \
+    ignite/odbc/dsn_config.h \
     ignite/odbc/system/odbc_constants.h \
     ignite/odbc/system/socket_client.h \
     ignite/odbc/meta/primary_key_meta.h \

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index 6636ca4..250eaf2 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -60,6 +60,9 @@ namespace ignite
             /** Output data has been truncated. */
             SQL_STATE_01004_DATA_TRUNCATED,
 
+            /** Invalid connection string attribute. */
+            SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE,
+
             /** Error in row. */
             SQL_STATE_01S01_ERROR_IN_ROW,
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
index d6d7944..f90fa2d 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
@@ -23,6 +23,8 @@
 #include <map>
 
 #include <ignite/common/common.h>
+#include <ignite/common/utils.h>
+#include "ignite/odbc/protocol_version.h"
 
 namespace ignite
 {
@@ -36,6 +38,71 @@ namespace ignite
             class Configuration
             {
             public:
+                /** Map containing connect arguments. */
+                typedef std::map<std::string, std::string> ArgumentMap;
+
+                /** Connection attribute keywords. */
+                struct Key
+                {
+                    /** Connection attribute keyword for DSN attribute. */
+                    static const std::string dsn;
+
+                    /** Connection attribute keyword for Driver attribute. */
+                    static const std::string driver;
+
+                    /** Connection attribute keyword for cache attribute. */
+                    static const std::string cache;
+
+                    /** Connection attribute keyword for address attribute. */
+                    static const std::string address;
+
+                    /** Connection attribute keyword for server attribute. */
+                    static const std::string server;
+
+                    /** Connection attribute keyword for port attribute. */
+                    static const std::string port;
+
+                    /** Connection attribute keyword for protocol version attribute. */
+                    static const std::string protocolVersion;
+                };
+
+                /** Default values for configuration. */
+                struct DefaultValue
+                {
+                    /** Default value for DSN attribute. */
+                    static const std::string dsn;
+
+                    /** Default value for Driver attribute. */
+                    static const std::string driver;
+
+                    /** Default value for cache attribute. */
+                    static const std::string cache;
+
+                    /** Default value for address attribute. */
+                    static const std::string address;
+
+                    /** Default value for server attribute. */
+                    static const std::string server;
+
+                    /** Default value for protocol version. */
+                    static const ProtocolVersion& protocolVersion;
+
+                    /** Default value for port attribute. */
+                    static const uint16_t port;
+                };
+
+                /**
+                 * Connection end point structure.
+                 */
+                struct EndPoint
+                {
+                    /** Remote host. */
+                    std::string host;
+
+                    /** TCP port. */
+                    uint16_t port;
+                };
+
                 /**
                  * Default constructor.
                  */
@@ -81,19 +148,36 @@ namespace ignite
                  *
                  * @return Server port.
                  */
-                uint16_t GetPort() const
+                uint16_t GetTcpPort() const
                 {
-                    return port;
+                    return endPoint.port;
                 }
 
                 /**
+                 * Set server port.
+                 *
+                 * @param port Server port.
+                 */
+                void SetTcpPort(uint16_t port);
+
+                /**
                  * Get DSN.
                  *
                  * @return Data Source Name.
                  */
                 const std::string& GetDsn() const
                 {
-                    return dsn;
+                    return GetStringValue(Key::dsn, DefaultValue::dsn);
+                }
+
+                /**
+                 * Set DSN.
+                 *
+                 * @param dsn Data Source Name.
+                 */
+                void SetDsn(const std::string& dsn)
+                {
+                    arguments[Key::dsn] = dsn;
                 }
 
                 /**
@@ -103,7 +187,7 @@ namespace ignite
                  */
                 const std::string& GetDriver() const
                 {
-                    return driver;
+                    return GetStringValue(Key::driver, DefaultValue::driver);
                 }
 
                 /**
@@ -113,7 +197,17 @@ namespace ignite
                  */
                 const std::string& GetHost() const
                 {
-                    return host;
+                    return endPoint.host;
+                }
+
+                /**
+                 * Set server host.
+                 *
+                 * @param server Server host.
+                 */
+                void SetHost(const std::string& server)
+                {
+                    arguments[Key::server] = server;
                 }
 
                 /**
@@ -123,38 +217,105 @@ namespace ignite
                  */
                 const std::string& GetCache() const
                 {
-                    return cache;
+                    return GetStringValue(Key::cache, DefaultValue::cache);
                 }
 
-            private:
-                IGNITE_NO_COPY_ASSIGNMENT(Configuration);
+                /**
+                 * Set cache.
+                 *
+                 * @param cache Cache name.
+                 */
+                void SetCache(const std::string& cache)
+                {
+                    arguments[Key::cache] = cache;
+                }
 
-                /** Map containing connect arguments. */
-                typedef std::map<std::string, std::string> ArgumentMap;
+                /**
+                 * Get address.
+                 *
+                 * @return Address.
+                 */
+                const std::string& GetAddress() const
+                {
+                    return GetStringValue(Key::address, DefaultValue::address);
+                }
+
+                /**
+                 * Set address.
+                 *
+                 * @param address Address.
+                 */
+                void SetAddress(const std::string& address)
+                {
+                    arguments[Key::address] = address;
+                }
 
                 /**
+                 * Get argument map.
+                 *
+                 * @return Argument map.
+                 */
+                const ArgumentMap& GetMap() const
+                {
+                    return arguments;
+                }
+
+                /**
+                 * Get protocol version.
+                 *
+                 * @return Protocol version.
+                 */
+                ProtocolVersion GetProtocolVersion() const;
+
+                /**
+                 * Set protocol version.
+                 *
+                 * @param version Version to set.
+                 */
+                void SetProtocolVersion(const std::string& version);
+
+                /**
+                 * Get string value from the config.
+                 *
+                 * @param key Configuration key.
+                 * @param dflt Default value to be returned if there is no value stored.
+                 * @return Found or default value.
+                 */
+                const std::string& GetStringValue(const std::string& key, const std::string& dflt) const;
+
+                /**
+                 * Get int value from the config.
+                 *
+                 * @param key Configuration key.
+                 * @param dflt Default value to be returned if there is no value stored.
+                 * @return Found or default value.
+                 */
+                int64_t GetIntValue(const std::string& key, int64_t dflt) const;
+
+            private:
+                /**
                  * Parse connect string into key-value storage.
                  *
                  * @param str String to parse.
                  * @param len String length.
                  * @param params Parsing result.
                  */
-                void ParseAttributeList(const char* str, size_t len, char delimeter, ArgumentMap& args) const;
+                static void ParseAttributeList(const char* str, size_t len, char delimeter, ArgumentMap& args);
 
-                /** Data Source Name. */
-                std::string dsn;
-
-                /** Driver name. */
-                std::string driver;
-
-                /** Server hostname. */
-                std::string host;
+                /**
+                 * Parse address and extract connection end-point.
+                 *
+                 * @throw IgniteException if address can not be parsed.
+                 * @param address Address string to parse.
+                 * @param res Result is placed here.
+                 */
+                static void ParseAddress(const std::string& address, EndPoint& res);
 
-                /** Port of the server. */
-                uint16_t port;
+                /** Arguments. */
+                ArgumentMap arguments;
 
-                /** Cache name. */
-                std::string cache;
+                /** Connection end-point. */
+                EndPoint endPoint;
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
index 10ceb19..acf82ba 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h
@@ -25,6 +25,7 @@
 #include "ignite/odbc/parser.h"
 #include "ignite/odbc/system/socket_client.h"
 #include "ignite/odbc/config/connection_info.h"
+#include "ignite/odbc/config/configuration.h"
 #include "ignite/odbc/diagnostic/diagnosable_adapter.h"
 
 namespace ignite
@@ -40,15 +41,6 @@ namespace ignite
         {
             friend class Environment;
         public:
-            /** ODBC communication protocol version. */
-            enum { PROTOCOL_VERSION = 1 };
-
-            /**
-             * Apache Ignite version when the current ODBC communication
-             * protocol version has been introduced.
-             */
-            static const std::string PROTOCOL_VERSION_SINCE;
-
             /**
              * Destructor.
              */
@@ -74,18 +66,16 @@ namespace ignite
             /**
              * Establish connection to ODBC server.
              *
-             * @param server Server (DSN).
+             * @param connectStr Connection string.
              */
-            void Establish(const std::string& server);
+            void Establish(const std::string& connectStr);
 
             /**
              * Establish connection to ODBC server.
              *
-             * @param host Host.
-             * @param port Port.
-             * @param cache Cache name to connect to.
+             * @param cfg Configuration.
              */
-            void Establish(const std::string& host, uint16_t port, const std::string& cache);
+            void Establish(const config::Configuration cfg);
 
             /**
              * Release established connection.
@@ -124,6 +114,13 @@ namespace ignite
             const std::string& GetCache() const;
 
             /**
+             * Get configuration.
+             *
+             * @return Connection configuration.
+             */
+            const config::Configuration& GetConfiguration() const;
+
+            /**
              * Create diagnostic record associated with the Connection instance.
              *
              * @param sqlState SQL state.
@@ -132,8 +129,8 @@ namespace ignite
              * @param columnNum Associated column number.
              * @return DiagnosticRecord associated with the instance.
              */
-            diagnostic::DiagnosticRecord CreateStatusRecord(SqlState sqlState,
-                const std::string& message, int32_t rowNum = 0, int32_t columnNum = 0) const;
+            static diagnostic::DiagnosticRecord CreateStatusRecord(SqlState sqlState,
+                const std::string& message, int32_t rowNum = 0, int32_t columnNum = 0);
 
             /**
              * Synchronously send request message and receive response.
@@ -172,21 +169,19 @@ namespace ignite
              * Establish connection to ODBC server.
              * Internal call.
              *
-             * @param server Server (DNS).
+             * @param connectStr Connection string.
              * @return Operation result.
              */
-            SqlResult InternalEstablish(const std::string& server);
+            SqlResult InternalEstablish(const std::string& connectStr);
 
             /**
              * Establish connection to ODBC server.
              * Internal call.
              *
-             * @param host Host.
-             * @param port Port.
-             * @param cache Cache name to connect to.
+             * @param cfg Configuration.
              * @return Operation result.
              */
-            SqlResult InternalEstablish(const std::string& host, uint16_t port, const std::string& cache);
+            SqlResult InternalEstablish(const config::Configuration cfg);
 
             /**
              * Release established connection.
@@ -269,11 +264,11 @@ namespace ignite
             /** State flag. */
             bool connected;
 
-            /** Cache name. */
-            std::string cache;
-
             /** Message parser. */
             Parser parser;
+
+            /** Configuration. */
+            config::Configuration config;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
index bfb4f4c..670e0aa 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
@@ -84,7 +84,7 @@ namespace ignite
                  *
                  * @return An informational message on the error or warning.
                  */
-                const std::string& GetMessage() const;
+                const std::string& GetMessageText() const;
 
                 /**
                  * Get connection name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h
new file mode 100644
index 0000000..dbad9b5
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_DSN_CONFIG
+#define _IGNITE_ODBC_DSN_CONFIG
+
+#include "ignite/odbc/config/configuration.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        /**
+         * Extract last setup error and throw it like IgniteError.
+         */
+        void ThrowLastSetupError();
+
+        /**
+         * Add new string to the DSN file.
+         *
+         * @param dsn DSN name.
+         * @param key Key.
+         * @param value Value.
+         */
+        void WriteDsnString(const char* dsn, const char* key, const char* value);
+
+        /**
+         * Get string from the DSN file.
+         *
+         * @param dsn DSN name.
+         * @param key Key.
+         * @param dflt Default value.
+         * @return Value.
+         */
+        std::string ReadDsnString(const char* dsn, const char* key, const char* dflt);
+
+        /**
+         * Read DSN to fill the configuration.
+         *
+         * @param dsn DSN name.
+         * @param config Configuration.
+         */
+        void ReadDsnConfiguration(const char* dsn, config::Configuration& config);
+    }
+}
+
+#endif //_IGNITE_ODBC_DSN_CONFIG
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
index c19e08c..a91af22 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h
@@ -42,9 +42,6 @@ namespace ignite
             /** Default initial size of operational memory. */
             enum { DEFAULT_MEM_ALLOCATION = 4096 };
 
-            /** ODBC communication protocol version. */
-            enum { PROTOCOL_VERSION = 1 };
-
             /**
              * Constructor.
              */

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
new file mode 100644
index 0000000..8682119
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
@@ -0,0 +1,168 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_PROTOCOL_VERSION
+#define _IGNITE_ODBC_PROTOCOL_VERSION
+
+#include <stdint.h>
+
+#include <string>
+#include <map>
+
+namespace ignite
+{
+    namespace odbc
+    {
+        /** Protocol version. */
+        class ProtocolVersion
+        {
+        public:
+            /** String to version map type alias. */
+            typedef std::map<std::string, ProtocolVersion> StringToVersionMap;
+
+            /** Version to string map type alias. */
+            typedef std::map<ProtocolVersion, std::string> VersionToStringMap;
+
+            /** First version of the protocol that was introduced in Ignite 1.6.0. */
+            static const ProtocolVersion VERSION_1_6_0;
+
+            /** Unknown version of the protocol. */
+            static const ProtocolVersion VERSION_UNKNOWN;
+
+            /**
+             * Get string to version map.
+             *
+             * @return String to version map.
+             */
+            static const StringToVersionMap& GetMap();
+
+            /**
+             * Get current version.
+             *
+             * @return Current version.
+             */
+            static const ProtocolVersion& GetCurrent();
+
+            /**
+             * Parse string and extract protocol version.
+             *
+             * @throw IgniteException if version can not be parsed.
+             * @param version Version string to parse.
+             * @return Protocol version.
+             */
+            static ProtocolVersion FromString(const std::string& version);
+
+            /**
+             * Convert to string value.
+             *
+             * @throw IgniteException if version is unknow parsed.
+             * @param version Version string to parse.
+             * @return Protocol version.
+             */
+            const std::string& ToString() const;
+
+            /**
+             * Get int value.
+             *
+             * @return Integer value.
+             */
+            int64_t GetIntValue() const;
+
+            /**
+             * Check if the version is unknown.
+             *
+             * @return True if the version is unknown.
+             */
+            bool IsUnknown() const;
+
+            /**
+             * Comparison operator.
+             *
+             * @param val1 First value.
+             * @param val2 Second value.
+             * @return True if equal.
+             */
+            friend bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2);
+
+            /**
+             * Comparison operator.
+             *
+             * @param val1 First value.
+             * @param val2 Second value.
+             * @return True if not equal.
+             */
+            friend bool operator!=(const ProtocolVersion& val1, const ProtocolVersion& val2);
+
+            /**
+             * Comparison operator.
+             *
+             * @param val1 First value.
+             * @param val2 Second value.
+             * @return True if less.
+             */
+            friend bool operator<(const ProtocolVersion& val1, const ProtocolVersion& val2);
+
+            /**
+             * Comparison operator.
+             *
+             * @param val1 First value.
+             * @param val2 Second value.
+             * @return True if less or equal.
+             */
+            friend bool operator<=(const ProtocolVersion& val1, const ProtocolVersion& val2);
+
+            /**
+             * Comparison operator.
+             *
+             * @param val1 First value.
+             * @param val2 Second value.
+             * @return True if gretter.
+             */
+            friend bool operator>(const ProtocolVersion& val1, const ProtocolVersion& val2);
+
+            /**
+             * Comparison operator.
+             *
+             * @param val1 First value.
+             * @param val2 Second value.
+             * @return True if gretter or equal.
+             */
+            friend bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2);
+
+        private:
+            /**
+             * Constructor.
+             *
+             * @param val Underlying value.
+             */
+            explicit ProtocolVersion(int64_t val);
+
+            ProtocolVersion();
+
+            /** String to version map. */
+            static const StringToVersionMap stringToVersionMap;
+
+            /** Version to string map. */
+            static const VersionToStringMap versionToStringMap;
+
+            /** Underlying int value. */
+            int64_t val;
+        };
+    }
+}
+
+#endif //_IGNITE_ODBC_PROTOCOL_VERSION
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
index 60a6552..ecd1a55 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
@@ -27,10 +27,6 @@
 #   undef min
 #endif // min
 
-#ifdef GetMessage
-#   undef GetMessage
-#endif // GetMessage
-
 #endif //_WIN32
 
 #define ODBCVER 0x0380

http://git-wip-us.apache.org/repos/asf/ignite/blob/70e69cb7/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
new file mode 100644
index 0000000..034de82
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
@@ -0,0 +1,136 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW
+#define _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW
+
+#include "ignite/odbc/config/configuration.h"
+#include "ignite/odbc/system/ui/custom_window.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                /**
+                 * DSN configuration window class.
+                 */
+                class DsnConfigurationWindow : public CustomWindow
+                {
+                    /**
+                     * Children windows ids.
+                     */
+                    enum ChildId
+                    {
+                        ID_CONNECTION_SETTINGS_GROUP_BOX,
+                        ID_NAME_EDIT,
+                        ID_NAME_LABEL,
+                        ID_ADDRESS_EDIT,
+                        ID_ADDRESS_LABEL,
+                        ID_CACHE_EDIT,
+                        ID_CACHE_LABEL,
+                        ID_OK_BUTTON,
+                        ID_CANCEL_BUTTON
+                    };
+
+                public:
+                    /**
+                     * Constructor.
+                     *
+                     * @param parent Parent window handle.
+                     */
+                    explicit DsnConfigurationWindow(Window* parent, config::Configuration& config);
+
+                    /**
+                     * Destructor.
+                     */
+                    virtual ~DsnConfigurationWindow();
+
+                    /**
+                     * Create window in the center of the parent window.
+                     */
+                    void Create();
+
+                    /**
+                    * @copedoc ignite::odbc::system::ui::CustomWindow::OnCreate
+                    */
+                    virtual void OnCreate();
+
+                    /**
+                     * @copedoc ignite::odbc::system::ui::CustomWindow::OnMessage
+                     */
+                    virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam);
+
+                private:
+                    IGNITE_NO_COPY_ASSIGNMENT(DsnConfigurationWindow)
+
+                    /**
+                     * Retrieves current values from the children and stores
+                     * them to the specified configuration.
+                     *
+                     * @param cfg Configuration.
+                     */
+                    void RetrieveParameters(config::Configuration& cfg) const;
+
+                    /** Window width. */
+                    int width;
+
+                    /** Window height. */
+                    int height;
+
+                    /** Connection settings group box. */
+                    std::auto_ptr<Window> connectionSettingsGroupBox;
+
+                    /** DSN name edit field label. */
+                    std::auto_ptr<Window> nameLabel;
+
+                    /** DSN name edit field. */
+                    std::auto_ptr<Window> nameEdit;
+
+                    /** DSN address edit field label. */
+                    std::auto_ptr<Window> addressLabel;
+
+                    /** DSN address edit field. */
+                    std::auto_ptr<Window> addressEdit;
+
+                    /** DSN cache edit field label. */
+                    std::auto_ptr<Window> cacheLabel;
+
+                    /** DSN cache edit field. */
+                    std::auto_ptr<Window> cacheEdit;
+
+                    /** Ok button. */
+                    std::auto_ptr<Window> okButton;
+
+                    /** Cancel button. */
+                    std::auto_ptr<Window> cancelButton;
+
+                    /** Configuration. */
+                    config::Configuration& config;
+
+                    /** Flag indicating whether OK option was selected. */
+                    bool accepted;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW
\ No newline at end of file