http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/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
deleted file mode 100644
index 3438d5a..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerConnectionContext.java
+++ /dev/null
@@ -1,62 +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.binary.BinaryReaderExImpl;
-
-/**
- * SQL listener connection context.
- */
-public interface SqlListenerConnectionContext {
-    /**
-     * @param ver Version to check.
-     * @return {@code true} if version is supported.
-     */
-    boolean isVersionSupported(SqlListenerProtocolVersion ver);
-
-    /**
-     * @return Current context version.
-     */
-    SqlListenerProtocolVersion currentVersion();
-
-    /**
-     * Initialize from handshake message.
-     *
-     * @param ver Protocol version.
-     * @param reader Reader set to the configuration part of the handshake 
message.
-     */
-    void initializeFromHandshake(SqlListenerProtocolVersion ver, 
BinaryReaderExImpl reader);
-
-    /**
-     * Handler getter.
-     * @return Request handler for the connection.
-     */
-    SqlListenerRequestHandler handler();
-
-    /**
-     * Parser getter
-     * @return Message parser for the connection.
-     */
-    SqlListenerMessageParser parser();
-
-    /**
-     * Called whenever client is disconnected due to correct connection close
-     * or due to {@code IOException} during network operations.
-     */
-    void onDisconnected();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/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
deleted file mode 100644
index c59ea54..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerMessageParser.java
+++ /dev/null
@@ -1,39 +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;
-
-/**
- * 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/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
deleted file mode 100644
index 81539dc..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
+++ /dev/null
@@ -1,248 +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 java.util.concurrent.atomic.AtomicLong;
-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.streams.BinaryHeapInputStream;
-import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
-import org.apache.ignite.internal.binary.streams.BinaryInputStream;
-import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
-import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext;
-import 
org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
-import 
org.apache.ignite.internal.processors.platform.client.ClientMessageParser;
-import 
org.apache.ignite.internal.processors.platform.client.ClientRequestHandler;
-import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
-import org.apache.ignite.internal.util.nio.GridNioSession;
-import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * SQL message listener.
- */
-public class SqlListenerNioListener extends 
GridNioServerListenerAdapter<byte[]> {
-    /** ODBC driver handshake code. */
-    public static final byte ODBC_CLIENT = 0;
-
-    /** JDBC driver handshake code. */
-    public static final byte JDBC_CLIENT = 1;
-
-    /** Thin client handshake code. */
-    public static final byte THIN_CLIENT = 2;
-
-    /** Connection-related metadata key. */
-    private static final int CONN_CTX_META_KEY = 
GridNioSessionMetaKey.nextUniqueKey();
-
-    /** Busy lock. */
-    private final GridSpinBusyLock busyLock;
-
-    /** Kernal context. */
-    private final GridKernalContext ctx;
-
-    /** Maximum allowed cursors. */
-    private final int maxCursors;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /**
-     * Constructor.
-     *
-     * @param ctx Context.
-     * @param busyLock Shutdown busy lock.
-     * @param maxCursors Maximum allowed cursors.
-     */
-    public SqlListenerNioListener(GridKernalContext ctx, GridSpinBusyLock 
busyLock, int maxCursors) {
-        this.ctx = ctx;
-        this.busyLock = busyLock;
-        this.maxCursors = maxCursors;
-
-        log = ctx.log(getClass());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onConnected(GridNioSession ses) {
-        if (log.isDebugEnabled())
-            log.debug("SQL client connected: " + ses.remoteAddress());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDisconnected(GridNioSession ses, @Nullable 
Exception e) {
-        SqlListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY);
-
-        if (connCtx != null)
-            connCtx.onDisconnected();
-
-        if (log.isDebugEnabled()) {
-            if (e == null)
-                log.debug("SQL client disconnected: " + ses.remoteAddress());
-            else
-                log.debug("SQL client disconnected due to an error [addr=" + 
ses.remoteAddress() + ", err=" + e + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onMessage(GridNioSession ses, byte[] msg) {
-        assert msg != null;
-
-        SqlListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY);
-
-        if (connCtx == null) {
-            onHandshake(ses, msg);
-
-            return;
-        }
-
-        SqlListenerMessageParser parser = connCtx.parser();
-        SqlListenerRequestHandler handler = connCtx.handler();
-
-        SqlListenerRequest req;
-
-        try {
-            req = parser.decode(msg);
-        }
-        catch (Exception e) {
-            log.error("Failed to parse SQL client request.", e);
-
-            ses.close();
-
-            return;
-        }
-
-        assert req != null;
-
-        try {
-            long startTime = 0;
-
-            if (log.isDebugEnabled()) {
-                startTime = System.nanoTime();
-
-                log.debug("Client request received [reqId=" + req.requestId() 
+ ", addr=" +
-                    ses.remoteAddress() + ", req=" + req + ']');
-            }
-
-            SqlListenerResponse resp = handler.handle(req);
-
-            if (log.isDebugEnabled()) {
-                long dur = (System.nanoTime() - startTime) / 1000;
-
-                log.debug("Client request processed [reqId=" + req.requestId() 
+ ", dur(mcs)=" + dur  +
-                    ", resp=" + resp.status() + ']');
-            }
-
-            byte[] outMsg = parser.encode(resp);
-
-            ses.send(outMsg);
-        }
-        catch (Exception e) {
-            log.error("Failed to process client request [req=" + req + ']', e);
-
-            ses.send(parser.encode(handler.handleException(e)));
-        }
-    }
-
-    /**
-     * Perform handshake.
-     *
-     * @param ses Session.
-     * @param msg Message bytes.
-     */
-    private void onHandshake(GridNioSession ses, byte[] msg) {
-        BinaryInputStream stream = new BinaryHeapInputStream(msg);
-
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null, 
true);
-
-        byte cmd = reader.readByte();
-
-        if (cmd != SqlListenerRequest.HANDSHAKE) {
-            log.error("Unexpected SQL client request (will close session): " + 
ses.remoteAddress());
-
-            ses.close();
-
-            return;
-        }
-
-        short verMajor = reader.readShort();
-        short verMinor = reader.readShort();
-        short verMaintenance = reader.readShort();
-
-        SqlListenerProtocolVersion ver = 
SqlListenerProtocolVersion.create(verMajor, verMinor, verMaintenance);
-
-        byte clientType = reader.readByte();
-
-        SqlListenerConnectionContext connCtx = prepareContext(clientType);
-
-        String errMsg = null;
-
-        if (connCtx.isVersionSupported(ver)) {
-            connCtx.initializeFromHandshake(ver, reader);
-
-            ses.addMeta(CONN_CTX_META_KEY, connCtx);
-        }
-        else {
-            log.warning("Unsupported version: " + ver.toString());
-
-            errMsg = "Unsupported version.";
-        }
-
-        // Send response.
-        BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new 
BinaryHeapOutputStream(8), null, null);
-
-        if (errMsg == null)
-            connCtx.handler().writeHandshake(writer);
-        else {
-            SqlListenerProtocolVersion currentVer = connCtx.currentVersion();
-
-            // Failed handshake response
-            writer.writeBoolean(false);
-            writer.writeShort(currentVer.major());
-            writer.writeShort(currentVer.minor());
-            writer.writeShort(currentVer.maintenance());
-            writer.doWriteString(errMsg);
-        }
-
-        ses.send(writer.array());
-    }
-
-    /**
-     * Prepare context.
-     *
-     * @param clientType Client type.
-     * @return Context.
-     */
-    private SqlListenerConnectionContext prepareContext(byte clientType) {
-        switch (clientType) {
-            case ODBC_CLIENT:
-                return new OdbcConnectionContext(ctx, busyLock, maxCursors);
-
-            case JDBC_CLIENT:
-                return new JdbcConnectionContext(ctx, busyLock, maxCursors);
-
-            case THIN_CLIENT:
-                return new ClientConnectionContext(ctx);
-
-            default:
-                throw new IgniteException("Unknown client type: " + 
clientType);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioServerBuffer.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioServerBuffer.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioServerBuffer.java
deleted file mode 100644
index cf8a0ad..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioServerBuffer.java
+++ /dev/null
@@ -1,114 +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.IgniteCheckedException;
-import org.jetbrains.annotations.Nullable;
-
-import java.nio.ByteBuffer;
-
-/**
- * NIO server buffer.
- */
-public class SqlListenerNioServerBuffer {
-    /** Current message data. */
-    private byte[] data;
-
-    /** Count of received bytes of the current message. */
-    private int cnt = -4;
-
-    /** Current message size. */
-    private int msgSize;
-
-    /**
-     * Reset buffer state.
-     */
-    public void reset() {
-        msgSize = 0;
-        cnt = -4;
-        data = null;
-    }
-
-    /**
-     * Checks whether the byte array is filled.
-     *
-     * @return Flag indicating whether byte array is filled or not.
-     */
-    public boolean isFilled() {
-        return cnt > 0 && cnt == msgSize;
-    }
-
-    /**
-     * Get data withing the buffer.
-     *
-     * @return Data.
-     */
-    public byte[] data() {
-        return data;
-    }
-
-    /**
-     * @param buf Buffer.
-     * @return Message bytes or {@code null} if message is not fully read yet.
-     * @throws IgniteCheckedException If failed to parse message.
-     */
-    @Nullable public byte[] read(ByteBuffer buf) throws IgniteCheckedException 
{
-        if (cnt < 0) {
-            for (; cnt < 0 && buf.hasRemaining(); cnt++)
-                msgSize |= (buf.get() & 0xFF) << (8*(4 + cnt));
-
-            if (cnt < 0)
-                return null;
-
-            // If count is 0 then message size should be inited.
-            if (msgSize <= 0)
-                throw new IgniteCheckedException("Invalid message size: " + 
msgSize);
-
-            data = new byte[msgSize];
-        }
-
-        assert msgSize > 0;
-        assert cnt >= 0;
-
-        int remaining = buf.remaining();
-
-        // If there are more bytes in buffer.
-        if (remaining > 0) {
-            int missing = msgSize - cnt;
-
-            // Read only up to message size.
-            if (missing > 0) {
-                int len = missing < remaining ? missing : remaining;
-
-                buf.get(data, cnt, len);
-
-                cnt += len;
-            }
-        }
-
-        if (cnt == msgSize) {
-            byte[] data0 = data;
-
-            reset();
-
-            return data0;
-        }
-        else
-            return null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
deleted file mode 100644
index 1b364b1..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProcessor.java
+++ /dev/null
@@ -1,278 +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 java.net.InetAddress;
-import java.nio.ByteOrder;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.OdbcConfiguration;
-import org.apache.ignite.configuration.SqlConnectorConfiguration;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.HostAndPortRange;
-import org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter;
-import org.apache.ignite.internal.util.nio.GridNioCodecFilter;
-import org.apache.ignite.internal.util.nio.GridNioFilter;
-import org.apache.ignite.internal.util.nio.GridNioServer;
-import org.apache.ignite.internal.util.nio.GridNioSession;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.spi.IgnitePortProtocol;
-import org.apache.ignite.thread.IgniteThreadPoolExecutor;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * SQL processor.
- */
-public class SqlListenerProcessor extends GridProcessorAdapter {
-    /** Default number of selectors. */
-    private static final int DFLT_SELECTOR_CNT = Math.min(4, 
Runtime.getRuntime().availableProcessors());
-
-    /** Default TCP direct buffer flag. */
-    private static final boolean DFLT_TCP_DIRECT_BUF = false;
-
-    /** Busy lock. */
-    private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
-
-    /** TCP Server. */
-    private GridNioServer<byte[]> srv;
-
-    /** Executor service. */
-    private ExecutorService execSvc;
-
-    /**
-     * @param ctx Kernal context.
-     */
-    public SqlListenerProcessor(GridKernalContext ctx) {
-        super(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteCheckedException {
-        IgniteConfiguration cfg = ctx.config();
-
-        SqlConnectorConfiguration sqlCfg = prepareConfiguration(cfg);
-
-        if (sqlCfg != null) {
-            try {
-                validateConfiguration(sqlCfg);
-
-                // Resolve host.
-                String host = sqlCfg.getHost();
-
-                if (host == null)
-                    host = cfg.getLocalHost();
-
-                InetAddress hostAddr;
-
-                try {
-                    hostAddr = U.resolveLocalHost(host);
-                }
-                catch (Exception e) {
-                    throw new IgniteCheckedException("Failed to resolve SQL 
connector host: " + host, e);
-                }
-
-                execSvc = new IgniteThreadPoolExecutor(
-                    "sql-connector",
-                    cfg.getIgniteInstanceName(),
-                    sqlCfg.getThreadPoolSize(),
-                    sqlCfg.getThreadPoolSize(),
-                    0,
-                    new LinkedBlockingQueue<Runnable>());
-
-                Exception lastErr = null;
-
-                int portTo = sqlCfg.getPort() + sqlCfg.getPortRange();
-
-                if (portTo <= 0) // Handle int overflow.
-                    portTo = Integer.MAX_VALUE;
-
-                for (int port = sqlCfg.getPort(); port <= portTo && port <= 
65535; port++) {
-                    try {
-                        GridNioFilter[] filters = new GridNioFilter[] {
-                            new 
GridNioAsyncNotifyFilter(ctx.igniteInstanceName(), execSvc, log) {
-                                @Override public void 
onSessionOpened(GridNioSession ses)
-                                    throws IgniteCheckedException {
-                                    proceedSessionOpened(ses);
-                                }
-                            },
-                            new GridNioCodecFilter(new 
SqlListenerBufferedParser(), log, false)
-                        };
-
-                        int maxOpenCursors = 
sqlCfg.getMaxOpenCursorsPerConnection();
-
-                        GridNioServer<byte[]> srv0 = 
GridNioServer.<byte[]>builder()
-                            .address(hostAddr)
-                            .port(port)
-                            .listener(new SqlListenerNioListener(ctx, 
busyLock, maxOpenCursors))
-                            .logger(log)
-                            .selectorCount(DFLT_SELECTOR_CNT)
-                            .igniteInstanceName(ctx.igniteInstanceName())
-                            .serverName("sql-listener")
-                            .tcpNoDelay(sqlCfg.isTcpNoDelay())
-                            .directBuffer(DFLT_TCP_DIRECT_BUF)
-                            .byteOrder(ByteOrder.nativeOrder())
-                            
.socketSendBufferSize(sqlCfg.getSocketSendBufferSize())
-                            
.socketReceiveBufferSize(sqlCfg.getSocketReceiveBufferSize())
-                            .filters(filters)
-                            .directMode(false)
-                            .idleTimeout(Long.MAX_VALUE)
-                            .build();
-
-                        srv0.start();
-
-                        srv = srv0;
-
-                        ctx.ports().registerPort(port, IgnitePortProtocol.TCP, 
getClass());
-
-                        if (log.isInfoEnabled())
-                            log.info("SQL connector processor has started on 
TCP port " + port);
-
-                        lastErr = null;
-
-                        break;
-                    }
-                    catch (Exception e) {
-                        lastErr = e;
-                    }
-                }
-
-                assert (srv != null && lastErr == null) || (srv == null && 
lastErr != null);
-
-                if (lastErr != null)
-                    throw new IgniteCheckedException("Failed to bind to any 
[host:port] from the range [" +
-                        "host=" + host + ", portFrom=" + sqlCfg.getPort() + ", 
portTo=" + portTo +
-                        ", lastErr=" + lastErr + ']');
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to start SQL 
connector processor.", e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStop(boolean cancel) {
-        if (srv != null) {
-            busyLock.block();
-
-            srv.stop();
-
-            ctx.ports().deregisterPorts(getClass());
-
-            if (execSvc != null) {
-                U.shutdownNow(getClass(), execSvc, log);
-
-                execSvc = null;
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("SQL connector processor stopped.");
-        }
-    }
-
-    /**
-     * Prepare connector configuration.
-     *
-     * @param cfg Ignote configuration.
-     * @return Connector configuration.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("deprecation")
-    @Nullable private SqlConnectorConfiguration 
prepareConfiguration(IgniteConfiguration cfg)
-        throws IgniteCheckedException {
-        SqlConnectorConfiguration res = cfg.getSqlConnectorConfiguration();
-
-        OdbcConfiguration odbcCfg = cfg.getOdbcConfiguration();
-
-        if (odbcCfg != null) {
-            if (res == null) {
-                // SQL connector is either default or null, so we replace it 
with ODBC stuff.
-                HostAndPortRange hostAndPort = parseOdbcEndpoint(odbcCfg);
-
-                res = new SqlConnectorConfiguration();
-
-                res.setHost(hostAndPort.host());
-                res.setPort(hostAndPort.portFrom());
-                res.setPortRange(hostAndPort.portTo() - 
hostAndPort.portFrom());
-                res.setThreadPoolSize(odbcCfg.getThreadPoolSize());
-                res.setSocketSendBufferSize(odbcCfg.getSocketSendBufferSize());
-                
res.setSocketReceiveBufferSize(odbcCfg.getSocketReceiveBufferSize());
-                
res.setMaxOpenCursorsPerConnection(odbcCfg.getMaxOpenCursors());
-
-                U.warn(log, "Automatically converted deprecated ODBC 
configuration to SQL connector configuration: " +
-                    res);
-            }
-            else {
-                // Non-default SQL connector is set, ignore ODBC.
-                U.warn(log, "Deprecated ODBC configuration will be ignored 
because SQL connector configuration is " +
-                    "set (either migrate to new SqlConnectorConfiguration or 
set " +
-                    "IgniteConfiguration.sqlConnectorConfiguration to null 
explicitly).");
-            }
-        }
-
-        return res;
-    }
-
-    /**
-     * Validate SQL connector configuration.
-     *
-     * @param cfg Configuration.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void validateConfiguration(SqlConnectorConfiguration cfg) throws 
IgniteCheckedException {
-        assertParameter(cfg.getPort() > 1024, "port > 1024");
-        assertParameter(cfg.getPort() <= 65535, "port <= 65535");
-        assertParameter(cfg.getPortRange() >= 0, "portRange > 0");
-        assertParameter(cfg.getSocketSendBufferSize() >= 0, 
"socketSendBufferSize > 0");
-        assertParameter(cfg.getSocketReceiveBufferSize() >= 0, 
"socketReceiveBufferSize > 0");
-        assertParameter(cfg.getMaxOpenCursorsPerConnection() >= 0, 
"maxOpenCursorsPerConnection() >= 0");
-        assertParameter(cfg.getThreadPoolSize() > 0, "threadPoolSize > 0");
-    }
-
-    /**
-     * Parse ODBC endpoint.
-     *
-     * @param odbcCfg ODBC configuration.
-     * @return ODBC host and port range.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("deprecation")
-    private HostAndPortRange parseOdbcEndpoint(OdbcConfiguration odbcCfg) 
throws IgniteCheckedException {
-        HostAndPortRange res;
-
-        if (F.isEmpty(odbcCfg.getEndpointAddress())) {
-            res = new HostAndPortRange(OdbcConfiguration.DFLT_TCP_HOST,
-                OdbcConfiguration.DFLT_TCP_PORT_FROM,
-                OdbcConfiguration.DFLT_TCP_PORT_TO
-            );
-        }
-        else {
-            res = HostAndPortRange.parse(odbcCfg.getEndpointAddress(),
-                OdbcConfiguration.DFLT_TCP_PORT_FROM,
-                OdbcConfiguration.DFLT_TCP_PORT_TO,
-                "Failed to parse SQL connector endpoint address"
-            );
-        }
-
-        return res;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/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
deleted file mode 100644
index 690822b..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerProtocolVersion.java
+++ /dev/null
@@ -1,116 +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.F;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jetbrains.annotations.NotNull;
-
-/**
- * SQL listener protocol version.
- */
-public class SqlListenerProtocolVersion implements 
Comparable<SqlListenerProtocolVersion> {
-    /** Major part. */
-    private final short major;
-
-    /** Minor part. */
-    private final short minor;
-
-    /** Maintenance part. */
-    private final short maintenance;
-
-    /**
-     * Create version.
-     *
-     * @param major Major part.
-     * @param minor Minor part.
-     * @param maintenance Maintenance part.
-     * @return Version.
-     */
-    public static SqlListenerProtocolVersion create(int major, int minor, int 
maintenance) {
-        return new SqlListenerProtocolVersion((short)major, (short)minor, 
(short)maintenance);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param major Major part.
-     * @param minor Minor part.
-     * @param maintenance Maintenance part.
-     */
-    private SqlListenerProtocolVersion(short major, short minor, short 
maintenance) {
-        this.major = major;
-        this.minor = minor;
-        this.maintenance = maintenance;
-    }
-
-    /**
-     * @return Major part.
-     */
-    public short major() {
-        return major;
-    }
-
-    /**
-     * @return Minor part.
-     */
-    public short minor() {
-        return minor;
-    }
-
-    /**
-     * @return Maintenance part.
-     */
-    public short maintenance() {
-        return maintenance;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int compareTo(@NotNull SqlListenerProtocolVersion other) {
-        int res = major - other.major;
-
-        if (res == 0)
-            res = minor - other.minor;
-
-        if (res == 0)
-            res = maintenance - other.maintenance;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return 31 * (31 * major + minor) + maintenance;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (obj != null && obj instanceof SqlListenerProtocolVersion) {
-            SqlListenerProtocolVersion other = (SqlListenerProtocolVersion)obj;
-
-            return F.eq(major, other.major) && F.eq(minor, other.minor) && 
F.eq(maintenance, other.maintenance);
-        }
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(SqlListenerProtocolVersion.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/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
deleted file mode 100644
index 5b1c5fb..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequest.java
+++ /dev/null
@@ -1,31 +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;
-
-/**
- * SQL listener command request.
- */
-public interface SqlListenerRequest {
-    /** Handshake request. */
-    public static final int HANDSHAKE = 1;
-
-    /**
-     * @return Request ID.
-     */
-    public long requestId();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/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
deleted file mode 100644
index 348054f..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
+++ /dev/null
@@ -1,48 +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.binary.BinaryWriterExImpl;
-
-/**
- * SQL listener request handler.
- */
-public interface SqlListenerRequestHandler {
-    /**
-     * Handle request.
-     *
-     * @param req Request.
-     * @return Response.
-     */
-    public SqlListenerResponse handle(SqlListenerRequest req);
-
-    /**
-     * Handle exception.
-     *
-     * @param e Exception.
-     * @return Error response.
-     */
-    public SqlListenerResponse handleException(Exception e);
-
-    /**
-     * Write successful handshake response.
-     *
-     * @param writer Binary writer.
-     */
-    public void writeHandshake(BinaryWriterExImpl writer);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestNoId.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestNoId.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestNoId.java
deleted file mode 100644
index 68dbc5c..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestNoId.java
+++ /dev/null
@@ -1,28 +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;
-
-/**
- * Client request with no ID.
- */
-public abstract class SqlListenerRequestNoId implements SqlListenerRequest {
-    /** {@inheritDoc} */
-    @Override public long requestId() {
-        return 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/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
deleted file mode 100644
index e35d8f2..0000000
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerResponse.java
+++ /dev/null
@@ -1,76 +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.jetbrains.annotations.Nullable;
-
-/**
- * SQL listener response.
- */
-public abstract class SqlListenerResponse {
-    /** Command succeeded. */
-    public static final int STATUS_SUCCESS = 0;
-
-    /** Command failed. */
-    public static final int STATUS_FAILED = 1;
-
-    /** Success status. */
-    private int status;
-
-    /** Error. */
-    private String err;
-
-    /**
-     * 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) {
-        this.status = status;
-        this.err = err;
-    }
-
-    /**
-     * @return Success flag.
-     */
-    public int status() {
-        return status;
-    }
-
-    /**
-     * @param status Status.
-     */
-    public void status(int status) {
-        this.status = status;
-    }
-
-    /**
-     * @return Error.
-     */
-    public String error() {
-        return err;
-    }
-
-    /**
-     * @param err Error message.
-     */
-    public void error(String err) {
-        this.err = err;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index 1f13fcf..12be361 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -21,27 +21,27 @@ import java.util.HashSet;
 import java.util.Set;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerConnectionContext;
-import org.apache.ignite.internal.processors.odbc.SqlListenerMessageParser;
-import org.apache.ignite.internal.processors.odbc.SqlListenerProtocolVersion;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
+import 
org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
+import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
+import 
org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 
 /**
  * ODBC Connection Context.
  */
-public class JdbcConnectionContext implements SqlListenerConnectionContext {
+public class JdbcConnectionContext implements ClientListenerConnectionContext {
     /** Version 2.1.0. */
-    private static final SqlListenerProtocolVersion VER_2_1_0 = 
SqlListenerProtocolVersion.create(2, 1, 0);
+    private static final ClientListenerProtocolVersion VER_2_1_0 = 
ClientListenerProtocolVersion.create(2, 1, 0);
 
     /** Version 2.1.5: added "lazy" flag. */
-    private static final SqlListenerProtocolVersion VER_2_1_5 = 
SqlListenerProtocolVersion.create(2, 1, 5);
+    private static final ClientListenerProtocolVersion VER_2_1_5 = 
ClientListenerProtocolVersion.create(2, 1, 5);
 
     /** Current version. */
-    private static final SqlListenerProtocolVersion CURRENT_VER = VER_2_1_5;
+    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_1_5;
 
     /** Supported versions. */
-    private static final Set<SqlListenerProtocolVersion> SUPPORTED_VERS = new 
HashSet<>();
+    private static final Set<ClientListenerProtocolVersion> SUPPORTED_VERS = 
new HashSet<>();
 
     /** Context. */
     private final GridKernalContext ctx;
@@ -76,17 +76,17 @@ public class JdbcConnectionContext implements 
SqlListenerConnectionContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isVersionSupported(SqlListenerProtocolVersion 
ver) {
+    @Override public boolean isVersionSupported(ClientListenerProtocolVersion 
ver) {
         return SUPPORTED_VERS.contains(ver);
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerProtocolVersion currentVersion() {
+    @Override public ClientListenerProtocolVersion currentVersion() {
         return CURRENT_VER;
     }
 
     /** {@inheritDoc} */
-    @Override public void initializeFromHandshake(SqlListenerProtocolVersion 
ver, BinaryReaderExImpl reader) {
+    @Override public void 
initializeFromHandshake(ClientListenerProtocolVersion ver, BinaryReaderExImpl 
reader) {
         assert SUPPORTED_VERS.contains(ver): "Unsupported JDBC protocol 
version.";
 
         boolean distributedJoins = reader.readBoolean();
@@ -107,12 +107,12 @@ public class JdbcConnectionContext implements 
SqlListenerConnectionContext {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerRequestHandler handler() {
+    @Override public ClientListenerRequestHandler handler() {
         return handler;
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerMessageParser parser() {
+    @Override public ClientListenerMessageParser parser() {
         return parser;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
index 0f8a3ef..7a9c2ab 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMessageParser.java
@@ -23,14 +23,14 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
-import org.apache.ignite.internal.processors.odbc.SqlListenerMessageParser;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 
 /**
  * JDBC message parser.
  */
-public class JdbcMessageParser implements SqlListenerMessageParser {
+public class JdbcMessageParser implements ClientListenerMessageParser {
     /** Kernal context. */
     private final GridKernalContext ctx;
 
@@ -63,7 +63,7 @@ public class JdbcMessageParser implements 
SqlListenerMessageParser {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerRequest decode(byte[] msg) {
+    @Override public ClientListenerRequest decode(byte[] msg) {
         assert msg != null;
 
         BinaryReaderExImpl reader = createReader(msg);
@@ -72,7 +72,7 @@ public class JdbcMessageParser implements 
SqlListenerMessageParser {
     }
 
     /** {@inheritDoc} */
-    @Override public byte[] encode(SqlListenerResponse msg) {
+    @Override public byte[] encode(ClientListenerResponse msg) {
         assert msg != null;
 
         assert msg instanceof JdbcResponse;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
index ab97dc5..385924c 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
@@ -21,12 +21,12 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequestNoId;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestNoId;
 
 /**
  * JDBC request.
  */
-public class JdbcRequest extends SqlListenerRequestNoId implements 
JdbcRawBinarylizable {
+public class JdbcRequest extends ClientListenerRequestNoId implements 
JdbcRawBinarylizable {
     /** Execute sql query request. */
     static final byte QRY_EXEC = 2;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 378c6cb..4250ba5 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -36,9 +36,9 @@ import org.apache.ignite.internal.IgniteVersionUtils;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
-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.ClientListenerRequest;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import 
org.apache.ignite.internal.processors.odbc.odbc.OdbcQueryGetColumnsMetaRequest;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
@@ -63,7 +63,7 @@ import static 
org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_ME
 /**
  * JDBC request handler.
  */
-public class JdbcRequestHandler implements SqlListenerRequestHandler {
+public class JdbcRequestHandler implements ClientListenerRequestHandler {
     /** Query ID sequence. */
     private static final AtomicLong QRY_ID_GEN = new AtomicLong();
 
@@ -130,7 +130,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerResponse handle(SqlListenerRequest req0) {
+    @Override public ClientListenerResponse handle(ClientListenerRequest req0) 
{
         assert req0 != null;
 
         assert req0 instanceof JdbcRequest;
@@ -138,7 +138,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         JdbcRequest req = (JdbcRequest)req0;
 
         if (!busyLock.enterBusy())
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED,
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED,
                 "Failed to handle JDBC request because node is stopping.");
 
         try {
@@ -177,7 +177,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
                     return getSchemas((JdbcMetaSchemasRequest)req);
             }
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
"Unsupported JDBC request [req=" + req + ']');
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
"Unsupported JDBC request [req=" + req + ']');
         }
         finally {
             busyLock.leaveBusy();
@@ -185,8 +185,8 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerResponse handleException(Exception e) {
-        return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+    @Override public ClientListenerResponse handleException(Exception e) {
+        return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
     }
 
     /** {@inheritDoc} */
@@ -232,7 +232,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         int cursorCnt = qryCursors.size();
 
         if (maxCursors > 0 && cursorCnt >= maxCursors)
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, "Too 
many opened cursors (either close other " +
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, "Too 
many opened cursors (either close other " +
                 "opened cursors or increase the limit through 
OdbcConfiguration.setMaxOpenCursors()) " +
                 "[maximum=" + maxCursors + ", current=" + cursorCnt + ']');
 
@@ -269,7 +269,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
             qry.setLazy(lazy);
 
             if (req.pageSize() <= 0)
-                return new JdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new JdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Invalid fetch size : [fetchSize=" + req.pageSize() + ']');
 
             qry.setPageSize(req.pageSize());
@@ -312,7 +312,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
 
             U.error(log, "Failed to execute SQL query [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -327,7 +327,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
             JdbcQueryCursor cur = qryCursors.remove(req.queryId());
 
             if (cur == null)
-                return new JdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new JdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Failed to find query cursor with ID: " + req.queryId());
 
             cur.close();
@@ -339,7 +339,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
 
             U.error(log, "Failed to close SQL query [reqId=" + req.requestId() 
+ ", req=" + req.queryId() + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -354,11 +354,11 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
             JdbcQueryCursor cur = qryCursors.get(req.queryId());
 
             if (cur == null)
-                return new JdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new JdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Failed to find query cursor with ID: " + req.queryId());
 
             if (req.pageSize() <= 0)
-                return new JdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new JdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Invalid fetch size : [fetchSize=" + req.pageSize() + ']');
 
             cur.pageSize(req.pageSize());
@@ -376,7 +376,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to fetch SQL query result [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -389,7 +389,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
             JdbcQueryCursor cur = qryCursors.get(req.queryId());
 
             if (cur == null)
-                return new JdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new JdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Failed to find query with ID: " + req.queryId());
 
             JdbcQueryMetadataResult res = new 
JdbcQueryMetadataResult(req.queryId(),
@@ -400,7 +400,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to fetch SQL query result [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -408,7 +408,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Request.
      * @return Response.
      */
-    private SqlListenerResponse executeBatch(JdbcBatchExecuteRequest req) {
+    private ClientListenerResponse executeBatch(JdbcBatchExecuteRequest req) {
         String schemaName = req.schemaName();
 
         if (F.isEmpty(schemaName))
@@ -446,13 +446,13 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
                 updCnts[successQueries++] = 
((Long)items.get(0).get(0)).intValue();
             }
 
-            return new JdbcResponse(new JdbcBatchExecuteResult(updCnts, 
SqlListenerResponse.STATUS_SUCCESS, null));
+            return new JdbcResponse(new JdbcBatchExecuteResult(updCnts, 
ClientListenerResponse.STATUS_SUCCESS, null));
         }
         catch (Exception e) {
             U.error(log, "Failed to execute batch query [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
             return new JdbcResponse(new 
JdbcBatchExecuteResult(Arrays.copyOf(updCnts, successQueries),
-                SqlListenerResponse.STATUS_FAILED, e.toString()));
+                ClientListenerResponse.STATUS_FAILED, e.toString()));
         }
     }
 
@@ -486,7 +486,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get tables metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -528,7 +528,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get columns metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -536,7 +536,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Request.
      * @return Response.
      */
-    private SqlListenerResponse getIndexesMeta(JdbcMetaIndexesRequest req) {
+    private ClientListenerResponse getIndexesMeta(JdbcMetaIndexesRequest req) {
         try {
             Collection<JdbcIndexMeta> meta = new HashSet<>();
 
@@ -558,7 +558,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get parameters metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -566,7 +566,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Request.
      * @return Response.
      */
-    private SqlListenerResponse getParametersMeta(JdbcMetaParamsRequest req) {
+    private ClientListenerResponse getParametersMeta(JdbcMetaParamsRequest 
req) {
         try {
             ParameterMetaData paramMeta = 
ctx.query().prepareNativeStatement(req.schemaName(), req.sql())
                 .getParameterMetaData();
@@ -585,7 +585,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get parameters metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -593,7 +593,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Request.
      * @return Response.
      */
-    private SqlListenerResponse getPrimaryKeys(JdbcMetaPrimaryKeysRequest req) 
{
+    private ClientListenerResponse getPrimaryKeys(JdbcMetaPrimaryKeysRequest 
req) {
         try {
             Collection<JdbcPrimaryKeyMeta> meta = new HashSet<>();
 
@@ -631,7 +631,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get parameters metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 
@@ -639,7 +639,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Request.
      * @return Response.
      */
-    private SqlListenerResponse getSchemas(JdbcMetaSchemasRequest req) {
+    private ClientListenerResponse getSchemas(JdbcMetaSchemasRequest req) {
         try {
             String schemaPtrn = req.schemaName();
 
@@ -657,7 +657,7 @@ public class JdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get schemas metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+            return new JdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java
index f039db7..91f26d8 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.odbc.jdbc;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
@@ -28,7 +28,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * SQL listener response.
  */
-public class JdbcResponse extends SqlListenerResponse implements 
JdbcRawBinarylizable {
+public class JdbcResponse extends ClientListenerResponse implements 
JdbcRawBinarylizable {
     /** Response object. */
     @GridToStringInclude
     private JdbcResult res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
index 9b7bcbd..f2450e2 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java
@@ -21,27 +21,27 @@ import java.util.HashSet;
 import java.util.Set;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerConnectionContext;
-import org.apache.ignite.internal.processors.odbc.SqlListenerMessageParser;
-import org.apache.ignite.internal.processors.odbc.SqlListenerProtocolVersion;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
+import 
org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
+import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
+import 
org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 
 /**
  * ODBC Connection Context.
  */
-public class OdbcConnectionContext implements SqlListenerConnectionContext {
+public class OdbcConnectionContext implements ClientListenerConnectionContext {
     /** Version 2.1.0. */
-    private static final SqlListenerProtocolVersion VER_2_1_0 = 
SqlListenerProtocolVersion.create(2, 1, 0);
+    private static final ClientListenerProtocolVersion VER_2_1_0 = 
ClientListenerProtocolVersion.create(2, 1, 0);
 
     /** Version 2.1.5: added "lazy" flag. */
-    private static final SqlListenerProtocolVersion VER_2_1_5 = 
SqlListenerProtocolVersion.create(2, 1, 5);
+    private static final ClientListenerProtocolVersion VER_2_1_5 = 
ClientListenerProtocolVersion.create(2, 1, 5);
 
     /** Current version. */
-    private static final SqlListenerProtocolVersion CURRENT_VER = VER_2_1_5;
+    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_1_5;
 
     /** Supported versions. */
-    private static final Set<SqlListenerProtocolVersion> SUPPORTED_VERS = new 
HashSet<>();
+    private static final Set<ClientListenerProtocolVersion> SUPPORTED_VERS = 
new HashSet<>();
 
     /** Context. */
     private final GridKernalContext ctx;
@@ -76,17 +76,17 @@ public class OdbcConnectionContext implements 
SqlListenerConnectionContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isVersionSupported(SqlListenerProtocolVersion 
ver) {
+    @Override public boolean isVersionSupported(ClientListenerProtocolVersion 
ver) {
         return SUPPORTED_VERS.contains(ver);
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerProtocolVersion currentVersion() {
+    @Override public ClientListenerProtocolVersion currentVersion() {
         return CURRENT_VER;
     }
 
     /** {@inheritDoc} */
-    @Override public void initializeFromHandshake(SqlListenerProtocolVersion 
ver, BinaryReaderExImpl reader) {
+    @Override public void 
initializeFromHandshake(ClientListenerProtocolVersion ver, BinaryReaderExImpl 
reader) {
         assert SUPPORTED_VERS.contains(ver): "Unsupported ODBC protocol 
version.";
 
         boolean distributedJoins = reader.readBoolean();
@@ -105,12 +105,12 @@ public class OdbcConnectionContext implements 
SqlListenerConnectionContext {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerRequestHandler handler() {
+    @Override public ClientListenerRequestHandler handler() {
         return handler;
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerMessageParser parser() {
+    @Override public ClientListenerMessageParser parser() {
         return parser;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
index e1f3f82..2ab7cee 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcMessageParser.java
@@ -29,16 +29,16 @@ 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.SqlListenerMessageParser;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
-import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
 import org.jetbrains.annotations.NotNull;
 
 /**
  * JDBC message parser.
  */
-public class OdbcMessageParser implements SqlListenerMessageParser {
+public class OdbcMessageParser implements ClientListenerMessageParser {
     /** Marshaller. */
     private final GridBinaryMarshaller marsh;
 
@@ -71,7 +71,7 @@ public class OdbcMessageParser implements 
SqlListenerMessageParser {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerRequest decode(byte[] msg) {
+    @Override public ClientListenerRequest decode(byte[] msg) {
         assert msg != null;
 
         BinaryInputStream stream = new BinaryHeapInputStream(msg);
@@ -80,7 +80,7 @@ public class OdbcMessageParser implements 
SqlListenerMessageParser {
 
         byte cmd = reader.readByte();
 
-        SqlListenerRequest res;
+        ClientListenerRequest res;
 
         switch (cmd) {
             case OdbcRequest.QRY_EXEC: {
@@ -182,7 +182,7 @@ public class OdbcMessageParser implements 
SqlListenerMessageParser {
     }
 
     /** {@inheritDoc} */
-    @Override public byte[] encode(SqlListenerResponse msg0) {
+    @Override public byte[] encode(ClientListenerResponse msg0) {
         assert msg0 != null;
 
         assert msg0 instanceof OdbcResponse;
@@ -196,7 +196,7 @@ public class OdbcMessageParser implements 
SqlListenerMessageParser {
         // Writing status.
         writer.writeByte((byte) msg.status());
 
-        if (msg.status() != SqlListenerResponse.STATUS_SUCCESS) {
+        if (msg.status() != ClientListenerResponse.STATUS_SUCCESS) {
             writer.writeString(msg.error());
 
             return writer.array();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequest.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequest.java
index 58ffad6..4b21b79 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequest.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequest.java
@@ -17,12 +17,12 @@
 
 package org.apache.ignite.internal.processors.odbc.odbc;
 
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequestNoId;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestNoId;
 
 /**
  * SQL listener command request.
  */
-public class OdbcRequest extends SqlListenerRequestNoId {
+public class OdbcRequest extends ClientListenerRequestNoId {
     /** Execute sql query. */
     public static final int QRY_EXEC = 2;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index 31053c7..b66b1af 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -35,9 +35,9 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
-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.ClientListenerRequest;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 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.GridQueryIndexing;
@@ -58,7 +58,7 @@ import static 
org.apache.ignite.internal.processors.odbc.odbc.OdbcRequest.QRY_FE
 /**
  * SQL query handler.
  */
-public class OdbcRequestHandler implements SqlListenerRequestHandler {
+public class OdbcRequestHandler implements ClientListenerRequestHandler {
     /** Query ID sequence. */
     private static final AtomicLong QRY_ID_GEN = new AtomicLong();
 
@@ -119,7 +119,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerResponse handle(SqlListenerRequest req0) {
+    @Override public ClientListenerResponse handle(ClientListenerRequest req0) 
{
         assert req0 != null;
 
         OdbcRequest req = (OdbcRequest)req0;
@@ -160,8 +160,8 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerResponse handleException(Exception e) {
-        return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
e.toString());
+    @Override public ClientListenerResponse handleException(Exception e) {
+        return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
e.toString());
     }
 
     /** {@inheritDoc} */
@@ -215,11 +215,11 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Execute query request.
      * @return Response.
      */
-    private SqlListenerResponse executeQuery(OdbcQueryExecuteRequest req) {
+    private ClientListenerResponse executeQuery(OdbcQueryExecuteRequest req) {
         int cursorCnt = qryCursors.size();
 
         if (maxCursors > 0 && cursorCnt >= maxCursors)
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, "Too 
many opened cursors (either close other " +
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, "Too 
many opened cursors (either close other " +
                 "opened cursors or increase the limit through 
OdbcConfiguration.setMaxOpenCursors()) " +
                 "[maximum=" + maxCursors + ", current=" + cursorCnt + ']');
 
@@ -257,7 +257,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
 
             U.error(log, "Failed to execute SQL query [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
         }
     }
 
@@ -267,7 +267,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Execute query request.
      * @return Response.
      */
-    private SqlListenerResponse executeBatchQuery(OdbcQueryExecuteBatchRequest 
req) {
+    private ClientListenerResponse 
executeBatchQuery(OdbcQueryExecuteBatchRequest req) {
         long rowsAffected = 0;
         int currentSet = 0;
 
@@ -356,14 +356,14 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Execute query request.
      * @return Response.
      */
-    private SqlListenerResponse closeQuery(OdbcQueryCloseRequest req) {
+    private ClientListenerResponse closeQuery(OdbcQueryCloseRequest req) {
         long queryId = req.queryId();
 
         try {
             IgniteBiTuple<QueryCursor, Iterator> tuple = 
qryCursors.get(queryId);
 
             if (tuple == null)
-                return new OdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new OdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Failed to find query with ID: " + queryId);
 
             CloseCursor(tuple, queryId);
@@ -377,7 +377,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
 
             U.error(log, "Failed to close SQL query [reqId=" + req.requestId() 
+ ", req=" + queryId + ']', e);
 
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
         }
     }
 
@@ -387,13 +387,13 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Execute query request.
      * @return Response.
      */
-    private SqlListenerResponse fetchQuery(OdbcQueryFetchRequest req) {
+    private ClientListenerResponse fetchQuery(OdbcQueryFetchRequest req) {
         try {
             long queryId = req.queryId();
             IgniteBiTuple<QueryCursor, Iterator> tuple = 
qryCursors.get(queryId);
 
             if (tuple == null)
-                return new OdbcResponse(SqlListenerResponse.STATUS_FAILED,
+                return new OdbcResponse(ClientListenerResponse.STATUS_FAILED,
                     "Failed to find query with ID: " + queryId);
 
             Iterator iter = tuple.get2();
@@ -426,7 +426,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to fetch SQL query result [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
         }
     }
 
@@ -436,7 +436,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Get columns metadata request.
      * @return Response.
      */
-    private SqlListenerResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest 
req) {
+    private ClientListenerResponse 
getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) {
         try {
             List<OdbcColumnMeta> meta = new ArrayList<>();
 
@@ -491,7 +491,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get columns metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
         }
     }
 
@@ -501,7 +501,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Get tables metadata request.
      * @return Response.
      */
-    private SqlListenerResponse getTablesMeta(OdbcQueryGetTablesMetaRequest 
req) {
+    private ClientListenerResponse getTablesMeta(OdbcQueryGetTablesMetaRequest 
req) {
         try {
             List<OdbcTableMeta> meta = new ArrayList<>();
 
@@ -539,7 +539,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get tables metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
         }
     }
 
@@ -549,7 +549,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
      * @param req Get params metadata request.
      * @return Response.
      */
-    private SqlListenerResponse getParamsMeta(OdbcQueryGetParamsMetaRequest 
req) {
+    private ClientListenerResponse getParamsMeta(OdbcQueryGetParamsMetaRequest 
req) {
         try {
             PreparedStatement stmt = 
ctx.query().getIndexing().prepareNativeStatement(req.schema(), req.query());
 
@@ -570,7 +570,7 @@ public class OdbcRequestHandler implements 
SqlListenerRequestHandler {
         catch (Exception e) {
             U.error(log, "Failed to get params metadata [reqId=" + 
req.requestId() + ", req=" + req + ']', e);
 
-            return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
+            return new OdbcResponse(ClientListenerResponse.STATUS_FAILED, 
OdbcUtils.retrieveH2ErrorMessage(e));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcResponse.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcResponse.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcResponse.java
index e1b5b30..4e40da3 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcResponse.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcResponse.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.odbc.odbc;
 
-import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
@@ -25,7 +25,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * SQL listener response.
  */
-public class OdbcResponse extends SqlListenerResponse {
+public class OdbcResponse extends ClientListenerResponse {
     /** Response object. */
     @GridToStringInclude
     private final Object obj;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fd82212/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index d4ea0be..6a3e98e 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -19,17 +19,17 @@ package 
org.apache.ignite.internal.processors.platform.client;
 
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
-import org.apache.ignite.internal.processors.odbc.SqlListenerConnectionContext;
-import org.apache.ignite.internal.processors.odbc.SqlListenerMessageParser;
-import org.apache.ignite.internal.processors.odbc.SqlListenerProtocolVersion;
-import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
+import 
org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
+import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
+import 
org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
+import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 
 /**
  * Thin Client connection context.
  */
-public class ClientConnectionContext implements SqlListenerConnectionContext {
+public class ClientConnectionContext implements 
ClientListenerConnectionContext {
     /** Version 1.0.0. */
-    private static final SqlListenerProtocolVersion VER_1_0_0 = 
SqlListenerProtocolVersion.create(1, 0, 0);
+    private static final ClientListenerProtocolVersion VER_1_0_0 = 
ClientListenerProtocolVersion.create(1, 0, 0);
 
     /** Message parser. */
     private final ClientMessageParser parser;
@@ -50,27 +50,27 @@ public class ClientConnectionContext implements 
SqlListenerConnectionContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isVersionSupported(SqlListenerProtocolVersion 
ver) {
+    @Override public boolean isVersionSupported(ClientListenerProtocolVersion 
ver) {
         return VER_1_0_0.equals(ver);
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerProtocolVersion currentVersion() {
+    @Override public ClientListenerProtocolVersion currentVersion() {
         return VER_1_0_0;
     }
 
     /** {@inheritDoc} */
-    @Override public void initializeFromHandshake(SqlListenerProtocolVersion 
ver, BinaryReaderExImpl reader) {
+    @Override public void 
initializeFromHandshake(ClientListenerProtocolVersion ver, BinaryReaderExImpl 
reader) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerRequestHandler handler() {
+    @Override public ClientListenerRequestHandler handler() {
         return handler;
     }
 
     /** {@inheritDoc} */
-    @Override public SqlListenerMessageParser parser() {
+    @Override public ClientListenerMessageParser parser() {
         return parser;
     }
 

Reply via email to