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/04 12:37:30 UTC

[2/3] ignite git commit: IGNITE-5167: ODBC: extracted handler/parser interfaces.

IGNITE-5167: ODBC: extracted handler/parser interfaces.


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

Branch: refs/heads/master
Commit: 4e5a82d657fb690d90aea33617ec05fffdb024c2
Parents: 7d288c7
Author: devozerov <vo...@gridgain.com>
Authored: Thu May 4 15:36:41 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu May 4 15:36:41 2017 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcConnectionData.java     |  60 ---
 .../processors/odbc/OdbcHandshakeRequest.java   |  83 ---
 .../processors/odbc/OdbcHandshakeResult.java    |  73 ---
 .../processors/odbc/OdbcMessageParser.java      | 313 -----------
 .../processors/odbc/OdbcNioListener.java        |  37 +-
 .../processors/odbc/OdbcRequestHandler.java     | 530 ------------------
 .../internal/processors/odbc/OdbcResponse.java  |  96 ----
 .../internal/processors/odbc/OdbcUtils.java     |   2 +-
 .../odbc/SqlListenerConnectionContext.java      |  56 ++
 .../odbc/SqlListenerHandshakeRequest.java       |  83 +++
 .../odbc/SqlListenerHandshakeResult.java        |  73 +++
 .../odbc/SqlListenerMessageParser.java          |  39 ++
 .../odbc/SqlListenerProtocolVersion.java        |   4 +-
 .../processors/odbc/SqlListenerRequest.java     |  17 +
 .../odbc/SqlListenerRequestHandler.java         |  31 ++
 .../processors/odbc/SqlListenerResponse.java    |  96 ++++
 .../odbc/escape/OdbcEscapeParseResult.java      |  73 ---
 .../processors/odbc/escape/OdbcEscapeToken.java |  61 ---
 .../processors/odbc/escape/OdbcEscapeType.java  | 112 ----
 .../processors/odbc/escape/OdbcEscapeUtils.java | 440 ---------------
 .../processors/odbc/odbc/OdbcMessageParser.java | 322 +++++++++++
 .../odbc/odbc/OdbcRequestHandler.java           | 540 +++++++++++++++++++
 .../odbc/odbc/escape/OdbcEscapeParseResult.java |  73 +++
 .../odbc/odbc/escape/OdbcEscapeToken.java       |  61 +++
 .../odbc/odbc/escape/OdbcEscapeType.java        | 112 ++++
 .../odbc/odbc/escape/OdbcEscapeUtils.java       | 441 +++++++++++++++
 .../resources/META-INF/classnames.properties    |   2 +-
 .../odbc/OdbcEscapeSequenceSelfTest.java        |   2 +-
 .../cpp/odbc-test/src/configuration_test.cpp    |   3 +-
 .../cpp/odbc-test/src/queries_test.cpp          |  62 +--
 .../odbc/include/ignite/odbc/protocol_version.h |  14 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  32 +-
 .../platforms/cpp/odbc/src/protocol_version.cpp |  16 +-
 33 files changed, 1980 insertions(+), 1979 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcConnectionData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcConnectionData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcConnectionData.java
deleted file mode 100644
index 28a903d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcConnectionData.java
+++ /dev/null
@@ -1,60 +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.GridKernalContext;
-import org.apache.ignite.internal.util.GridSpinBusyLock;
-
-/**
- * Connection-related data.
- */
-class OdbcConnectionData {
-    /** Request handler. */
-    private final OdbcRequestHandler handler;
-
-    /** Message parser. */
-    private final OdbcMessageParser parser;
-
-    /**
-     * Constructor.
-     *
-     * @param ctx Context.
-     * @param busyLock Shutdown busy lock.
-     * @param maxCursors Maximum cursors.
-     */
-    public OdbcConnectionData(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) {
-        handler = new OdbcRequestHandler(ctx, busyLock, maxCursors);
-        parser = new OdbcMessageParser(ctx);
-    }
-
-    /**
-     * Handler getter.
-     * @return Request handler for the connection.
-     */
-    public OdbcRequestHandler getHandler() {
-        return handler;
-    }
-
-    /**
-     * Parser getter
-     * @return Message parser for the connection.
-     */
-    public OdbcMessageParser getParser() {
-        return parser;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java
deleted file mode 100644
index 7d337b7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.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 OdbcHandshakeRequest 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 OdbcHandshakeRequest(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(OdbcHandshakeRequest.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java
deleted file mode 100644
index 74c5bd4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.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 OdbcHandshakeResult {
-    /** 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 OdbcHandshakeResult(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(OdbcHandshakeResult.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
deleted file mode 100644
index 517a1a4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
+++ /dev/null
@@ -1,313 +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.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 java.util.Collection;
-
-/**
- * ODBC message parser.
- */
-public class OdbcMessageParser {
-    /** 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());
-    }
-
-    /**
-     * Decode OdbcRequest from byte array.
-     *
-     * @param msg Message.
-     * @return Assembled ODBC request.
-     */
-    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();
-
-                OdbcHandshakeRequest res = new OdbcHandshakeRequest(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;
-    }
-
-    /**
-     * Encode OdbcResponse to byte array.
-     *
-     * @param msg Message.
-     * @return Byte array.
-     */
-    public byte[] encode(OdbcResponse 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() != OdbcResponse.STATUS_SUCCESS) {
-            writer.writeString(msg.error());
-
-            return writer.array();
-        }
-
-        Object res0 = msg.response();
-
-        if (res0 == null)
-            return writer.array();
-        if (res0 instanceof OdbcHandshakeResult) {
-            OdbcHandshakeResult res = (OdbcHandshakeResult) 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/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 b0146e5..423c413 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,8 @@ package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
+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;
 import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
 import org.apache.ignite.internal.util.nio.GridNioSession;
@@ -32,7 +34,7 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
     /** Connection-related metadata key. */
-    private static final int CONN_DATA_META_KEY = GridNioSessionMetaKey.nextUniqueKey();
+    private static final int CONN_CTX_META_KEY = GridNioSessionMetaKey.nextUniqueKey();
 
     /** Request ID generator. */
     private static final AtomicLong REQ_ID_GEN = new AtomicLong();
@@ -67,18 +69,21 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
     /** {@inheritDoc} */
     @Override public void onConnected(GridNioSession ses) {
         if (log.isDebugEnabled())
-            log.debug("ODBC client connected: " + ses.remoteAddress());
+            log.debug("SQL client connected: " + ses.remoteAddress());
 
-        ses.addMeta(CONN_DATA_META_KEY, new OdbcConnectionData(ctx, busyLock, maxCursors));
+        OdbcRequestHandler handler = new OdbcRequestHandler(ctx, busyLock, maxCursors);
+        OdbcMessageParser parser = new OdbcMessageParser(ctx);
+
+        ses.addMeta(CONN_CTX_META_KEY, new SqlListenerConnectionContext(handler, parser));
     }
 
     /** {@inheritDoc} */
     @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
         if (log.isDebugEnabled()) {
             if (e == null)
-                log.debug("ODBC client disconnected: " + ses.remoteAddress());
+                log.debug("SQL client disconnected: " + ses.remoteAddress());
             else
-                log.debug("ODBC client disconnected due to an error [addr=" + ses.remoteAddress() + ", err=" + e + ']');
+                log.debug("SQL client disconnected due to an error [addr=" + ses.remoteAddress() + ", err=" + e + ']');
         }
     }
 
@@ -86,13 +91,11 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
     @Override public void onMessage(GridNioSession ses, byte[] msg) {
         assert msg != null;
 
-        long reqId = REQ_ID_GEN.incrementAndGet();
-
-        OdbcConnectionData connData = ses.meta(CONN_DATA_META_KEY);
+        SqlListenerConnectionContext connData = ses.meta(CONN_CTX_META_KEY);
 
         assert connData != null;
 
-        OdbcMessageParser parser = connData.getParser();
+        SqlListenerMessageParser parser = connData.parser();
 
         SqlListenerRequest req;
 
@@ -100,7 +103,7 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
             req = parser.decode(msg);
         }
         catch (Exception e) {
-            log.error("Failed to parse message [id=" + reqId + ", err=" + e + ']');
+            log.error("Failed to parse SQL client request [err=" + e + ']');
 
             ses.close();
 
@@ -109,24 +112,26 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
 
         assert req != null;
 
+        req.requestId(REQ_ID_GEN.incrementAndGet());
+
         try {
             long startTime = 0;
 
             if (log.isDebugEnabled()) {
                 startTime = System.nanoTime();
 
-                log.debug("ODBC request received [id=" + reqId + ", addr=" + ses.remoteAddress() +
+                log.debug("SQL client request received [reqId=" + req.requestId() + ", addr=" + ses.remoteAddress() +
                     ", req=" + req + ']');
             }
 
-            OdbcRequestHandler handler = connData.getHandler();
+            SqlListenerRequestHandler handler = connData.handler();
 
-            OdbcResponse resp = handler.handle(reqId, req);
+            SqlListenerResponse resp = handler.handle(req);
 
             if (log.isDebugEnabled()) {
                 long dur = (System.nanoTime() - startTime) / 1000;
 
-                log.debug("ODBC request processed [id=" + reqId + ", dur(mcs)=" + dur  +
+                log.debug("SQL client request processed [reqId=" + req.requestId() + ", dur(mcs)=" + dur  +
                     ", resp=" + resp.status() + ']');
             }
 
@@ -135,9 +140,9 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
             ses.send(outMsg);
         }
         catch (Exception e) {
-            log.error("Failed to process ODBC request [id=" + reqId + ", err=" + e + ']');
+            log.error("Failed to process SQL client request [reqId=" + req.requestId() + ", err=" + e + ']');
 
-            ses.send(parser.encode(new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString())));
+            ses.send(parser.encode(new SqlListenerResponse(SqlListenerResponse.STATUS_FAILED, e.toString())));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
deleted file mode 100644
index d98c366..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ /dev/null
@@ -1,530 +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.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.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 {
-    /** 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(OdbcRequestHandler.class);
-    }
-
-    /**
-     * Handle request.
-     *
-     * @param reqId Request ID.
-     * @param req Request.
-     * @return Response.
-     */
-    public OdbcResponse handle(long reqId, SqlListenerRequest req) {
-        assert req != null;
-
-        if (!busyLock.enterBusy())
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED,
-                    "Failed to handle ODBC request because node is stopping: " + req);
-
-        try {
-            switch (req.command()) {
-                case HANDSHAKE:
-                    return performHandshake(reqId, (OdbcHandshakeRequest)req);
-
-                case QRY_EXEC:
-                    return executeQuery(reqId, (SqlListenerQueryExecuteRequest)req);
-
-                case QRY_FETCH:
-                    return fetchQuery(reqId, (SqlListenerQueryFetchRequest)req);
-
-                case QRY_CLOSE:
-                    return closeQuery(reqId, (SqlListenerQueryCloseRequest)req);
-
-                case META_COLS:
-                    return getColumnsMeta(reqId, (OdbcQueryGetColumnsMetaRequest)req);
-
-                case META_TBLS:
-                    return getTablesMeta(reqId, (OdbcQueryGetTablesMetaRequest)req);
-
-                case META_PARAMS:
-                    return getParamsMeta(reqId, (OdbcQueryGetParamsMetaRequest)req);
-            }
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
-     * {@link OdbcHandshakeRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Handshake request.
-     * @return Response.
-     */
-    private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest 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();
-
-                OdbcHandshakeResult res = new OdbcHandshakeResult(false, OdbcUtils.VER_LATEST.since(), verStr);
-
-                return new OdbcResponse(res);
-            }
-
-            OdbcHandshakeResult res = new OdbcHandshakeResult(true, null, null);
-
-            distributedJoins = req.distributedJoins();
-            enforceJoinOrder = req.enforceJoinOrder();
-
-            return new OdbcResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to perform handshake [reqId=" + reqId + ", req=" + req + ']', e);
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link SqlListenerQueryExecuteRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Execute query request.
-     * @return Response.
-     */
-    private OdbcResponse executeQuery(long reqId, SqlListenerQueryExecuteRequest req) {
-        int cursorCnt = qryCursors.size();
-
-        if (maxCursors > 0 && cursorCnt >= maxCursors)
-            return new OdbcResponse(OdbcResponse.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=" + reqId + ", 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 OdbcResponse(OdbcResponse.STATUS_FAILED,
-                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
-
-            IgniteCache<Object, Object> cache = cache0.withKeepBinary();
-
-            if (cache == null)
-                return new OdbcResponse(OdbcResponse.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 OdbcResponse(res);
-        }
-        catch (Exception e) {
-            qryCursors.remove(qryId);
-
-            U.error(log, "Failed to execute SQL query [reqId=" + reqId + ", req=" + req + ']', e);
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link SqlListenerQueryCloseRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Execute query request.
-     * @return Response.
-     */
-    private OdbcResponse closeQuery(long reqId, SqlListenerQueryCloseRequest req) {
-        try {
-            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
-
-            if (tuple == null)
-                return new OdbcResponse(OdbcResponse.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 OdbcResponse(res);
-        }
-        catch (Exception e) {
-            qryCursors.remove(req.queryId());
-
-            U.error(log, "Failed to close SQL query [reqId=" + reqId + ", req=" + req.queryId() + ']', e);
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link SqlListenerQueryFetchRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Execute query request.
-     * @return Response.
-     */
-    private OdbcResponse fetchQuery(long reqId, SqlListenerQueryFetchRequest req) {
-        try {
-            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
-
-            if (tuple == null)
-                return new OdbcResponse(OdbcResponse.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 OdbcResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to fetch SQL query result [reqId=" + reqId + ", req=" + req + ']', e);
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link OdbcQueryGetColumnsMetaRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Get columns metadata request.
-     * @return Response.
-     */
-    private OdbcResponse getColumnsMeta(long reqId, 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 OdbcResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to get columns metadata [reqId=" + reqId + ", req=" + req + ']', e);
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link OdbcQueryGetTablesMetaRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Get tables metadata request.
-     * @return Response.
-     */
-    private OdbcResponse getTablesMeta(long reqId, 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 OdbcResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to get tables metadata [reqId=" + reqId + ", req=" + req + ']', e);
-
-            return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.toString());
-        }
-    }
-
-    /**
-     * {@link OdbcQueryGetParamsMetaRequest} command handler.
-     *
-     * @param reqId Request ID.
-     * @param req Get params metadata request.
-     * @return Response.
-     */
-    private OdbcResponse getParamsMeta(long reqId, 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 OdbcResponse(res);
-        }
-        catch (Exception e) {
-            U.error(log, "Failed to get params metadata [reqId=" + reqId + ", req=" + req + ']', e);
-
-            return new OdbcResponse(OdbcResponse.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/OdbcResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcResponse.java
deleted file mode 100644
index 12ac4bc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcResponse.java
+++ /dev/null
@@ -1,96 +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.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * ODBC protocol response.
- */
-public class OdbcResponse {
-    /** Command succeeded. */
-    public static final int STATUS_SUCCESS = 0;
-
-    /** Command failed. */
-    public static final int STATUS_FAILED = 1;
-
-    /** Success status. */
-    private final int status;
-
-    /** Error. */
-    private final String err;
-
-    /** Response object. */
-    @GridToStringInclude
-    private final Object obj;
-
-    /**
-     * Constructs successful rest response.
-     *
-     * @param obj Response object.
-     */
-    public OdbcResponse(Object obj) {
-        this.status = STATUS_SUCCESS;
-
-        this.obj = obj;
-        this.err = null;
-    }
-
-    /**
-     * Constructs failed rest response.
-     *
-     * @param status Response status.
-     * @param err Error, {@code null} if success is {@code true}.
-     */
-    public OdbcResponse(int status, @Nullable String err) {
-        assert status != STATUS_SUCCESS;
-
-        this.status = status;
-
-        this.obj = null;
-        this.err = err;
-    }
-
-    /**
-     * @return Success flag.
-     */
-    public int status() {
-        return status;
-    }
-
-    /**
-     * @return Response object.
-     */
-    public Object response() {
-        return obj;
-    }
-
-    /**
-     * @return Error.
-     */
-    public String error() {
-        return err;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(OdbcResponse.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 8bf1959..da56813 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
@@ -24,7 +24,7 @@ import org.apache.ignite.IgniteException;
  */
 public class OdbcUtils {
     /** Latest version. */
-    public static final SqlListenerProtocolVersion VER_LATEST = SqlListenerProtocolVersion.VER_2_1_0;
+    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/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerConnectionContext.java
new file mode 100644
index 0000000..759f3d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerConnectionContext.java
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/**
+ * SQL listener connection context.
+ */
+public class SqlListenerConnectionContext {
+    /** Request handler. */
+    private final SqlListenerRequestHandler handler;
+
+    /** Message parser. */
+    private final SqlListenerMessageParser parser;
+
+    /**
+     * Constructor.
+     *
+     * @param handler Handler.
+     * @param parser Parser.
+     */
+    public SqlListenerConnectionContext(SqlListenerRequestHandler handler, SqlListenerMessageParser parser) {
+        this.handler = handler;
+        this.parser = parser;
+    }
+
+    /**
+     * Handler getter.
+     * @return Request handler for the connection.
+     */
+    public SqlListenerRequestHandler handler() {
+        return handler;
+    }
+
+    /**
+     * Parser getter
+     * @return Message parser for the connection.
+     */
+    public SqlListenerMessageParser parser() {
+        return parser;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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
new file mode 100644
index 0000000..09ac05a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeRequest.java
@@ -0,0 +1,83 @@
+/*
+ * 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/4e5a82d6/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
new file mode 100644
index 0000000..ca8b629
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerHandshakeResult.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;
+
+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/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerMessageParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerMessageParser.java
new file mode 100644
index 0000000..c59ea54
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerMessageParser.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/**
+ * SQL listener message parser.
+ */
+public interface SqlListenerMessageParser {
+    /**
+     * Decode request from byte array.
+     *
+     * @param msg Message.
+     * @return Request.
+     */
+    public SqlListenerRequest decode(byte[] msg);
+
+    /**
+     * Encode response to byte array.
+     *
+     * @param resp Response.
+     * @return Message.
+     */
+    public byte[] encode(SqlListenerResponse resp);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/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 372188c..62e07ee 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
@@ -21,8 +21,8 @@ package org.apache.ignite.internal.processors.odbc;
  * SQL listener protocol version.
  */
 public enum SqlListenerProtocolVersion {
-    /** Version 2.1.0. */
-    VER_2_1_0(makeVersion(2, 1, 0), "2.1.0"),
+    /** Version 2.0.0. */
+    VER_2_0_0(makeVersion(2, 0, 0), "2.0.0"),
 
     /** Unknown version. */
     UNKNOWN(Long.MIN_VALUE, "UNKNOWN");

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequest.java
index 96f24c8..2714237 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequest.java
@@ -45,6 +45,9 @@ public class SqlListenerRequest {
     /** Command. */
     private final int cmd;
 
+    /** Request ID. */
+    private long reqId;
+
     /**
      * @param cmd Command type.
      */
@@ -58,4 +61,18 @@ public class SqlListenerRequest {
     public int command() {
         return cmd;
     }
+
+    /**
+     * @return Request ID.
+     */
+    public long requestId() {
+        return reqId;
+    }
+
+    /**
+     * @param reqId Request ID.
+     */
+    public void requestId(long reqId) {
+        this.reqId = reqId;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
new file mode 100644
index 0000000..0ebb084
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+/**
+ * SQL listener request handler.
+ */
+public interface SqlListenerRequestHandler {
+    /**
+     * Handle request.
+     *
+     * @param req Request.
+     * @return Response.
+     */
+    public SqlListenerResponse handle(SqlListenerRequest req);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerResponse.java
new file mode 100644
index 0000000..84c1e26
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerResponse.java
@@ -0,0 +1,96 @@
+/*
+ * 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.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * SQL listener response.
+ */
+public class SqlListenerResponse {
+    /** Command succeeded. */
+    public static final int STATUS_SUCCESS = 0;
+
+    /** Command failed. */
+    public static final int STATUS_FAILED = 1;
+
+    /** Success status. */
+    private final int status;
+
+    /** Error. */
+    private final String err;
+
+    /** Response object. */
+    @GridToStringInclude
+    private final Object obj;
+
+    /**
+     * Constructs successful rest response.
+     *
+     * @param obj Response object.
+     */
+    public SqlListenerResponse(Object obj) {
+        this.status = STATUS_SUCCESS;
+
+        this.obj = obj;
+        this.err = null;
+    }
+
+    /**
+     * Constructs failed rest response.
+     *
+     * @param status Response status.
+     * @param err Error, {@code null} if success is {@code true}.
+     */
+    public SqlListenerResponse(int status, @Nullable String err) {
+        assert status != STATUS_SUCCESS;
+
+        this.status = status;
+
+        this.obj = null;
+        this.err = err;
+    }
+
+    /**
+     * @return Success flag.
+     */
+    public int status() {
+        return status;
+    }
+
+    /**
+     * @return Response object.
+     */
+    public Object response() {
+        return obj;
+    }
+
+    /**
+     * @return Error.
+     */
+    public String error() {
+        return err;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SqlListenerResponse.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e5a82d6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java
deleted file mode 100644
index cf05651..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.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.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/escape/OdbcEscapeToken.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java
deleted file mode 100644
index 6bb4f81..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java
+++ /dev/null
@@ -1,61 +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.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/escape/OdbcEscapeType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
deleted file mode 100644
index 44d8361..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
+++ /dev/null
@@ -1,112 +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;
-
-/**
- * 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;
-    }
-}