You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/05/05 09:45:51 UTC

[40/50] [abbrv] ignite git commit: IGNITE-5167: ODBC: extracted handler/parser interfaces.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
deleted file mode 100644
index 01f32d1..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
+++ /dev/null
@@ -1,440 +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.escape;
-
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.odbc.OdbcUtils;
-import java.util.LinkedList;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * ODBC escape sequence parse.
- */
-public class OdbcEscapeUtils {
-    /** Odbc date regexp pattern: '2016-08-23' */
-    private static final Pattern DATE_PATTERN = Pattern.compile("^'\\d{4}-\\d{2}-\\d{2}'$");
-
-    /** Odbc time regexp pattern: '14:33:44' */
-    private static final Pattern TIME_PATTERN = Pattern.compile("^'\\d{2}:\\d{2}:\\d{2}'$");
-
-    /** Odbc timestamp regexp pattern: '2016-08-23 14:33:44.12345' */
-    private static final Pattern TIMESTAMP_PATTERN =
-        Pattern.compile("^'\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}(\\.\\d+)?'$");
-
-    /** GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc' */
-    private static final Pattern GUID_PATTERN =
-        Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$");
-
-    /** CONVERT function data type parameter pattern: last parameter, after comma */
-    private static final Pattern CONVERT_TYPE_PATTERN =
-        Pattern.compile(",\\s*(SQL_[\\w_]+)\\s*(?:\\(\\s*\\d+\\s*(?:,\\s*\\d+\\s*)?\\))?\\s*\\)\\s*$",
-                        Pattern.CASE_INSENSITIVE | Pattern.UNICODE_CASE);
-
-    /**
-     * Parse escape sequence.
-     *
-     * @param text Original text.
-     * @return Result.
-     */
-    public static String parse(String text) {
-        if (text == null)
-            throw new IgniteException("Text cannot be null.");
-
-        return parse0(text.trim(), 0, false).result();
-    }
-
-    /**
-     * Internal parse routine.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param earlyExit When set to {@code true} we must return as soon as single expression is parsed.
-     * @return Parse result.
-     */
-    private static OdbcEscapeParseResult parse0(String text, int startPos, boolean earlyExit) {
-        StringBuilder res = new StringBuilder();
-
-        int curPos = startPos;
-
-        int plainPos = startPos;
-        int openPos = -1;
-
-        boolean insideLiteral = false;
-
-        LinkedList<OdbcEscapeParseResult> nested = null;
-
-        while (curPos < text.length()) {
-            char curChar = text.charAt(curPos);
-
-            if (curChar == '\'')
-                /* Escaped quote in odbc is two successive singe quotes. They'll flip flag twice without side-effect. */
-                insideLiteral = !insideLiteral;
-            else if (!insideLiteral) {
-                if (curChar == '{') {
-                    if (openPos == -1) {
-                        // Top-level opening brace. Append previous portion and remember current position.
-                        res.append(text, plainPos, curPos);
-
-                        openPos = curPos;
-                    }
-                    else {
-                        // Nested opening brace -> perform recursion.
-                        OdbcEscapeParseResult nestedRes = parse0(text, curPos, true);
-
-                        if (nested == null)
-                            nested = new LinkedList<>();
-
-                        nested.add(nestedRes);
-
-                        curPos += nestedRes.originalLength() - 1;
-
-                        plainPos = curPos + 1;
-                    }
-                }
-                else if (curChar == '}') {
-                    if (openPos == -1)
-                        // Close without open -> exception.
-                        throw new IgniteException("Malformed escape sequence " +
-                            "(closing curly brace without opening curly brace): " + text);
-                    else {
-                        String parseRes;
-
-                        if (nested == null)
-                            // Found sequence without nesting, process it.
-                            parseRes = parseEscapeSequence(text, openPos, curPos + 1 - openPos);
-                        else {
-                            // Special case to process nesting.
-                            String res0 = appendNested(text, openPos, curPos + 1, nested);
-
-                            nested = null;
-
-                            parseRes = parseEscapeSequence(res0, 0, res0.length());
-                        }
-
-                        if (earlyExit)
-                            return new OdbcEscapeParseResult(startPos, curPos + 1 - startPos, parseRes);
-                        else
-                            res.append(parseRes);
-
-                        openPos = -1;
-
-                        plainPos = curPos + 1;
-                    }
-                }
-            }
-
-            curPos++;
-        }
-
-        if (openPos != -1)
-            throw new IgniteException("Malformed escape sequence (closing curly brace missing): " + text);
-
-        if (insideLiteral)
-            throw new IgniteException("Malformed literal expression (closing quote missing): " + text);
-
-        if (curPos > plainPos)
-            res.append(text, plainPos, curPos);
-
-        return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, res.toString());
-    }
-
-    /**
-     * Parse escape sequence: {escape_sequence}.
-     *
-     * @param text Text.
-     * @param startPos Start position within text.
-     * @param len Length.
-     * @return Result.
-     */
-    private static String parseEscapeSequence(String text, int startPos, int len) {
-        assert validSubstring(text, startPos, len);
-
-        char firstChar = text.charAt(startPos);
-
-        if (firstChar == '{') {
-            char lastChar = text.charAt(startPos + len - 1);
-
-            if (lastChar != '}')
-                throw new IgniteException("Failed to parse escape sequence because it is not enclosed: " +
-                    substring(text, startPos, len));
-
-            OdbcEscapeToken token = parseToken(text, startPos, len);
-
-            return parseEscapeSequence(text, startPos, len, token);
-        }
-        else {
-            // Nothing to escape, return original string.
-            if (startPos == 0 || text.length() == len)
-                return text;
-            else
-                return substring(text, startPos, len);
-        }
-    }
-
-    /**
-     * Get escape sequence info.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @return Escape sequence info.
-     */
-    private static OdbcEscapeToken parseToken(String text, int startPos, int len) {
-        assert validSubstring(text, startPos, len);
-        assert text.charAt(startPos) == '{';
-
-        int pos = startPos + 1;
-
-        while (Character.isWhitespace(text.charAt(pos)))
-            pos++;
-
-        OdbcEscapeType curTyp = null;
-        boolean empty = false;
-
-        for (OdbcEscapeType typ : OdbcEscapeType.sortedValues()) {
-            if (text.startsWith(typ.body(), pos)) {
-                if (typ.standard())
-                    pos += typ.body().length();
-
-                empty = (startPos + len == pos + 1);
-
-                if (!empty && typ.standard()) {
-                    char charAfter = text.charAt(pos);
-
-                    if (!Character.isWhitespace(charAfter))
-                        throw new IgniteException("Unexpected escape sequence token: " +
-                            substring(text, startPos, len));
-                }
-
-                curTyp = typ;
-
-                break;
-            }
-        }
-
-        if (curTyp == null)
-            throw new IgniteException("Unsupported escape sequence: " + substring(text, startPos, len));
-
-        if (empty && !curTyp.allowEmpty())
-            throw new IgniteException("Escape sequence cannot be empty: " + substring(text, startPos, len));
-
-        return new OdbcEscapeToken(curTyp, pos - (startPos + 1));
-    }
-
-    /**
-     * Parse standard expression: {TOKEN expression}
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param len Length.
-     * @param token Token.
-     * @return Result.
-     */
-    private static String parseEscapeSequence(String text, int startPos, int len, OdbcEscapeToken token) {
-        assert validSubstring(text, startPos, len);
-
-        // Get expression borders.
-        int startPos0 = startPos + 1 /* open brace */ + token.length() /* token. */;
-        int len0 = len - 1 /* open brace */ - token.length() /* token */ - 1 /* close brace */;
-
-        switch (token.type()) {
-            case SCALAR_FUNCTION:
-                return parseScalarFunctionExpression(text, startPos0, len0);
-
-            case GUID: {
-                String res = parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN);
-
-                return "CAST(" + res + " AS UUID)";
-            }
-
-            case DATE:
-                return parseExpression(text, startPos0, len0, token.type(), DATE_PATTERN);
-
-            case TIME:
-                return parseExpression(text, startPos0, len0, token.type(), TIME_PATTERN);
-
-            case TIMESTAMP:
-                return parseExpression(text, startPos0, len0, token.type(), TIMESTAMP_PATTERN);
-
-            case OUTER_JOIN:
-                return parseExpression(text, startPos0, len0);
-
-            case CALL: {
-                String val = parseExpression(text, startPos0, len0);
-
-                return "CALL " + val;
-            }
-
-            case ESCAPE:
-            case ESCAPE_WO_TOKEN:
-                return parseLikeEscCharacterExpression(text, startPos0, len0);
-
-            default:
-                throw new IgniteException("Unsupported escape sequence token [text=" +
-                    substring(text, startPos, len) + ", token=" + token.type().body() + ']');
-        }
-    }
-
-    /**
-     * Parse simple expression.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param len Length.
-     * @return Parsed expression.
-     */
-    private static String parseExpression(String text, int startPos, int len) {
-        return substring(text, startPos, len).trim();
-    }
-
-    /**
-     * Parse LIKE escape character expression.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param len Length.
-     * @return Parsed expression.
-     */
-    private static String parseLikeEscCharacterExpression(String text, int startPos, int len) {
-        return "ESCAPE " + substring(text, startPos, len).trim();
-    }
-
-    /**
-     * Parse expression and validate against ODBC specification with regex pattern.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param len Length.
-     * @return Parsed expression.
-     */
-    private static String parseExpression(String text, int startPos, int len, OdbcEscapeType type, Pattern pattern) {
-        String val = parseExpression(text, startPos, len);
-
-        if (!pattern.matcher(val).matches())
-            throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len));
-
-        return val;
-    }
-
-    /**
-     * Parse scalar function expression.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param len Length.
-     * @return Parsed expression.
-     */
-    private static String parseScalarFunctionExpression(String text, int startPos, int len) {
-        int pos = startPos;
-        int endPos = startPos + len;
-        final String errPrefix = "Malformed scalar function escape sequence.";
-
-        while ((++pos < endPos) && Character.isWhitespace(text.charAt(pos)));
-        if (pos == endPos)
-            throw new IgniteException(errPrefix + " Expected function name.");
-
-        int funcNamePos = pos;
-        while ((++pos < endPos) && Character.isAlphabetic(text.charAt(pos)));
-        if (pos == endPos)
-            throw new IgniteException(errPrefix + " Expected function parameter list: " +
-                                      substring(text, startPos, len));
-
-        String funcName = text.substring(funcNamePos, pos);
-
-        switch (funcName.toUpperCase()) {
-            case "CONVERT": {
-                Matcher matcher = CONVERT_TYPE_PATTERN.matcher(text.substring(startPos, endPos));
-
-                if (!matcher.find())
-                    throw new IgniteException(errPrefix + " Invalid arguments :" +
-                                              substring(text, startPos, len));
-
-                return (text.substring(startPos, startPos + matcher.start(1)) +
-                        OdbcUtils.getIgniteTypeFromOdbcType(matcher.group(1)) +
-                        text.substring(startPos + matcher.end(1), startPos + len)).trim();
-            }
-            default:
-                return substring(text, startPos, len).trim();
-        }
-    }
-
-    /**
-     * Append nested results.
-     *
-     * @param text Original text.
-     * @param startPos Start position.
-     * @param endPos End position.
-     * @param nestedRess Nested results.
-     * @return Result.
-     */
-    private static String appendNested(String text, int startPos, int endPos,
-        LinkedList<OdbcEscapeParseResult> nestedRess) {
-        StringBuilder res = new StringBuilder();
-
-        int curPos = startPos;
-
-        for (OdbcEscapeParseResult nestedRes : nestedRess) {
-            // Append text between current position and replace.
-            res.append(text, curPos, nestedRes.originalStart());
-
-            // Append replaced text.
-            res.append(nestedRes.result());
-
-            // Advance position.
-            curPos = nestedRes.originalStart() + nestedRes.originalLength();
-        }
-
-        // Append remainder.
-        res.append(text, curPos, endPos);
-
-        return res.toString();
-    }
-
-    /**
-     * Perform "substring" using start position and length.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @param len Length.
-     * @return Substring.
-     */
-    private static String substring(String text, int startPos, int len) {
-        assert validSubstring(text, startPos, len);
-
-        return text.substring(startPos, startPos + len);
-    }
-
-    /**
-     * Check whether substring is valid.
-     *
-     * @param text Substring.
-     * @param startPos Start position.
-     * @param len Length.
-     * @return {@code True} if valid.
-     */
-    private static boolean validSubstring(String text, int startPos, int len) {
-        return text != null && startPos + len <= text.length();
-    }
-
-    /**
-     * Private constructor.
-     */
-    private OdbcEscapeUtils() {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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
new file mode 100644
index 0000000..37fe3bf
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
@@ -0,0 +1,322 @@
+/*
+ * 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.odbc;
+
+import org.apache.ignite.IgniteException;
+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.GridBinaryMarshaller;
+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;
+import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaResult;
+import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaRequest;
+import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaResult;
+import org.apache.ignite.internal.processors.odbc.OdbcTableMeta;
+import org.apache.ignite.internal.processors.odbc.SqlListenerColumnMeta;
+import org.apache.ignite.internal.processors.odbc.SqlListenerMessageParser;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryCloseResult;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteResult;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchResult;
+import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+
+import java.util.Collection;
+
+/**
+ * ODBC message parser.
+ */
+public class OdbcMessageParser implements SqlListenerMessageParser {
+    /** Initial output stream capacity. */
+    private static final int INIT_CAP = 1024;
+
+    /** Marshaller. */
+    private final GridBinaryMarshaller marsh;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Protocol version confirmation flag. */
+    private boolean verConfirmed = false;
+
+    /**
+     * @param ctx Context.
+     */
+    public OdbcMessageParser(final GridKernalContext ctx) {
+        CacheObjectBinaryProcessorImpl cacheObjProc = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects();
+
+        this.marsh = cacheObjProc.marshaller();
+
+        this.log = ctx.log(getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlListenerRequest decode(byte[] msg) {
+        assert msg != null;
+
+        BinaryInputStream stream = new BinaryHeapInputStream(msg);
+
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null, true);
+
+        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) {
+            case SqlListenerRequest.QRY_EXEC: {
+                String cache = reader.readString();
+                String sql = reader.readString();
+                int argsNum = reader.readInt();
+
+                Object[] params = new Object[argsNum];
+
+                for (int i = 0; i < argsNum; ++i)
+                    params[i] = reader.readObjectDetached();
+
+                res = new SqlListenerQueryExecuteRequest(cache, sql, params);
+
+                break;
+            }
+
+            case SqlListenerRequest.QRY_FETCH: {
+                long queryId = reader.readLong();
+                int pageSize = reader.readInt();
+
+                res = new SqlListenerQueryFetchRequest(queryId, pageSize);
+
+                break;
+            }
+
+            case SqlListenerRequest.QRY_CLOSE: {
+                long queryId = reader.readLong();
+
+                res = new SqlListenerQueryCloseRequest(queryId);
+
+                break;
+            }
+
+            case SqlListenerRequest.META_COLS: {
+                String cache = reader.readString();
+                String table = reader.readString();
+                String column = reader.readString();
+
+                res = new OdbcQueryGetColumnsMetaRequest(cache, table, column);
+
+                break;
+            }
+
+            case SqlListenerRequest.META_TBLS: {
+                String catalog = reader.readString();
+                String schema = reader.readString();
+                String table = reader.readString();
+                String tableType = reader.readString();
+
+                res = new OdbcQueryGetTablesMetaRequest(catalog, schema, table, tableType);
+
+                break;
+            }
+
+            case SqlListenerRequest.META_PARAMS: {
+                String cacheName = reader.readString();
+                String sqlQuery = reader.readString();
+
+                res = new OdbcQueryGetParamsMetaRequest(cacheName, sqlQuery);
+
+                break;
+            }
+
+            default:
+                throw new IgniteException("Unknown ODBC command: [cmd=" + cmd + ']');
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] encode(SqlListenerResponse msg) {
+        assert msg != null;
+
+        // Creating new binary writer
+        BinaryWriterExImpl writer = marsh.writer(new BinaryHeapOutputStream(INIT_CAP));
+
+        // Writing status.
+        writer.writeByte((byte) msg.status());
+
+        if (msg.status() != SqlListenerResponse.STATUS_SUCCESS) {
+            writer.writeString(msg.error());
+
+            return writer.array();
+        }
+
+        Object res0 = msg.response();
+
+        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;
+
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.getQueryId());
+
+            writer.writeLong(res.getQueryId());
+
+            Collection<SqlListenerColumnMeta> metas = res.getColumnsMetadata();
+
+            assert metas != null;
+
+            writer.writeInt(metas.size());
+
+            for (SqlListenerColumnMeta meta : metas)
+                meta.write(writer);
+        }
+        else if (res0 instanceof SqlListenerQueryFetchResult) {
+            SqlListenerQueryFetchResult res = (SqlListenerQueryFetchResult) res0;
+
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.queryId());
+
+            writer.writeLong(res.queryId());
+
+            Collection<?> items0 = res.items();
+
+            assert items0 != null;
+
+            writer.writeBoolean(res.last());
+
+            writer.writeInt(items0.size());
+
+            for (Object row0 : items0) {
+                if (row0 != null) {
+                    Collection<?> row = (Collection<?>)row0;
+
+                    writer.writeInt(row.size());
+
+                    for (Object obj : row) {
+                        if (obj == null) {
+                            writer.writeObjectDetached(null);
+                            continue;
+                        }
+
+                        Class<?> cls = obj.getClass();
+
+                        if (cls == java.sql.Time.class)
+                            writer.writeTime((java.sql.Time)obj);
+                        else if (cls == java.sql.Timestamp.class)
+                            writer.writeTimestamp((java.sql.Timestamp)obj);
+                        else if (cls == java.sql.Date.class)
+                            writer.writeDate((java.util.Date)obj);
+                        else
+                            writer.writeObjectDetached(obj);
+                    }
+                }
+            }
+        }
+        else if (res0 instanceof SqlListenerQueryCloseResult) {
+            SqlListenerQueryCloseResult res = (SqlListenerQueryCloseResult) res0;
+
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.getQueryId());
+
+            writer.writeLong(res.getQueryId());
+        }
+        else if (res0 instanceof OdbcQueryGetColumnsMetaResult) {
+            OdbcQueryGetColumnsMetaResult res = (OdbcQueryGetColumnsMetaResult) res0;
+
+            Collection<SqlListenerColumnMeta> columnsMeta = res.meta();
+
+            assert columnsMeta != null;
+
+            writer.writeInt(columnsMeta.size());
+
+            for (SqlListenerColumnMeta columnMeta : columnsMeta)
+                columnMeta.write(writer);
+        }
+        else if (res0 instanceof OdbcQueryGetTablesMetaResult) {
+            OdbcQueryGetTablesMetaResult res = (OdbcQueryGetTablesMetaResult) res0;
+
+            Collection<OdbcTableMeta> tablesMeta = res.meta();
+
+            assert tablesMeta != null;
+
+            writer.writeInt(tablesMeta.size());
+
+            for (OdbcTableMeta tableMeta : tablesMeta)
+                tableMeta.writeBinary(writer);
+        }
+        else if (res0 instanceof OdbcQueryGetParamsMetaResult) {
+            OdbcQueryGetParamsMetaResult res = (OdbcQueryGetParamsMetaResult) res0;
+
+            byte[] typeIds = res.typeIds();
+
+            writer.writeObjectDetached(typeIds);
+        }
+        else
+            assert false : "Should not reach here.";
+
+        return writer.array();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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
new file mode 100644
index 0000000..815f650
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -0,0 +1,540 @@
+/*
+ * 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.odbc;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.QueryCursor;
+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;
+import org.apache.ignite.internal.processors.odbc.OdbcQueryGetParamsMetaResult;
+import org.apache.ignite.internal.processors.odbc.OdbcQueryGetTablesMetaRequest;
+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;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryExecuteResult;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerQueryFetchResult;
+import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
+import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
+import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.odbc.escape.OdbcEscapeUtils;
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+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.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.ignite.internal.processors.odbc.SqlListenerRequest.*;
+
+/**
+ * SQL query handler.
+ */
+public class OdbcRequestHandler implements SqlListenerRequestHandler {
+    /** Query ID sequence. */
+    private static final AtomicLong QRY_ID_GEN = new AtomicLong();
+
+    /** Kernel context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Busy lock. */
+    private final GridSpinBusyLock busyLock;
+
+    /** Maximum allowed cursors. */
+    private final int maxCursors;
+
+    /** Current queries cursors. */
+    private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCursors = new ConcurrentHashMap<>();
+
+    /** Distributed joins flag. */
+    private boolean distributedJoins = false;
+
+    /** Enforce join order flag. */
+    private boolean enforceJoinOrder = false;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     * @param busyLock Shutdown latch.
+     * @param maxCursors Maximum allowed cursors.
+     */
+    public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) {
+        this.ctx = ctx;
+        this.busyLock = busyLock;
+        this.maxCursors = maxCursors;
+
+        log = ctx.log(getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlListenerResponse handle(SqlListenerRequest req) {
+        assert req != null;
+
+        if (!busyLock.enterBusy())
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Failed to handle ODBC request because node is stopping: " + req);
+
+        try {
+            switch (req.command()) {
+                case HANDSHAKE:
+                    return performHandshake((SqlListenerHandshakeRequest)req);
+
+                case QRY_EXEC:
+                    return executeQuery((SqlListenerQueryExecuteRequest)req);
+
+                case QRY_FETCH:
+                    return fetchQuery((SqlListenerQueryFetchRequest)req);
+
+                case QRY_CLOSE:
+                    return closeQuery((SqlListenerQueryCloseRequest)req);
+
+                case META_COLS:
+                    return getColumnsMeta((OdbcQueryGetColumnsMetaRequest)req);
+
+                case META_TBLS:
+                    return getTablesMeta((OdbcQueryGetTablesMetaRequest)req);
+
+                case META_PARAMS:
+                    return getParamsMeta((OdbcQueryGetParamsMetaRequest)req);
+            }
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * {@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.
+     * @return Response.
+     */
+    private SqlListenerResponse executeQuery(SqlListenerQueryExecuteRequest req) {
+        int cursorCnt = qryCursors.size();
+
+        if (maxCursors > 0 && cursorCnt >= maxCursors)
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, "Too many opened cursors (either close other " +
+                "opened cursors or increase the limit through OdbcConfiguration.setMaxOpenCursors()) " +
+                "[maximum=" + maxCursors + ", current=" + cursorCnt + ']');
+
+        long qryId = QRY_ID_GEN.getAndIncrement();
+
+        try {
+            String sql = OdbcEscapeUtils.parse(req.sqlQuery());
+
+            if (log.isDebugEnabled())
+                log.debug("ODBC query parsed [reqId=" + req.requestId() + ", original=" + req.sqlQuery() +
+                    ", parsed=" + sql + ']');
+
+            SqlFieldsQuery qry = new SqlFieldsQuery(sql);
+
+            qry.setArgs(req.arguments());
+
+            qry.setDistributedJoins(distributedJoins);
+            qry.setEnforceJoinOrder(enforceJoinOrder);
+
+            IgniteCache<Object, Object> cache0 = ctx.grid().cache(req.cacheName());
+
+            if (cache0 == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
+
+            IgniteCache<Object, Object> cache = cache0.withKeepBinary();
+
+            if (cache == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Can not get cache with keep binary: " + req.cacheName());
+
+            QueryCursor qryCur = cache.query(qry);
+
+            qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
+
+            List<?> fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta();
+
+            SqlListenerQueryExecuteResult res = new SqlListenerQueryExecuteResult(qryId, convertMetadata(fieldsMeta));
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            qryCursors.remove(qryId);
+
+            U.error(log, "Failed to execute SQL query [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link SqlListenerQueryCloseRequest} command handler.
+     *
+     * @param req Execute query request.
+     * @return Response.
+     */
+    private SqlListenerResponse closeQuery(SqlListenerQueryCloseRequest req) {
+        try {
+            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
+
+            if (tuple == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Failed to find query with ID: " + req.queryId());
+
+            QueryCursor cur = tuple.get1();
+
+            assert(cur != null);
+
+            cur.close();
+
+            qryCursors.remove(req.queryId());
+
+            SqlListenerQueryCloseResult res = new SqlListenerQueryCloseResult(req.queryId());
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            qryCursors.remove(req.queryId());
+
+            U.error(log, "Failed to close SQL query [reqId=" + req.requestId() + ", req=" + req.queryId() + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link SqlListenerQueryFetchRequest} command handler.
+     *
+     * @param req Execute query request.
+     * @return Response.
+     */
+    private SqlListenerResponse fetchQuery(SqlListenerQueryFetchRequest req) {
+        try {
+            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
+
+            if (tuple == null)
+                return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED,
+                    "Failed to find query with ID: " + req.queryId());
+
+            Iterator iter = tuple.get2();
+
+            if (iter == null) {
+                QueryCursor cur = tuple.get1();
+
+                iter = cur.iterator();
+
+                tuple.put(cur, iter);
+            }
+
+            List<Object> items = new ArrayList<>();
+
+            for (int i = 0; i < req.pageSize() && iter.hasNext(); ++i)
+                items.add(iter.next());
+
+            SqlListenerQueryFetchResult res = new SqlListenerQueryFetchResult(req.queryId(), items, !iter.hasNext());
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to fetch SQL query result [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetColumnsMetaRequest} command handler.
+     *
+     * @param req Get columns metadata request.
+     * @return Response.
+     */
+    private SqlListenerResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) {
+        try {
+            List<SqlListenerColumnMeta> meta = new ArrayList<>();
+
+            String cacheName;
+            String tableName;
+
+            if (req.tableName().contains(".")) {
+                // Parsing two-part table name.
+                String[] parts = req.tableName().split("\\.");
+
+                cacheName = OdbcUtils.removeQuotationMarksIfNeeded(parts[0]);
+
+                tableName = parts[1];
+            }
+            else {
+                cacheName = OdbcUtils.removeQuotationMarksIfNeeded(req.cacheName());
+
+                tableName = req.tableName();
+            }
+
+            Collection<GridQueryTypeDescriptor> tablesMeta = ctx.query().types(cacheName);
+
+            for (GridQueryTypeDescriptor table : tablesMeta) {
+                if (!matches(table.name(), tableName))
+                    continue;
+
+                for (Map.Entry<String, Class<?>> field : table.fields().entrySet()) {
+                    if (!matches(field.getKey(), req.columnName()))
+                        continue;
+
+                    SqlListenerColumnMeta columnMeta = new SqlListenerColumnMeta(req.cacheName(), table.name(),
+                        field.getKey(), field.getValue());
+
+                    if (!meta.contains(columnMeta))
+                        meta.add(columnMeta);
+                }
+            }
+
+            OdbcQueryGetColumnsMetaResult res = new OdbcQueryGetColumnsMetaResult(meta);
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get columns metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetTablesMetaRequest} command handler.
+     *
+     * @param req Get tables metadata request.
+     * @return Response.
+     */
+    private SqlListenerResponse getTablesMeta(OdbcQueryGetTablesMetaRequest req) {
+        try {
+            List<OdbcTableMeta> meta = new ArrayList<>();
+
+            String realSchema = OdbcUtils.removeQuotationMarksIfNeeded(req.schema());
+
+            for (String cacheName : ctx.cache().cacheNames())
+            {
+                if (!matches(cacheName, realSchema))
+                    continue;
+
+                Collection<GridQueryTypeDescriptor> tablesMeta = ctx.query().types(cacheName);
+
+                for (GridQueryTypeDescriptor table : tablesMeta) {
+                    if (!matches(table.name(), req.table()))
+                        continue;
+
+                    if (!matches("TABLE", req.tableType()))
+                        continue;
+
+                    OdbcTableMeta tableMeta = new OdbcTableMeta(null, cacheName, table.name(), "TABLE");
+
+                    if (!meta.contains(tableMeta))
+                        meta.add(tableMeta);
+                }
+            }
+
+            OdbcQueryGetTablesMetaResult res = new OdbcQueryGetTablesMetaResult(meta);
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get tables metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetParamsMetaRequest} command handler.
+     *
+     * @param req Get params metadata request.
+     * @return Response.
+     */
+    private SqlListenerResponse getParamsMeta(OdbcQueryGetParamsMetaRequest req) {
+        try {
+            PreparedStatement stmt = ctx.query().prepareNativeStatement(req.cacheName(), req.query());
+
+            ParameterMetaData pmd = stmt.getParameterMetaData();
+
+            byte[] typeIds = new byte[pmd.getParameterCount()];
+
+            for (int i = 1; i <= pmd.getParameterCount(); ++i) {
+                int sqlType = pmd.getParameterType(i);
+
+                typeIds[i - 1] = sqlTypeToBinary(sqlType);
+            }
+
+            OdbcQueryGetParamsMetaResult res = new OdbcQueryGetParamsMetaResult(typeIds);
+
+            return new SqlListenerResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get params metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * Convert {@link java.sql.Types} to binary type constant (See {@link GridBinaryMarshaller} constants).
+     *
+     * @param sqlType SQL type.
+     * @return Binary type.
+     */
+    private static byte sqlTypeToBinary(int sqlType) {
+        switch (sqlType) {
+            case Types.BIGINT:
+                return GridBinaryMarshaller.LONG;
+
+            case Types.BOOLEAN:
+                return GridBinaryMarshaller.BOOLEAN;
+
+            case Types.DATE:
+                return GridBinaryMarshaller.DATE;
+
+            case Types.DOUBLE:
+                return GridBinaryMarshaller.DOUBLE;
+
+            case Types.FLOAT:
+            case Types.REAL:
+                return GridBinaryMarshaller.FLOAT;
+
+            case Types.NUMERIC:
+            case Types.DECIMAL:
+                return GridBinaryMarshaller.DECIMAL;
+
+            case Types.INTEGER:
+                return GridBinaryMarshaller.INT;
+
+            case Types.SMALLINT:
+                return GridBinaryMarshaller.SHORT;
+
+            case Types.TIME:
+                return GridBinaryMarshaller.TIME;
+
+            case Types.TIMESTAMP:
+                return GridBinaryMarshaller.TIMESTAMP;
+
+            case Types.TINYINT:
+                return GridBinaryMarshaller.BYTE;
+
+            case Types.CHAR:
+            case Types.VARCHAR:
+            case Types.LONGNVARCHAR:
+                return GridBinaryMarshaller.STRING;
+
+            case Types.NULL:
+                return GridBinaryMarshaller.NULL;
+
+            case Types.BINARY:
+            case Types.VARBINARY:
+            case Types.LONGVARBINARY:
+            default:
+                return GridBinaryMarshaller.BYTE_ARR;
+        }
+    }
+
+    /**
+     * Convert metadata in collection from {@link GridQueryFieldMetadata} to
+     * {@link SqlListenerColumnMeta}.
+     *
+     * @param meta Internal query field metadata.
+     * @return Odbc query field metadata.
+     */
+    private static Collection<SqlListenerColumnMeta> convertMetadata(Collection<?> meta) {
+        List<SqlListenerColumnMeta> res = new ArrayList<>();
+
+        if (meta != null) {
+            for (Object info : meta) {
+                assert info instanceof GridQueryFieldMetadata;
+
+                res.add(new SqlListenerColumnMeta((GridQueryFieldMetadata)info));
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * Checks whether string matches SQL pattern.
+     *
+     * @param str String.
+     * @param ptrn Pattern.
+     * @return Whether string matches pattern.
+     */
+    private static boolean matches(String str, String ptrn) {
+        return str != null && (F.isEmpty(ptrn) ||
+            str.toUpperCase().matches(ptrn.toUpperCase().replace("%", ".*").replace("_", ".")));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeParseResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeParseResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeParseResult.java
new file mode 100644
index 0000000..369e7ac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeParseResult.java
@@ -0,0 +1,73 @@
+/*
+ * 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.odbc.escape;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * ODBC escape sequence parse result.
+ */
+public class OdbcEscapeParseResult {
+    /** Original start position. */
+    private final int originalStart;
+
+    /** Original length. */
+    private final int originalLen;
+
+    /** Resulting text. */
+    private final String res;
+
+    /**
+     * Constructor.
+     *
+     * @param originalStart Original start position.
+     * @param originalLen Original length.
+     * @param res Resulting text.
+     */
+    public OdbcEscapeParseResult(int originalStart, int originalLen, String res) {
+        this.originalStart = originalStart;
+        this.originalLen = originalLen;
+        this.res = res;
+    }
+
+    /**
+     * @return Original start position.
+     */
+    public int originalStart() {
+        return originalStart;
+    }
+
+    /**
+     * @return Original length.
+     */
+    public int originalLength() {
+        return originalLen;
+    }
+
+    /**
+     * @return Resulting text.
+     */
+    public String result() {
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OdbcEscapeParseResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeToken.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeToken.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeToken.java
new file mode 100644
index 0000000..a8b40fa
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeToken.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.odbc.odbc.escape;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * ODBC escape sequence token.
+ */
+public class OdbcEscapeToken {
+    /** Escape sequence type. */
+    private final OdbcEscapeType type;
+
+    /** Token length. */
+    private final int len;
+
+    /**
+     * Constructor.
+     *
+     * @param type Escape sequence type.
+     * @param len Token length.
+     */
+    public OdbcEscapeToken(OdbcEscapeType type, int len) {
+        this.type = type;
+        this.len = len;
+    }
+
+    /**
+     * @return Escape sequence type.
+     */
+    public OdbcEscapeType type() {
+        return type;
+    }
+
+    /**
+     * @return Token length.
+     */
+    public int length() {
+        return len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OdbcEscapeToken.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeType.java
new file mode 100644
index 0000000..748263d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeType.java
@@ -0,0 +1,112 @@
+/*
+ * 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.odbc.escape;
+
+/**
+ * ODBC escape sequence type.
+ */
+public enum OdbcEscapeType {
+    /** Scalar function. */
+    SCALAR_FUNCTION("fn", true, false),
+
+    /** Outer join. */
+    OUTER_JOIN("oj", true, false),
+
+    /** Stored procedure call */
+    CALL("call", true, false),
+
+    /** Date. */
+    DATE("d", true, false),
+
+    /** Timestamp. */
+    TIMESTAMP("ts", true, false),
+
+    /** Time. */
+    TIME("t", true, false),
+
+    /** GUID. */
+    GUID("guid", true, false),
+
+    /** LIKE escape character clause. */
+    ESCAPE_WO_TOKEN("\'", false, false),
+
+    /** LIKE escape character clause. */
+    ESCAPE("escape", true, false);
+
+    /** Values in convenient order. */
+    private static final OdbcEscapeType[] VALS = new OdbcEscapeType[] {
+        SCALAR_FUNCTION, // Assume that scalar functions are very frequent.
+        DATE, TIMESTAMP, // Date and timestamp are relatively frequent as well; also TS must go before T.
+        OUTER_JOIN,      // Joins are less frequent,
+        CALL,            // Procedure calls are less frequent than joins.
+        ESCAPE_WO_TOKEN, ESCAPE, TIME, GUID // LIKE, TIME and GUID are even less frequent.
+    };
+
+    /**
+     * Get values in convenient order, where the most frequent values goes first, and "startsWith" invocation is
+     * enough to get type (i.e. "ts" goes before "t").
+     *
+     * @return Values.
+     */
+    public static OdbcEscapeType[] sortedValues() {
+        return VALS;
+    }
+
+    /** Escape sequence body. */
+    private final String body;
+
+    /** Whether this is a standard token with no special handling. */
+    private final boolean standard;
+
+    /** Whether empty escape sequence is allowed. */
+    private final boolean allowEmpty;
+
+    /**
+     * Constructor.
+     *
+     * @param body Escape sequence body.
+     * @param standard Whether this is a standard token with no special handling.
+     * @param allowEmpty Whether empty escape sequence is allowed.
+     */
+    OdbcEscapeType(String body, boolean standard, boolean allowEmpty) {
+        this.body = body;
+        this.standard = standard;
+        this.allowEmpty = allowEmpty;
+    }
+
+    /**
+     * @return Escape sequence body.
+     */
+    public String body() {
+        return body;
+    }
+
+    /**
+     * @return Whether this is a standard token with no special handling.
+     */
+    public boolean standard() {
+        return standard;
+    }
+
+    /**
+     * @return Whether empty escape sequence is allowed.
+     */
+    public boolean allowEmpty() {
+        return allowEmpty;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeUtils.java
new file mode 100644
index 0000000..cd97667
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/escape/OdbcEscapeUtils.java
@@ -0,0 +1,441 @@
+/*
+ * 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.odbc.escape;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.odbc.OdbcUtils;
+
+import java.util.LinkedList;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * ODBC escape sequence parse.
+ */
+public class OdbcEscapeUtils {
+    /** Odbc date regexp pattern: '2016-08-23' */
+    private static final Pattern DATE_PATTERN = Pattern.compile("^'\\d{4}-\\d{2}-\\d{2}'$");
+
+    /** Odbc time regexp pattern: '14:33:44' */
+    private static final Pattern TIME_PATTERN = Pattern.compile("^'\\d{2}:\\d{2}:\\d{2}'$");
+
+    /** Odbc timestamp regexp pattern: '2016-08-23 14:33:44.12345' */
+    private static final Pattern TIMESTAMP_PATTERN =
+        Pattern.compile("^'\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}(\\.\\d+)?'$");
+
+    /** GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc' */
+    private static final Pattern GUID_PATTERN =
+        Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$");
+
+    /** CONVERT function data type parameter pattern: last parameter, after comma */
+    private static final Pattern CONVERT_TYPE_PATTERN =
+        Pattern.compile(",\\s*(SQL_[\\w_]+)\\s*(?:\\(\\s*\\d+\\s*(?:,\\s*\\d+\\s*)?\\))?\\s*\\)\\s*$",
+                        Pattern.CASE_INSENSITIVE | Pattern.UNICODE_CASE);
+
+    /**
+     * Parse escape sequence.
+     *
+     * @param text Original text.
+     * @return Result.
+     */
+    public static String parse(String text) {
+        if (text == null)
+            throw new IgniteException("Text cannot be null.");
+
+        return parse0(text.trim(), 0, false).result();
+    }
+
+    /**
+     * Internal parse routine.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param earlyExit When set to {@code true} we must return as soon as single expression is parsed.
+     * @return Parse result.
+     */
+    private static OdbcEscapeParseResult parse0(String text, int startPos, boolean earlyExit) {
+        StringBuilder res = new StringBuilder();
+
+        int curPos = startPos;
+
+        int plainPos = startPos;
+        int openPos = -1;
+
+        boolean insideLiteral = false;
+
+        LinkedList<OdbcEscapeParseResult> nested = null;
+
+        while (curPos < text.length()) {
+            char curChar = text.charAt(curPos);
+
+            if (curChar == '\'')
+                /* Escaped quote in odbc is two successive singe quotes. They'll flip flag twice without side-effect. */
+                insideLiteral = !insideLiteral;
+            else if (!insideLiteral) {
+                if (curChar == '{') {
+                    if (openPos == -1) {
+                        // Top-level opening brace. Append previous portion and remember current position.
+                        res.append(text, plainPos, curPos);
+
+                        openPos = curPos;
+                    }
+                    else {
+                        // Nested opening brace -> perform recursion.
+                        OdbcEscapeParseResult nestedRes = parse0(text, curPos, true);
+
+                        if (nested == null)
+                            nested = new LinkedList<>();
+
+                        nested.add(nestedRes);
+
+                        curPos += nestedRes.originalLength() - 1;
+
+                        plainPos = curPos + 1;
+                    }
+                }
+                else if (curChar == '}') {
+                    if (openPos == -1)
+                        // Close without open -> exception.
+                        throw new IgniteException("Malformed escape sequence " +
+                            "(closing curly brace without opening curly brace): " + text);
+                    else {
+                        String parseRes;
+
+                        if (nested == null)
+                            // Found sequence without nesting, process it.
+                            parseRes = parseEscapeSequence(text, openPos, curPos + 1 - openPos);
+                        else {
+                            // Special case to process nesting.
+                            String res0 = appendNested(text, openPos, curPos + 1, nested);
+
+                            nested = null;
+
+                            parseRes = parseEscapeSequence(res0, 0, res0.length());
+                        }
+
+                        if (earlyExit)
+                            return new OdbcEscapeParseResult(startPos, curPos + 1 - startPos, parseRes);
+                        else
+                            res.append(parseRes);
+
+                        openPos = -1;
+
+                        plainPos = curPos + 1;
+                    }
+                }
+            }
+
+            curPos++;
+        }
+
+        if (openPos != -1)
+            throw new IgniteException("Malformed escape sequence (closing curly brace missing): " + text);
+
+        if (insideLiteral)
+            throw new IgniteException("Malformed literal expression (closing quote missing): " + text);
+
+        if (curPos > plainPos)
+            res.append(text, plainPos, curPos);
+
+        return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, res.toString());
+    }
+
+    /**
+     * Parse escape sequence: {escape_sequence}.
+     *
+     * @param text Text.
+     * @param startPos Start position within text.
+     * @param len Length.
+     * @return Result.
+     */
+    private static String parseEscapeSequence(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+
+        char firstChar = text.charAt(startPos);
+
+        if (firstChar == '{') {
+            char lastChar = text.charAt(startPos + len - 1);
+
+            if (lastChar != '}')
+                throw new IgniteException("Failed to parse escape sequence because it is not enclosed: " +
+                    substring(text, startPos, len));
+
+            OdbcEscapeToken token = parseToken(text, startPos, len);
+
+            return parseEscapeSequence(text, startPos, len, token);
+        }
+        else {
+            // Nothing to escape, return original string.
+            if (startPos == 0 || text.length() == len)
+                return text;
+            else
+                return substring(text, startPos, len);
+        }
+    }
+
+    /**
+     * Get escape sequence info.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @return Escape sequence info.
+     */
+    private static OdbcEscapeToken parseToken(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+        assert text.charAt(startPos) == '{';
+
+        int pos = startPos + 1;
+
+        while (Character.isWhitespace(text.charAt(pos)))
+            pos++;
+
+        OdbcEscapeType curTyp = null;
+        boolean empty = false;
+
+        for (OdbcEscapeType typ : OdbcEscapeType.sortedValues()) {
+            if (text.startsWith(typ.body(), pos)) {
+                if (typ.standard())
+                    pos += typ.body().length();
+
+                empty = (startPos + len == pos + 1);
+
+                if (!empty && typ.standard()) {
+                    char charAfter = text.charAt(pos);
+
+                    if (!Character.isWhitespace(charAfter))
+                        throw new IgniteException("Unexpected escape sequence token: " +
+                            substring(text, startPos, len));
+                }
+
+                curTyp = typ;
+
+                break;
+            }
+        }
+
+        if (curTyp == null)
+            throw new IgniteException("Unsupported escape sequence: " + substring(text, startPos, len));
+
+        if (empty && !curTyp.allowEmpty())
+            throw new IgniteException("Escape sequence cannot be empty: " + substring(text, startPos, len));
+
+        return new OdbcEscapeToken(curTyp, pos - (startPos + 1));
+    }
+
+    /**
+     * Parse standard expression: {TOKEN expression}
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @param token Token.
+     * @return Result.
+     */
+    private static String parseEscapeSequence(String text, int startPos, int len, OdbcEscapeToken token) {
+        assert validSubstring(text, startPos, len);
+
+        // Get expression borders.
+        int startPos0 = startPos + 1 /* open brace */ + token.length() /* token. */;
+        int len0 = len - 1 /* open brace */ - token.length() /* token */ - 1 /* close brace */;
+
+        switch (token.type()) {
+            case SCALAR_FUNCTION:
+                return parseScalarFunctionExpression(text, startPos0, len0);
+
+            case GUID: {
+                String res = parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN);
+
+                return "CAST(" + res + " AS UUID)";
+            }
+
+            case DATE:
+                return parseExpression(text, startPos0, len0, token.type(), DATE_PATTERN);
+
+            case TIME:
+                return parseExpression(text, startPos0, len0, token.type(), TIME_PATTERN);
+
+            case TIMESTAMP:
+                return parseExpression(text, startPos0, len0, token.type(), TIMESTAMP_PATTERN);
+
+            case OUTER_JOIN:
+                return parseExpression(text, startPos0, len0);
+
+            case CALL: {
+                String val = parseExpression(text, startPos0, len0);
+
+                return "CALL " + val;
+            }
+
+            case ESCAPE:
+            case ESCAPE_WO_TOKEN:
+                return parseLikeEscCharacterExpression(text, startPos0, len0);
+
+            default:
+                throw new IgniteException("Unsupported escape sequence token [text=" +
+                    substring(text, startPos, len) + ", token=" + token.type().body() + ']');
+        }
+    }
+
+    /**
+     * Parse simple expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseExpression(String text, int startPos, int len) {
+        return substring(text, startPos, len).trim();
+    }
+
+    /**
+     * Parse LIKE escape character expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseLikeEscCharacterExpression(String text, int startPos, int len) {
+        return "ESCAPE " + substring(text, startPos, len).trim();
+    }
+
+    /**
+     * Parse expression and validate against ODBC specification with regex pattern.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseExpression(String text, int startPos, int len, OdbcEscapeType type, Pattern pattern) {
+        String val = parseExpression(text, startPos, len);
+
+        if (!pattern.matcher(val).matches())
+            throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len));
+
+        return val;
+    }
+
+    /**
+     * Parse scalar function expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseScalarFunctionExpression(String text, int startPos, int len) {
+        int pos = startPos;
+        int endPos = startPos + len;
+        final String errPrefix = "Malformed scalar function escape sequence.";
+
+        while ((++pos < endPos) && Character.isWhitespace(text.charAt(pos)));
+        if (pos == endPos)
+            throw new IgniteException(errPrefix + " Expected function name.");
+
+        int funcNamePos = pos;
+        while ((++pos < endPos) && Character.isAlphabetic(text.charAt(pos)));
+        if (pos == endPos)
+            throw new IgniteException(errPrefix + " Expected function parameter list: " +
+                                      substring(text, startPos, len));
+
+        String funcName = text.substring(funcNamePos, pos);
+
+        switch (funcName.toUpperCase()) {
+            case "CONVERT": {
+                Matcher matcher = CONVERT_TYPE_PATTERN.matcher(text.substring(startPos, endPos));
+
+                if (!matcher.find())
+                    throw new IgniteException(errPrefix + " Invalid arguments :" +
+                                              substring(text, startPos, len));
+
+                return (text.substring(startPos, startPos + matcher.start(1)) +
+                        OdbcUtils.getIgniteTypeFromOdbcType(matcher.group(1)) +
+                        text.substring(startPos + matcher.end(1), startPos + len)).trim();
+            }
+            default:
+                return substring(text, startPos, len).trim();
+        }
+    }
+
+    /**
+     * Append nested results.
+     *
+     * @param text Original text.
+     * @param startPos Start position.
+     * @param endPos End position.
+     * @param nestedRess Nested results.
+     * @return Result.
+     */
+    private static String appendNested(String text, int startPos, int endPos,
+        LinkedList<OdbcEscapeParseResult> nestedRess) {
+        StringBuilder res = new StringBuilder();
+
+        int curPos = startPos;
+
+        for (OdbcEscapeParseResult nestedRes : nestedRess) {
+            // Append text between current position and replace.
+            res.append(text, curPos, nestedRes.originalStart());
+
+            // Append replaced text.
+            res.append(nestedRes.result());
+
+            // Advance position.
+            curPos = nestedRes.originalStart() + nestedRes.originalLength();
+        }
+
+        // Append remainder.
+        res.append(text, curPos, endPos);
+
+        return res.toString();
+    }
+
+    /**
+     * Perform "substring" using start position and length.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Substring.
+     */
+    private static String substring(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+
+        return text.substring(startPos, startPos + len);
+    }
+
+    /**
+     * Check whether substring is valid.
+     *
+     * @param text Substring.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return {@code True} if valid.
+     */
+    private static boolean validSubstring(String text, int startPos, int len) {
+        return text != null && startPos + len <= text.length();
+    }
+
+    /**
+     * Private constructor.
+     */
+    private OdbcEscapeUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index ca5f756..8349687 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1220,7 +1220,7 @@ org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem
 org.apache.ignite.internal.processors.marshaller.MissingMappingRequestMessage
 org.apache.ignite.internal.processors.marshaller.MissingMappingResponseMessage
 org.apache.ignite.internal.processors.odbc.OdbcProtocolVersion
-org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeType
+org.apache.ignite.internal.processors.odbc.odbc.escape.OdbcEscapeType
 org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure
 org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate
 org.apache.ignite.internal.processors.platform.PlatformEventFilterListener

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
index ecb6c2d..c08c40c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeUtils;
+import org.apache.ignite.internal.processors.odbc.odbc.escape.OdbcEscapeUtils;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 3893ad9..d182f93 100644
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@ -276,8 +276,7 @@ BOOST_AUTO_TEST_CASE(TestConnectStringInvalidVersion)
 
 BOOST_AUTO_TEST_CASE(TestConnectStringValidVersion)
 {
-    CheckValidProtocolVersion("Protocol_Version=1.6.0;", ignite::odbc::ProtocolVersion::VERSION_1_6_0);
-    CheckValidProtocolVersion("Protocol_Version=1.8.0;", ignite::odbc::ProtocolVersion::VERSION_1_8_0);
+    CheckValidProtocolVersion("Protocol_Version=2.0.0;", ignite::odbc::ProtocolVersion::VERSION_2_0_0);
 }
 
 BOOST_AUTO_TEST_CASE(TestConnectStringInvalidBoolKeys)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 fdf3503..0f9eaee 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -378,14 +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_1_6_0)
+BOOST_AUTO_TEST_CASE(TestConnectionProtocolVERSION_2_0_0)
 {
-    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=1.6.0");
-}
-
-BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_1_8_0)
-{
-    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=1.8.0");
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=2.0.0");
 }
 
 BOOST_AUTO_TEST_CASE(TestTwoRowsInt8)
@@ -950,7 +945,6 @@ BOOST_AUTO_TEST_CASE(TestNullFields)
     BOOST_CHECK(ret == SQL_NO_DATA);
 }
 
-
 BOOST_AUTO_TEST_CASE(TestDistributedJoins)
 {
     // Starting additional node.
@@ -1025,58 +1019,6 @@ BOOST_AUTO_TEST_CASE(TestDistributedJoins)
     BOOST_CHECK_EQUAL(rowsNum, entriesNum);
 }
 
-BOOST_AUTO_TEST_CASE(TestDistributedJoinsWithOldVersion)
-{
-    // Starting additional node.
-    Ignite node1 = StartAdditionalNode("Node1");
-    Ignite node2 = StartAdditionalNode("Node2");
-
-    const int entriesNum = 1000;
-
-    // Filling cache with data.
-    for (int i = 0; i < entriesNum; ++i)
-    {
-        TestType entry;
-
-        entry.i32Field = i;
-        entry.i64Field = entriesNum - i - 1;
-
-        cache1.Put(i, entry);
-    }
-
-    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;DISTRIBUTED_JOINS=true;PROTOCOL_VERSION=1.6.0");
-
-    SQLRETURN ret;
-
-    const size_t columnsCnt = 2;
-
-    SQLBIGINT columns[columnsCnt] = { 0 };
-
-    // Binding colums.
-    for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
-    {
-        ret = SQLBindCol(stmt, i + 1, SQL_C_SLONG, &columns[i], 0, 0);
-
-        if (!SQL_SUCCEEDED(ret))
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
-    }
-
-    SQLCHAR request[] =
-        "SELECT T0.i32Field, T1.i64Field FROM TestType AS T0 "
-        "INNER JOIN TestType AS T1 "
-        "ON (T0.i32Field = T1.i64Field)";
-
-    ret = SQLExecDirect(stmt, request, SQL_NTS);
-
-    if (!SQL_SUCCEEDED(ret))
-        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
-
-    int rowsNum = CountRows(stmt);
-
-    BOOST_CHECK_GT(rowsNum, 0);
-    BOOST_CHECK_LT(rowsNum, entriesNum);
-}
-
 BOOST_AUTO_TEST_CASE(TestInsertSelect)
 {
     Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache");

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 d3d288e..bb42dd4 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
@@ -37,11 +37,8 @@ namespace ignite
             /** 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;
-
-            /** First version of the protocol that was introduced in Ignite 1.8.0. */
-            static const ProtocolVersion VERSION_1_8_0;
+            /** 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;
@@ -93,13 +90,6 @@ namespace ignite
             bool IsUnknown() const;
 
             /**
-             * Check if the distributed joins supported.
-             *
-             * @retuen True if the distributed joins supported.
-             */
-            bool IsDistributedJoinsSupported() const;
-
-            /**
              * Comparison operator.
              *
              * @param val1 First value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 c839999..4a88052 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
@@ -159,12 +159,6 @@ namespace ignite
                     enforceJoinOrderCheckBox = CreateCheckBox(editPosX + checkBoxSize + interval, rowPos, checkBoxSize,
                         rowSize, "Enforce Join Order", ChildId::ENFORCE_JOIN_ORDER_CHECK_BOX, config.IsEnforceJoinOrder());
 
-                    if (!config.GetProtocolVersion().IsDistributedJoinsSupported())
-                    {
-                        distributedJoinsCheckBox->SetEnabled(false);
-                        enforceJoinOrderCheckBox->SetEnabled(false);
-                    }
-
                     rowPos += interval * 2 + rowSize;
 
                     connectionSettingsGroupBox = CreateGroupBox(margin, sectionBegin, width - 2 * margin,
@@ -206,31 +200,6 @@ namespace ignite
                                     break;
                                 }
 
-                                case ChildId::PROTOCOL_VERSION_COMBO_BOX:
-                                {
-                                    if (HIWORD(wParam) == CBN_SELCHANGE)
-                                    {
-                                        std::string text;
-
-                                        protocolVersionComboBox->GetText(text);
-
-                                        ProtocolVersion version = ProtocolVersion::FromString(text);
-
-                                        if (!version.IsUnknown() && !version.IsDistributedJoinsSupported())
-                                        {
-                                            distributedJoinsCheckBox->SetEnabled(false);
-                                            enforceJoinOrderCheckBox->SetEnabled(false);
-                                        }
-                                        else
-                                        {
-                                            distributedJoinsCheckBox->SetEnabled(true);
-                                            enforceJoinOrderCheckBox->SetEnabled(true);
-                                        }
-                                    }
-
-                                    break;
-                                }
-
                                 case IDCANCEL:
                                 case ChildId::CANCEL_BUTTON:
                                 {
@@ -253,6 +222,7 @@ namespace ignite
                                     break;
                                 }
 
+                                case ChildId::PROTOCOL_VERSION_COMBO_BOX:
                                 default:
                                     return false;
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 ebd3b6a..859135d 100644
--- a/modules/platforms/cpp/odbc/src/protocol_version.cpp
+++ b/modules/platforms/cpp/odbc/src/protocol_version.cpp
@@ -25,21 +25,18 @@ namespace ignite
 {
     namespace odbc
     {
-        const ProtocolVersion ProtocolVersion::VERSION_1_6_0(1);
-        const ProtocolVersion ProtocolVersion::VERSION_1_8_0(MakeVersion(1,8,0));
+        const ProtocolVersion ProtocolVersion::VERSION_2_0_0(MakeVersion(2,0,0));
         const ProtocolVersion ProtocolVersion::VERSION_UNKNOWN(INT64_MIN);
 
         ProtocolVersion::StringToVersionMap::value_type s2vInitVals[] = {
-            std::make_pair("1.6.0", ProtocolVersion::VERSION_1_6_0),
-            std::make_pair("1.8.0", ProtocolVersion::VERSION_1_8_0)
+            std::make_pair("2.0.0", ProtocolVersion::VERSION_2_0_0)
         };
 
         const ProtocolVersion::StringToVersionMap ProtocolVersion::stringToVersionMap(s2vInitVals,
             s2vInitVals + (sizeof(s2vInitVals) / sizeof(s2vInitVals[0])));
 
         ProtocolVersion::VersionToStringMap::value_type v2sInitVals[] = {
-            std::make_pair(ProtocolVersion::VERSION_1_6_0, "1.6.0"),
-            std::make_pair(ProtocolVersion::VERSION_1_8_0, "1.8.0")
+            std::make_pair(ProtocolVersion::VERSION_2_0_0, "2.0.0")
         };
 
         const ProtocolVersion::VersionToStringMap ProtocolVersion::versionToStringMap(v2sInitVals,
@@ -64,7 +61,7 @@ namespace ignite
 
         const ProtocolVersion& ProtocolVersion::GetCurrent()
         {
-            return VERSION_1_8_0;
+            return VERSION_2_0_0;
         }
 
         ProtocolVersion ProtocolVersion::FromString(const std::string& version)
@@ -106,11 +103,6 @@ namespace ignite
             return *this == VERSION_UNKNOWN;
         }
 
-        bool ProtocolVersion::IsDistributedJoinsSupported() const
-        {
-            return *this >= VERSION_1_8_0;
-        }
-
         bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
             return val1.val == val2.val;