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 2017/05/05 13:57:32 UTC

ignite git commit: IGNITE-5169: ODBC: reworked handshake semantics to support JDBC driver integration. This closes #1908.

Repository: ignite
Updated Branches:
  refs/heads/master cc2a6badb -> dddc6d4a4


IGNITE-5169: ODBC: reworked handshake semantics to support JDBC driver integration. This closes #1908.


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

Branch: refs/heads/master
Commit: dddc6d4a4719beaa3d73a62a8b91ee0a51c5392d
Parents: cc2a6ba
Author: devozerov <vo...@gridgain.com>
Authored: Fri May 5 16:57:24 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri May 5 16:57:24 2017 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcNioListener.java        | 112 +++++++++++++--
 .../internal/processors/odbc/OdbcUtils.java     |   3 -
 .../odbc/SqlListenerHandshakeRequest.java       |  83 -----------
 .../odbc/SqlListenerHandshakeResult.java        |  73 ----------
 .../odbc/SqlListenerProtocolVersion.java        | 110 +++++++++------
 .../processors/odbc/odbc/OdbcMessageParser.java |  43 ------
 .../odbc/odbc/OdbcRequestHandler.java           |  65 +++------
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   1 +
 .../project/vs/odbc-test.vcxproj.filters        |   4 +
 .../cpp/odbc-test/src/configuration_test.cpp    |  39 +++++-
 .../cpp/odbc-test/src/queries_test.cpp          |   4 +-
 .../cpp/odbc/include/ignite/odbc/message.h      |  45 +++---
 .../odbc/include/ignite/odbc/protocol_version.h | 101 ++++++++------
 .../src/system/ui/dsn_configuration_window.cpp  |  10 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |  35 ++---
 .../platforms/cpp/odbc/src/protocol_version.cpp | 136 ++++++++++++-------
 16 files changed, 424 insertions(+), 440 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
index 423c413..cdb3de3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
@@ -19,6 +19,11 @@ package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
+import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
+import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcMessageParser;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcRequestHandler;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -27,12 +32,20 @@ import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.HashSet;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * ODBC message listener.
  */
 public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
+    /** Current version. */
+    private static final SqlListenerProtocolVersion CURRENT_VER = SqlListenerProtocolVersion.create(2, 1, 0);
+
+    /** Supported versions. */
+    private static final Set<SqlListenerProtocolVersion> SUPPORTED_VERS = new HashSet<>();
+
     /** Connection-related metadata key. */
     private static final int CONN_CTX_META_KEY = GridNioSessionMetaKey.nextUniqueKey();
 
@@ -51,6 +64,10 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
     /** Logger. */
     private final IgniteLogger log;
 
+    static {
+        SUPPORTED_VERS.add(CURRENT_VER);
+    }
+
     /**
      * Constructor.
      *
@@ -70,11 +87,6 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
     @Override public void onConnected(GridNioSession ses) {
         if (log.isDebugEnabled())
             log.debug("SQL client connected: " + ses.remoteAddress());
-
-        OdbcRequestHandler handler = new OdbcRequestHandler(ctx, busyLock, maxCursors);
-        OdbcMessageParser parser = new OdbcMessageParser(ctx);
-
-        ses.addMeta(CONN_CTX_META_KEY, new SqlListenerConnectionContext(handler, parser));
     }
 
     /** {@inheritDoc} */
@@ -91,11 +103,15 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
     @Override public void onMessage(GridNioSession ses, byte[] msg) {
         assert msg != null;
 
-        SqlListenerConnectionContext connData = ses.meta(CONN_CTX_META_KEY);
+        SqlListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY);
+
+        if (connCtx == null) {
+            onHandshake(ses, msg);
 
-        assert connData != null;
+            return;
+        }
 
-        SqlListenerMessageParser parser = connData.parser();
+        SqlListenerMessageParser parser = connCtx.parser();
 
         SqlListenerRequest req;
 
@@ -124,7 +140,7 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
                     ", req=" + req + ']');
             }
 
-            SqlListenerRequestHandler handler = connData.handler();
+            SqlListenerRequestHandler handler = connCtx.handler();
 
             SqlListenerResponse resp = handler.handle(req);
 
@@ -145,4 +161,82 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
             ses.send(parser.encode(new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString())));
         }
     }
+
+    /**
+     * Perform handshake.
+     *
+     * @param ses Session.
+     * @param msg Message bytes.
+     */
+    private void onHandshake(GridNioSession ses, byte[] msg) {
+        BinaryInputStream stream = new BinaryHeapInputStream(msg);
+
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null, true);
+
+        byte cmd = reader.readByte();
+
+        if (cmd != SqlListenerRequest.HANDSHAKE) {
+            log.error("Unexpected SQL client request (will close session): " + ses.remoteAddress());
+
+            ses.close();
+
+            return;
+        }
+
+        short verMajor = reader.readShort();
+        short verMinor = reader.readShort();
+        short verMaintenance = reader.readShort();
+
+        SqlListenerProtocolVersion ver = SqlListenerProtocolVersion.create(verMajor, verMinor, verMaintenance);
+
+        String errMsg = null;
+
+        if (SUPPORTED_VERS.contains(ver)) {
+            // Prepare context.
+            SqlListenerConnectionContext connCtx = prepareContext(ver, reader);
+
+            ses.addMeta(CONN_CTX_META_KEY, connCtx);
+        }
+        else {
+            log.warning("Unsupported version: " + ver.toString());
+
+            errMsg = "Unsupported version.";
+        }
+
+        // Send response.
+        BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(8), null, null);
+
+        if (errMsg == null) {
+            writer.writeBoolean(true);
+        }
+        else {
+            writer.writeBoolean(false);
+            writer.writeShort(CURRENT_VER.major());
+            writer.writeShort(CURRENT_VER.minor());
+            writer.writeShort(CURRENT_VER.maintenance());
+            writer.doWriteString(errMsg);
+        }
+
+        ses.send(writer.array());
+    }
+
+    /**
+     * Prepare context.
+     *
+     * @param ver Version.
+     * @param reader Reader.
+     * @return Context.
+     */
+    private SqlListenerConnectionContext prepareContext(SqlListenerProtocolVersion ver, BinaryReaderExImpl reader) {
+        // TODO: Switch between ODBC and JDBC.
+        boolean distributedJoins = reader.readBoolean();
+        boolean enforceJoinOrder = reader.readBoolean();
+
+        OdbcRequestHandler handler =
+            new OdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder);
+
+        OdbcMessageParser parser = new OdbcMessageParser(ctx);
+
+        return new SqlListenerConnectionContext(handler, parser);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
index da56813..4b046e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
@@ -23,9 +23,6 @@ import org.apache.ignite.IgniteException;
  * Various ODBC utility methods.
  */
 public class OdbcUtils {
-    /** Latest version. */
-    public static final SqlListenerProtocolVersion VER_LATEST = SqlListenerProtocolVersion.VER_2_0_0;
-
     /**
      * Add quotation marks at the beginning and end of the string.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeRequest.java
deleted file mode 100644
index 09ac05a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeRequest.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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 org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * ODBC handshake request.
- */
-public class SqlListenerHandshakeRequest extends SqlListenerRequest {
-    /** Protocol version. */
-    private final SqlListenerProtocolVersion ver;
-
-    /** Distributed joins flag. */
-    private boolean distributedJoins = false;
-
-    /** Enforce join order flag. */
-    private boolean enforceJoinOrder = false;
-
-    /**
-     * @param ver Long value for protocol version.
-     */
-    public SqlListenerHandshakeRequest(long ver) {
-        super(HANDSHAKE);
-
-        this.ver = SqlListenerProtocolVersion.fromLong(ver);
-    }
-
-    /**
-     * @return Protocol version.
-     */
-    public SqlListenerProtocolVersion version() {
-        return ver;
-    }
-
-    /**
-     * @return Distributed joins flag.
-     */
-    public boolean distributedJoins() {
-        return distributedJoins;
-    }
-
-    /**
-     * @param distributedJoins Distributed joins flag.
-     */
-    public void distributedJoins(boolean distributedJoins) {
-        this.distributedJoins = distributedJoins;
-    }
-
-    /**
-     * @return Enforce join order flag.
-     */
-    public boolean enforceJoinOrder() {
-        return enforceJoinOrder;
-    }
-
-    /**
-     * @param enforceJoinOrder Enforce join order flag.
-     */
-    public void enforceJoinOrder(boolean enforceJoinOrder) {
-        this.enforceJoinOrder = enforceJoinOrder;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(SqlListenerHandshakeRequest.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeResult.java
deleted file mode 100644
index ca8b629..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeResult.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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 org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * ODBC handshake result.
- */
-public class SqlListenerHandshakeResult {
-    /** Handshake accepted. */
-    private final boolean accepted;
-
-    /** Apache Ignite version when protocol version has been introduced. */
-    private final String protoVerSince;
-
-    /** Current Apache Ignite version. */
-    private final String curVer;
-
-    /**
-     * 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 SqlListenerHandshakeResult(boolean accepted, String protoVerSince, String curVer) {
-        this.accepted = accepted;
-        this.protoVerSince = protoVerSince;
-        this.curVer = curVer;
-    }
-
-    /**
-     * @return Query ID.
-     */
-    public boolean accepted() {
-        return accepted;
-    }
-
-    /**
-     * @return Apache Ignite version when protocol version has been introduced.
-     */
-    public String protocolVersionSince() {
-        return protoVerSince;
-    }
-
-    /**
-     * @return Current Apache Ignite version.
-     */
-    public String currentVersion() {
-        return curVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(SqlListenerHandshakeResult.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProtocolVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProtocolVersion.java
index 62e07ee..690822b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProtocolVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProtocolVersion.java
@@ -17,72 +17,100 @@
 
 package org.apache.ignite.internal.processors.odbc;
 
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.NotNull;
+
 /**
  * SQL listener protocol version.
  */
-public enum SqlListenerProtocolVersion {
-    /** Version 2.0.0. */
-    VER_2_0_0(makeVersion(2, 0, 0), "2.0.0"),
-
-    /** Unknown version. */
-    UNKNOWN(Long.MIN_VALUE, "UNKNOWN");
-
-    /** Mask to get 2 lowest bytes of the value and cast to long. */
-    private static final long LONG_MASK = 0x000000000000FFFFL;
+public class SqlListenerProtocolVersion implements Comparable<SqlListenerProtocolVersion> {
+    /** Major part. */
+    private final short major;
 
-    /** Long value for version. */
-    private final long longVal;
+    /** Minor part. */
+    private final short minor;
 
-    /** Since string. */
-    private final String since;
+    /** Maintenance part. */
+    private final short maintenance;
 
     /**
-     * Constructor.
+     * Create version.
      *
-     * @param longVal Long value.
-     * @param since Since string.
+     * @param major Major part.
+     * @param minor Minor part.
+     * @param maintenance Maintenance part.
+     * @return Version.
      */
-    SqlListenerProtocolVersion(long longVal, String since) {
-        this.longVal = longVal;
-        this.since = since;
+    public static SqlListenerProtocolVersion create(int major, int minor, int maintenance) {
+        return new SqlListenerProtocolVersion((short)major, (short)minor, (short)maintenance);
     }
 
     /**
-     * Make long value for the version.
+     * Constructor.
      *
-     * @param major Major version.
-     * @param minor Minor version.
-     * @param maintenance Maintenance version.
-     * @return Long value for the version.
+     * @param major Major part.
+     * @param minor Minor part.
+     * @param maintenance Maintenance part.
      */
-    private static long makeVersion(int major, int minor, int maintenance) {
-        return ((major & LONG_MASK) << 48) | ((minor & LONG_MASK) << 32) | ((maintenance & LONG_MASK) << 16);
+    private SqlListenerProtocolVersion(short major, short minor, short maintenance) {
+        this.major = major;
+        this.minor = minor;
+        this.maintenance = maintenance;
     }
 
     /**
-     * @param longVal Long value.
-     * @return Protocol version.
+     * @return Major part.
      */
-    public static SqlListenerProtocolVersion fromLong(long longVal) {
-        for (SqlListenerProtocolVersion ver : SqlListenerProtocolVersion.values()) {
-            if (ver.longValue() == longVal)
-                return ver;
-        }
-
-        return UNKNOWN;
+    public short major() {
+        return major;
     }
 
     /**
-     * @return Long value.
+     * @return Minor part.
      */
-    public long longValue() {
-        return longVal;
+    public short minor() {
+        return minor;
     }
 
     /**
-     * @return Ignite version when introduced.
+     * @return Maintenance part.
      */
-    public String since() {
-        return since;
+    public short maintenance() {
+        return maintenance;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(@NotNull SqlListenerProtocolVersion other) {
+        int res = major - other.major;
+
+        if (res == 0)
+            res = minor - other.minor;
+
+        if (res == 0)
+            res = maintenance - other.maintenance;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return 31 * (31 * major + minor) + maintenance;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (obj != null && obj instanceof SqlListenerProtocolVersion) {
+            SqlListenerProtocolVersion other = (SqlListenerProtocolVersion)obj;
+
+            return F.eq(major, other.major) && F.eq(minor, other.minor) && F.eq(maintenance, other.maintenance);
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SqlListenerProtocolVersion.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
index 37fe3bf..af595b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
@@ -27,8 +27,6 @@ import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerHandshakeRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerHandshakeResult;
 import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaRequest;
 import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaResult;
 import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaRequest;
@@ -62,9 +60,6 @@ public class OdbcMessageParser implements SqlListenerMessageParser {
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Protocol version confirmation flag. */
-    private boolean verConfirmed = false;
-
     /**
      * @param ctx Context.
      */
@@ -86,25 +81,6 @@ public class OdbcMessageParser implements SqlListenerMessageParser {
 
         byte cmd = reader.readByte();
 
-        // This is a special case because we can not decode protocol messages until
-        // we has not confirmed that the remote client uses the same protocol version.
-        if (!verConfirmed) {
-            if (cmd == SqlListenerRequest.HANDSHAKE)
-            {
-                long longVersion = reader.readLong();
-
-                SqlListenerHandshakeRequest res = new SqlListenerHandshakeRequest(longVersion);
-
-                res.distributedJoins(reader.readBoolean());
-                res.enforceJoinOrder(reader.readBoolean());
-
-                return res;
-            }
-            else
-                throw new IgniteException("Unexpected ODBC command " +
-                        "(first message is not a handshake request): [cmd=" + cmd + ']');
-        }
-
         SqlListenerRequest res;
 
         switch (cmd) {
@@ -197,25 +173,6 @@ public class OdbcMessageParser implements SqlListenerMessageParser {
 
         if (res0 == null)
             return writer.array();
-        if (res0 instanceof SqlListenerHandshakeResult) {
-            SqlListenerHandshakeResult res = (SqlListenerHandshakeResult) res0;
-
-            if (log.isDebugEnabled())
-                log.debug("Handshake result: " + (res.accepted() ? "accepted" : "rejected"));
-
-            verConfirmed = res.accepted();
-
-            if (res.accepted()) {
-                verConfirmed = true;
-
-                writer.writeBoolean(true);
-            }
-            else {
-                writer.writeBoolean(false);
-                writer.writeString(res.protocolVersionSince());
-                writer.writeString(res.currentVersion());
-            }
-        }
         else if (res0 instanceof SqlListenerQueryExecuteResult) {
             SqlListenerQueryExecuteResult res = (SqlListenerQueryExecuteResult) res0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index 815f650..eabc486 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -24,8 +24,6 @@ import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerHandshakeRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerHandshakeResult;
 import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaRequest;
 import org.apache.ignite.internal.processors.odbc.OdbcQueryGetColumnsMetaResult;
 import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaRequest;
@@ -35,7 +33,6 @@ import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaResult;
 import org.apache.ignite.internal.processors.odbc.OdbcTableMeta;
 import org.apache.ignite.internal.processors.odbc.OdbcUtils;
 import org.apache.ignite.internal.processors.odbc.SqlListenerColumnMeta;
-import org.apache.ignite.internal.processors.odbc.SqlListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseRequest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseResult;
 import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteRequest;
@@ -52,16 +49,24 @@ import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteProductVersion;
 
 import java.sql.ParameterMetaData;
 import java.sql.PreparedStatement;
 import java.sql.Types;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.*;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_COLS;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_PARAMS;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.META_TBLS;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_CLOSE;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_EXEC;
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.QRY_FETCH;
 
 /**
  * SQL query handler.
@@ -86,10 +91,10 @@ public class OdbcRequestHandler implements SqlListenerRequestHandler {
     private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCursors = new ConcurrentHashMap<>();
 
     /** Distributed joins flag. */
-    private boolean distributedJoins = false;
+    private final boolean distributedJoins;
 
     /** Enforce join order flag. */
-    private boolean enforceJoinOrder = false;
+    private final boolean enforceJoinOrder;
 
     /**
      * Constructor.
@@ -97,11 +102,16 @@ public class OdbcRequestHandler implements SqlListenerRequestHandler {
      * @param ctx Context.
      * @param busyLock Shutdown latch.
      * @param maxCursors Maximum allowed cursors.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce join order flag.
      */
-    public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) {
+    public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors,
+        boolean distributedJoins, boolean enforceJoinOrder) {
         this.ctx = ctx;
         this.busyLock = busyLock;
         this.maxCursors = maxCursors;
+        this.distributedJoins = distributedJoins;
+        this.enforceJoinOrder = enforceJoinOrder;
 
         log = ctx.log(getClass());
     }
@@ -116,9 +126,6 @@ public class OdbcRequestHandler implements SqlListenerRequestHandler {
 
         try {
             switch (req.command()) {
-                case HANDSHAKE:
-                    return performHandshake((SqlListenerHandshakeRequest)req);
-
                 case QRY_EXEC:
                     return executeQuery((SqlListenerQueryExecuteRequest)req);
 
@@ -146,40 +153,6 @@ public class OdbcRequestHandler implements SqlListenerRequestHandler {
     }
 
     /**
-     * {@link SqlListenerHandshakeRequest} command handler.
-     *
-     * @param req Handshake request.
-     * @return Response.
-     */
-    private SqlListenerResponse performHandshake(SqlListenerHandshakeRequest req) {
-        try {
-            SqlListenerProtocolVersion version = req.version();
-
-            if (version == SqlListenerProtocolVersion.UNKNOWN) {
-                IgniteProductVersion ver = ctx.grid().version();
-
-                String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance();
-
-                SqlListenerHandshakeResult res = new SqlListenerHandshakeResult(false, OdbcUtils.VER_LATEST.since(), verStr);
-
-                return new SqlListenerResponse(res);
-            }
-
-            SqlListenerHandshakeResult res = new SqlListenerHandshakeResult(true, null, null);
-
-            distributedJoins = req.distributedJoins();
-            enforceJoinOrder = req.enforceJoinOrder();
-
-            return new SqlListenerResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to perform handshake [reqId=" + req.requestId() + ", req=" + req + ']', e);
-
-            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
      * {@link SqlListenerQueryExecuteRequest} command handler.
      *
      * @param req Execute query request.

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/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 7e9c738..c332aad 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\log.cpp" />
     <ClCompile Include="..\..\..\odbc\src\protocol_version.cpp" />
     <ClCompile Include="..\..\..\odbc\src\result_page.cpp" />
     <ClCompile Include="..\..\..\odbc\src\row.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/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 6efc4b4..a344df9 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
@@ -118,6 +118,9 @@
     <ClCompile Include="..\..\src\api_robustness_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\..\odbc\src\log.cpp">
+      <Filter>Externals</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">
@@ -152,5 +155,6 @@
     <None Include="..\..\config\queries-default.xml">
       <Filter>Configs</Filter>
     </None>
+    <None Include="..\..\config\queries-test-default.xml" />
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/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 d182f93..ecf3c4b 100644
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@ -62,6 +62,15 @@ void CheckValidProtocolVersion(const char* connectStr, ignite::odbc::ProtocolVer
     BOOST_CHECK(cfg.GetProtocolVersion() == version);
 }
 
+void CheckSupportedProtocolVersion(const char* connectStr)
+{
+    Configuration cfg;
+
+    BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr));
+
+    BOOST_CHECK(cfg.GetProtocolVersion().IsSupported());
+}
+
 void CheckInvalidProtocolVersion(const char* connectStr)
 {
     Configuration cfg;
@@ -71,6 +80,15 @@ void CheckInvalidProtocolVersion(const char* connectStr)
     BOOST_CHECK_THROW(cfg.GetProtocolVersion(), ignite::IgniteError);
 }
 
+void CheckUnsupportedProtocolVersion(const char* connectStr)
+{
+    Configuration cfg;
+
+    cfg.FillFromConnectString(connectStr);
+
+    BOOST_CHECK(!cfg.GetProtocolVersion().IsSupported());
+}
+
 void CheckValidBoolValue(const std::string& connectStr, const std::string& key, bool val)
 {
     Configuration cfg;
@@ -269,14 +287,27 @@ 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;");
+    CheckInvalidProtocolVersion("Protocol_Version=2.1;");
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectStringUnsupportedVersion)
+{
+    CheckUnsupportedProtocolVersion("Protocol_Version=1.6.1;");
+    CheckUnsupportedProtocolVersion("Protocol_Version=1.7.0;");
+    CheckUnsupportedProtocolVersion("Protocol_Version=1.8.1;");
 }
 
 BOOST_AUTO_TEST_CASE(TestConnectStringValidVersion)
 {
-    CheckValidProtocolVersion("Protocol_Version=2.0.0;", ignite::odbc::ProtocolVersion::VERSION_2_0_0);
+    CheckValidProtocolVersion("Protocol_Version=2.1.0;", ignite::odbc::ProtocolVersion::VERSION_2_1_0);
+    CheckValidProtocolVersion("Protocol_Version=1.6.1;", ignite::odbc::ProtocolVersion(1, 6, 1));
+    CheckValidProtocolVersion("Protocol_Version=1.7.0;", ignite::odbc::ProtocolVersion(1, 7, 0));
+    CheckValidProtocolVersion("Protocol_Version=1.8.1;", ignite::odbc::ProtocolVersion(1, 8, 1));
+}
+
+BOOST_AUTO_TEST_CASE(TestConnectStringSupportedVersion)
+{
+    CheckSupportedProtocolVersion("Protocol_Version=2.1.0;");
 }
 
 BOOST_AUTO_TEST_CASE(TestConnectStringInvalidBoolKeys)

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/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 0f9eaee..4f7a6e2 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -378,9 +378,9 @@ BOOST_AUTO_TEST_CASE(TestLegacyConnection)
     Connect("DRIVER={Apache Ignite};SERVER=127.0.0.1;PORT=11110;CACHE=cache");
 }
 
-BOOST_AUTO_TEST_CASE(TestConnectionProtocolVERSION_2_0_0)
+BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_2_1_0)
 {
-    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=2.0.0");
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=2.1.0");
 }
 
 BOOST_AUTO_TEST_CASE(TestTwoRowsInt8)

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
index 60b2d9b..aca214a 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h
@@ -77,7 +77,7 @@ namespace ignite
              * @param distributedJoins Distributed joins flag.
              * @param enforceJoinOrder Enforce join order flag.
              */
-            HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder) :
+            HandshakeRequest(const ProtocolVersion& version, bool distributedJoins, bool enforceJoinOrder) :
                 version(version),
                 distributedJoins(distributedJoins),
                 enforceJoinOrder(enforceJoinOrder)
@@ -101,7 +101,9 @@ namespace ignite
             {
                 writer.WriteInt8(RequestType::HANDSHAKE);
 
-                writer.WriteInt64(version);
+                writer.WriteInt16(version.GetMajor());
+                writer.WriteInt16(version.GetMinor());
+                writer.WriteInt16(version.GetMaintenance());
 
                 writer.WriteBool(distributedJoins);
                 writer.WriteBool(enforceJoinOrder);
@@ -109,7 +111,7 @@ namespace ignite
 
         private:
             /** Protocol version. */
-            int64_t version;
+            ProtocolVersion version;
 
             /** Distributed joins flag. */
             bool distributedJoins;
@@ -508,7 +510,7 @@ namespace ignite
         /**
          * Handshake response.
          */
-        class HandshakeResponse : public Response
+        class HandshakeResponse
         {
         public:
             /**
@@ -516,8 +518,8 @@ namespace ignite
              */
             HandshakeResponse() :
                 accepted(false),
-                protoVerSince(),
-                currentVer()
+                currentVer(),
+                error()
             {
                 // No-op.
             }
@@ -540,47 +542,52 @@ namespace ignite
             }
 
             /**
-             * Get host Apache Ignite version when protocol version has been introduced.
-             * @return Host Apache Ignite version when protocol version has been introduced.
+             * Get optional error.
+             * @return Optional error message.
              */
-            const std::string& ProtoVerSince() const
+            const std::string& GetError() const
             {
-                return protoVerSince;
+                return error;
             }
 
             /**
              * Current host Apache Ignite version.
              * @return Current host Apache Ignite version.
              */
-            const std::string& CurrentVer() const
+            const ProtocolVersion& GetCurrentVer() const
             {
                 return currentVer;
             }
 
-        private:
             /**
              * Read response using provided reader.
              * @param reader Reader.
              */
-            virtual void ReadOnSuccess(ignite::impl::binary::BinaryReaderImpl& reader)
+            void Read(ignite::impl::binary::BinaryReaderImpl& reader)
             {
                 accepted = reader.ReadBool();
 
                 if (!accepted)
                 {
-                    utility::ReadString(reader, protoVerSince);
-                    utility::ReadString(reader, currentVer);
+                    int16_t major = reader.ReadInt16();
+                    int16_t minor = reader.ReadInt16();
+                    int16_t maintenance = reader.ReadInt16();
+
+                    currentVer = ProtocolVersion(major, minor, maintenance);
+
+                    utility::ReadString(reader, error);
                 }
             }
 
+        private:
             /** Handshake accepted. */
             bool accepted;
 
-            /** Host Apache Ignite version when protocol version has been introduced. */
-            std::string protoVerSince;
+            /** Node's protocol version. */
+            ProtocolVersion currentVer;
 
-            /** Current host Apache Ignite version. */
-            std::string currentVer;
+            /** Optional error message. */
+            std::string error;
         };
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/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
index bb42dd4..a3cb88e 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
@@ -21,7 +21,7 @@
 #include <stdint.h>
 
 #include <string>
-#include <map>
+#include <set>
 
 namespace ignite
 {
@@ -31,24 +31,17 @@ namespace ignite
         class ProtocolVersion
         {
         public:
-            /** String to version map type alias. */
-            typedef std::map<std::string, ProtocolVersion> StringToVersionMap;
+            /** Current protocol version. */
+            static const ProtocolVersion VERSION_2_1_0;
 
-            /** Version to string map type alias. */
-            typedef std::map<ProtocolVersion, std::string> VersionToStringMap;
-
-            /** First version of the protocol that was introduced in Ignite 2.0.0. */
-            static const ProtocolVersion VERSION_2_0_0;
-
-            /** Unknown version of the protocol. */
-            static const ProtocolVersion VERSION_UNKNOWN;
+            typedef std::set<ProtocolVersion> VersionSet;
 
             /**
              * Get string to version map.
              *
              * @return String to version map.
              */
-            static const StringToVersionMap& GetMap();
+            static const VersionSet& GetSupported();
 
             /**
              * Get current version.
@@ -69,25 +62,59 @@ namespace ignite
             /**
              * 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;
+            std::string ToString() const;
 
             /**
-             * Get int value.
+             * Default constructor.
+             */
+            ProtocolVersion();
+
+            /**
+             * Constructor.
              *
-             * @return Integer value.
+             * @param vmajor Major version part.
+             * @param vminor Minor version part.
+             * @param vmaintenance Maintenance version part.
              */
-            int64_t GetIntValue() const;
+            ProtocolVersion(int16_t vmajor, int16_t vminor, int16_t vmaintenance);
 
             /**
-             * Check if the version is unknown.
+             * Get major part.
              *
-             * @return True if the version is unknown.
+             * @return Major part.
              */
-            bool IsUnknown() const;
+            int16_t GetMajor() const;
+
+            /**
+             * Get minor part.
+             *
+             * @return Minor part.
+             */
+            int16_t GetMinor() const;
+
+            /**
+             * Get maintenance part.
+             *
+             * @return Maintenance part.
+             */
+            int16_t GetMaintenance() const;
+
+            /**
+             * Check if the version is supported.
+             *
+             * @return True if the version is supported.
+             */
+            bool IsSupported() const;
+
+            /**
+             * Compare to another value.
+             *
+             * @param other Instance to compare to.
+             * @return Zero if equeals, negative number if less and positive if more.
+             */
+            int32_t Compare(const ProtocolVersion& other) const;
 
             /**
              * Comparison operator.
@@ -144,33 +171,17 @@ namespace ignite
             friend bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2);
 
         private:
-            /**
-             * Constructor.
-             *
-             * @param val Underlying value.
-             */
-            explicit ProtocolVersion(int64_t val);
-            
-            /**
-             * Make int value for the version.
-             *
-             * @param major Major version.
-             * @param minor Minor version.
-             * @param revision Revision.
-             * @return Int value for the version.
-             */
-            static int64_t MakeVersion(uint16_t major, uint16_t minor, uint16_t revision);
-
-            ProtocolVersion();
+            /** Set of supported versions. */
+            const static VersionSet supported;
 
-            /** String to version map. */
-            static const StringToVersionMap stringToVersionMap;
+            /** Major part. */
+            int16_t vmajor;
 
-            /** Version to string map. */
-            static const VersionToStringMap versionToStringMap;
+            /** Minor part. */
+            int16_t vminor;
 
-            /** Underlying int value. */
-            int64_t val;
+            /** Maintenance part. */
+            int16_t vmaintenance;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
index 4a88052..36cb5e8 100644
--- a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
@@ -138,14 +138,14 @@ namespace ignite
 
                     int id = 0;
 
-                    const ProtocolVersion::StringToVersionMap& versionMap = ProtocolVersion::GetMap();
+                    const ProtocolVersion::VersionSet& supported = ProtocolVersion::GetSupported();
 
-                    ProtocolVersion::StringToVersionMap::const_iterator it;
-                    for (it = versionMap.begin(); it != versionMap.end(); ++it)
+                    ProtocolVersion::VersionSet::const_iterator it;
+                    for (it = supported.begin(); it != supported.end(); ++it)
                     {
-                        protocolVersionComboBox->AddString(it->first);
+                        protocolVersionComboBox->AddString(it->ToString());
 
-                        if (it->second == config.GetProtocolVersion())
+                        if (*it == config.GetProtocolVersion())
                             protocolVersionComboBox->SetSelection(id);
 
                         ++id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 62194d0..b9c15e5 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -255,6 +255,8 @@ namespace ignite
 
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message body");
             }
+
+            LOG_MSG("Message received: " << utility::HexDump(&msg[0], msg.size()));
         }
 
         size_t Connection::ReceiveAll(void* dst, size_t len)
@@ -321,13 +323,13 @@ namespace ignite
         {
             bool distributedJoins = false;
             bool enforceJoinOrder = false;
-            int64_t protocolVersion = 0;
+            ProtocolVersion protocolVersion;
 
             try
             {
                 distributedJoins = config.IsDistributedJoins();
                 enforceJoinOrder = config.IsEnforceJoinOrder();
-                protocolVersion = config.GetProtocolVersion().GetIntValue();
+                protocolVersion = config.GetProtocolVersion();
             }
             catch (const IgniteError& err)
             {
@@ -336,6 +338,13 @@ namespace ignite
                 return SqlResult::AI_ERROR;
             }
 
+            if (!protocolVersion.IsSupported())
+            {
+                AddStatusRecord(SqlState::S01S00_INVALID_CONNECTION_STRING_ATTRIBUTE, "Protocol version is not supported: " + protocolVersion.ToString());
+
+                return SqlResult::AI_ERROR;
+            }
+
             HandshakeRequest req(protocolVersion, distributedJoins, enforceJoinOrder);
             HandshakeResponse rsp;
 
@@ -350,27 +359,19 @@ namespace ignite
                 return SqlResult::AI_ERROR;
             }
 
-            if (rsp.GetStatus() != ResponseStatus::SUCCESS)
-            {
-                LOG_MSG("Error: " << rsp.GetError().c_str());
-
-                AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, rsp.GetError());
-
-                InternalRelease();
-
-                return SqlResult::AI_ERROR;
-            }
-
             if (!rsp.IsAccepted())
             {
                 LOG_MSG("Hanshake message has been rejected.");
 
                 std::stringstream constructor;
 
-                constructor << "Node rejected handshake message. "
-                    << "Current node Apache Ignite version: " << rsp.CurrentVer() << ", "
-                    << "node protocol version introduced in version: " << rsp.ProtoVerSince() << ", "
-                    << "driver protocol version introduced in version: " << config.GetProtocolVersion().ToString() << ".";
+                constructor << "Node rejected handshake message. ";
+
+                if (!rsp.GetError().empty())
+                    constructor << "Additional info: " << rsp.GetError();
+
+                constructor << "Current node Apache Ignite version: " << rsp.GetCurrentVer().ToString() << ", "
+                            << "driver protocol version introduced in version: " << config.GetProtocolVersion().ToString() << ".";
 
                 AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, constructor.str());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddc6d4a/modules/platforms/cpp/odbc/src/protocol_version.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/protocol_version.cpp b/modules/platforms/cpp/odbc/src/protocol_version.cpp
index 859135d..aa8915d 100644
--- a/modules/platforms/cpp/odbc/src/protocol_version.cpp
+++ b/modules/platforms/cpp/odbc/src/protocol_version.cpp
@@ -14,8 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <ignite/common/concurrent.h>
-#include <ignite/common/utils.h>
+
+#include <sstream>
+
 #include <ignite/ignite_error.h>
 
 #include "ignite/odbc/protocol_version.h"
@@ -25,112 +26,147 @@ namespace ignite
 {
     namespace odbc
     {
-        const ProtocolVersion ProtocolVersion::VERSION_2_0_0(MakeVersion(2,0,0));
-        const ProtocolVersion ProtocolVersion::VERSION_UNKNOWN(INT64_MIN);
+        const ProtocolVersion ProtocolVersion::VERSION_2_1_0(ProtocolVersion(2,1,0));
 
-        ProtocolVersion::StringToVersionMap::value_type s2vInitVals[] = {
-            std::make_pair("2.0.0", ProtocolVersion::VERSION_2_0_0)
+        ProtocolVersion::VersionSet::value_type supportedArray[] = {
+            ProtocolVersion::VERSION_2_1_0
         };
 
-        const ProtocolVersion::StringToVersionMap ProtocolVersion::stringToVersionMap(s2vInitVals,
-            s2vInitVals + (sizeof(s2vInitVals) / sizeof(s2vInitVals[0])));
-
-        ProtocolVersion::VersionToStringMap::value_type v2sInitVals[] = {
-            std::make_pair(ProtocolVersion::VERSION_2_0_0, "2.0.0")
-        };
+        const ProtocolVersion::VersionSet ProtocolVersion::supported(supportedArray,
+            supportedArray + (sizeof(supportedArray) / sizeof(supportedArray[0])));
 
-        const ProtocolVersion::VersionToStringMap ProtocolVersion::versionToStringMap(v2sInitVals,
-            v2sInitVals + (sizeof(v2sInitVals) / sizeof(v2sInitVals[0])));
-
-        ProtocolVersion::ProtocolVersion(int64_t val) :
-            val(val)
+        ProtocolVersion::ProtocolVersion(int16_t vmajor, int16_t vminor, int16_t vmaintenance) :
+            vmajor(vmajor),
+            vminor(vminor),
+            vmaintenance(vmaintenance)
         {
             // No-op.
         }
 
-        int64_t ProtocolVersion::MakeVersion(uint16_t major, uint16_t minor, uint16_t revision)
+        ProtocolVersion::ProtocolVersion() :
+            vmajor(0),
+            vminor(0),
+            vmaintenance(0)
         {
-            const static int64_t MASK = 0x000000000000FFFFLL;
-            return ((major & MASK) << 48) | ((minor & MASK) << 32) | ((revision & MASK) << 16);
+            // No-op.
         }
 
-        const ProtocolVersion::StringToVersionMap& ProtocolVersion::GetMap()
+        const ProtocolVersion::VersionSet& ProtocolVersion::GetSupported()
         {
-            return stringToVersionMap;
+            return supported;
         }
 
         const ProtocolVersion& ProtocolVersion::GetCurrent()
         {
-            return VERSION_2_0_0;
+            return VERSION_2_1_0;
+        }
+
+        void ThrowParseError()
+        {
+            throw IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major, "
+                "minor and maintenance version parts of Ignite since which protocol is introduced.");
         }
 
         ProtocolVersion ProtocolVersion::FromString(const std::string& version)
         {
-            StringToVersionMap::const_iterator it = stringToVersionMap.find(common::ToLower(version));
+            ProtocolVersion res;
+
+            std::stringstream buf(version);
+
+            buf >> res.vmajor;
+
+            if (!buf.good())
+                ThrowParseError();
+
+            if (buf.get() != '.' || !buf.good())
+                ThrowParseError();
+
+            buf >> res.vminor;
 
-            if (it == stringToVersionMap.end())
-            {
-                throw IgniteError(IgniteError::IGNITE_ERR_GENERIC,
-                    "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major "
-                    "and minor versions and revision of Ignite since which protocol is introduced.");
-            }
+            if (!buf.good())
+                ThrowParseError();
 
-            return it->second;
+            if (buf.get() != '.' || !buf.good())
+                ThrowParseError();
+
+            buf >> res.vmaintenance;
+
+            if (buf.bad())
+                ThrowParseError();
+
+            return res;
         }
 
-        const std::string& ProtocolVersion::ToString() const
+        std::string ProtocolVersion::ToString() const
         {
-            VersionToStringMap::const_iterator it = versionToStringMap.find(*this);
+            std::stringstream buf;
+            buf << vmajor << '.' << vminor << '.' << vmaintenance;
 
-            if (it == versionToStringMap.end())
-            {
-                throw IgniteError(IgniteError::IGNITE_ERR_GENERIC,
-                    "Unknown protocol version can not be converted to string.");
-            }
+            return buf.str();
+        }
 
-            return it->second;
+        int16_t ProtocolVersion::GetMajor() const
+        {
+            return vmajor;
         }
 
-        int64_t ProtocolVersion::GetIntValue() const
+        int16_t ProtocolVersion::GetMinor() const
         {
-            assert(!IsUnknown());
+            return vminor;
+        }
 
-            return val;
+        int16_t ProtocolVersion::GetMaintenance() const
+        {
+            return vmaintenance;
         }
 
-        bool ProtocolVersion::IsUnknown() const
+        bool ProtocolVersion::IsSupported() const
         {
-            return *this == VERSION_UNKNOWN;
+            return supported.count(*this) != 0;
+        }
+
+        int32_t ProtocolVersion::Compare(const ProtocolVersion& other) const
+        {
+            int32_t res = vmajor - other.vmajor;
+
+            if (res == 0)
+                res = vminor - other.vminor;
+
+            if (res == 0)
+                res = vmaintenance - other.vmaintenance;
+
+            return res;
         }
 
         bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
-            return val1.val == val2.val;
+            return val1.Compare(val2) == 0;
         }
 
         bool operator!=(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
-            return val1.val != val2.val;
+            return val1.Compare(val2) != 0;
         }
 
         bool operator<(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
-            return val1.val < val2.val;
+            return val1.Compare(val2) < 0;
         }
 
         bool operator<=(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
-            return val1.val <= val2.val;
+            return val1.Compare(val2) <= 0;
         }
 
         bool operator>(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
-            return val1.val > val2.val;
+            return val1.Compare(val2) > 0;
         }
 
         bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
-            return val1.val >= val2.val;
+            return val1.Compare(val2) >= 0;
         }
     }
 }