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 2016/02/10 15:18:30 UTC

ignite git commit: IGNITE-2469: Refactored ODBC requests handling.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1786 9d296c989 -> f1ee5650e


IGNITE-2469: Refactored ODBC requests handling.


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

Branch: refs/heads/ignite-1786
Commit: f1ee5650e327ed82ee94b24b3a7b6be221bffb3f
Parents: 9d296c9
Author: isapego <is...@gridgain.com>
Authored: Wed Feb 10 17:18:27 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 17:18:27 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcCommandHandler.java     |  97 +++++++---
 .../internal/processors/odbc/OdbcParser.java    |  58 +++---
 .../internal/processors/odbc/OdbcProcessor.java | 175 ++++++++++++------
 .../processors/odbc/OdbcProtocolHandler.java    |  35 ----
 .../processors/odbc/OdbcTcpNioListener.java     |  96 ----------
 .../internal/processors/odbc/OdbcTcpServer.java | 177 -------------------
 6 files changed, 231 insertions(+), 407 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f1ee5650/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcCommandHandler.java
index 48851ca..6bdd50c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcCommandHandler.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -26,8 +25,13 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 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.nio.GridNioServerListenerAdapter;
+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.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
 
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -38,13 +42,16 @@ import static org.apache.ignite.internal.processors.odbc.OdbcRequest.*;
 /**
  * SQL query handler.
  */
-public class OdbcCommandHandler {
+public class OdbcCommandHandler extends GridNioServerListenerAdapter<OdbcRequest> {
     /** Query ID sequence. */
     private static final AtomicLong qryIdGen = new AtomicLong();
 
     /** Kernel context. */
     private final GridKernalContext ctx;
 
+    /** Busy lock. */
+    private final GridSpinBusyLock busyLock;
+
     /** Logger. */
     private final IgniteLogger log;
 
@@ -56,10 +63,45 @@ public class OdbcCommandHandler {
      *
      * @param ctx Context.
      */
-    public OdbcCommandHandler(GridKernalContext ctx) {
+    public OdbcCommandHandler(final GridKernalContext ctx, final GridSpinBusyLock busyLock) {
         this.ctx = ctx;
+        this.busyLock = busyLock;
+
+        this.log = ctx.log(getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onConnected(GridNioSession ses) {
+        if (log.isDebugEnabled())
+            log.debug("Driver connected");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
+        if (log.isDebugEnabled())
+            log.debug("Driver disconnected");
+
+        if (e != null) {
+            if (e instanceof RuntimeException)
+                U.error(log, "Failed to process request from remote client: " + ses, e);
+            else
+                U.warn(log, "Closed client session due to exception [ses=" + ses + ", msg=" + e.getMessage() + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMessage(GridNioSession ses, OdbcRequest msg) {
+        assert msg != null;
+
+        if (log.isDebugEnabled())
+            log.debug("Received request from client: [msg=" + msg + ']');
 
-        log = ctx.log(getClass());
+        OdbcResponse res = handle(msg);
+
+        if (log.isDebugEnabled())
+            log.debug("Handling result: [res=" + res.status() + ']');
+
+        ses.send(res);
     }
 
     /**
@@ -67,29 +109,43 @@ public class OdbcCommandHandler {
      *
      * @param req Request.
      * @return Response.
-     * @throws IgniteCheckedException If failed.
      */
-    public OdbcResponse handle(OdbcRequest req) throws IgniteCheckedException {
+    public OdbcResponse handle(OdbcRequest req) {
         assert req != null;
 
-        switch (req.command()) {
-            case EXECUTE_SQL_QUERY:
-                return executeQuery((OdbcQueryExecuteRequest)req);
+        if (!busyLock.enterBusy()) {
+            String errMsg = "Failed to handle request [req=" + req +
+                    ", err=Received request while stopping grid]";
+
+            U.error(log, errMsg);
 
-            case FETCH_SQL_QUERY:
-                return fetchQuery((OdbcQueryFetchRequest)req);
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED, errMsg);
+        }
 
-            case CLOSE_SQL_QUERY:
-                return closeQuery((OdbcQueryCloseRequest)req);
+        try {
+            switch (req.command()) {
+                case EXECUTE_SQL_QUERY:
+                    return executeQuery((OdbcQueryExecuteRequest)req);
 
-            case GET_COLUMNS_META:
-                return getColumnsMeta((OdbcQueryGetColumnsMetaRequest) req);
+                case FETCH_SQL_QUERY:
+                    return fetchQuery((OdbcQueryFetchRequest)req);
 
-            case GET_TABLES_META:
-                return getTablesMeta((OdbcQueryGetTablesMetaRequest) req);
-        }
+                case CLOSE_SQL_QUERY:
+                    return closeQuery((OdbcQueryCloseRequest)req);
 
-        return null;
+                case GET_COLUMNS_META:
+                    return getColumnsMeta((OdbcQueryGetColumnsMetaRequest) req);
+
+                case GET_TABLES_META:
+                    return getTablesMeta((OdbcQueryGetTablesMetaRequest) req);
+            }
+
+            return new OdbcResponse(OdbcResponse.STATUS_FAILED,
+                    "Failed to find registered handler for command: " + req.command());
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
     }
 
     /**
@@ -120,7 +176,8 @@ public class OdbcCommandHandler {
 
             List<?> fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta();
 
-            log.debug("Field meta: " + fieldsMeta);
+            if (log.isDebugEnabled())
+                log.debug("Field meta: " + fieldsMeta);
 
             OdbcQueryExecuteResult res = new OdbcQueryExecuteResult(qryId, convertMetadata(fieldsMeta));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1ee5650/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcParser.java
index 9d6ee4e..43e511f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcParser.java
@@ -134,7 +134,8 @@ public class OdbcParser implements GridNioParser {
         assert msg != null;
         assert msg instanceof OdbcResponse;
 
-        log.debug("Encoding query processing result");
+        if (log.isDebugEnabled())
+            log.debug("Encoding query processing result");
 
         BinaryRawWriterEx writer = marsh.writer(new BinaryHeapOutputStream(INIT_CAP));
 
@@ -178,10 +179,12 @@ public class OdbcParser implements GridNioParser {
                 String sql = reader.readString();
                 int argsNum = reader.readInt();
 
-                log.debug("Message EXECUTE_SQL_QUERY:");
-                log.debug("cache: " + cache);
-                log.debug("query: " + sql);
-                log.debug("argsNum: " + argsNum);
+                if (log.isDebugEnabled()) {
+                    log.debug("Message EXECUTE_SQL_QUERY:");
+                    log.debug("cache: " + cache);
+                    log.debug("query: " + sql);
+                    log.debug("argsNum: " + argsNum);
+                }
 
                 Object[] params = new Object[argsNum];
 
@@ -198,9 +201,11 @@ public class OdbcParser implements GridNioParser {
                 long queryId = reader.readLong();
                 int pageSize = reader.readInt();
 
-                log.debug("Message FETCH_SQL_QUERY:");
-                log.debug("queryId: " + queryId);
-                log.debug("pageSize: " + pageSize);
+                if (log.isDebugEnabled()) {
+                    log.debug("Message FETCH_SQL_QUERY:");
+                    log.debug("queryId: " + queryId);
+                    log.debug("pageSize: " + pageSize);
+                }
 
                 res = new OdbcQueryFetchRequest(queryId, pageSize);
 
@@ -211,8 +216,10 @@ public class OdbcParser implements GridNioParser {
 
                 long queryId = reader.readLong();
 
-                log.debug("Message CLOSE_SQL_QUERY:");
-                log.debug("queryId: " + queryId);
+                if (log.isDebugEnabled()) {
+                    log.debug("Message CLOSE_SQL_QUERY:");
+                    log.debug("queryId: " + queryId);
+                }
 
                 res = new OdbcQueryCloseRequest(queryId);
 
@@ -225,10 +232,12 @@ public class OdbcParser implements GridNioParser {
                 String table = reader.readString();
                 String column = reader.readString();
 
-                log.debug("Message GET_COLUMNS_META:");
-                log.debug("cache: " + cache);
-                log.debug("table: " + table);
-                log.debug("column: " + column);
+                if (log.isDebugEnabled()) {
+                    log.debug("Message GET_COLUMNS_META:");
+                    log.debug("cache: " + cache);
+                    log.debug("table: " + table);
+                    log.debug("column: " + column);
+                }
 
                 res = new OdbcQueryGetColumnsMetaRequest(cache, table, column);
 
@@ -242,11 +251,13 @@ public class OdbcParser implements GridNioParser {
                 String table = reader.readString();
                 String tableType = reader.readString();
 
-                log.debug("Message GET_COLUMNS_META:");
-                log.debug("catalog: " + catalog);
-                log.debug("schema: " + schema);
-                log.debug("table: " + table);
-                log.debug("tableType: " + tableType);
+                if (log.isDebugEnabled()) {
+                    log.debug("Message GET_COLUMNS_META:");
+                    log.debug("catalog: " + catalog);
+                    log.debug("schema: " + schema);
+                    log.debug("table: " + table);
+                    log.debug("tableType: " + tableType);
+                }
 
                 res = new OdbcQueryGetTablesMetaRequest(catalog, schema, table, tableType);
 
@@ -284,7 +295,8 @@ public class OdbcParser implements GridNioParser {
         if (res0 instanceof OdbcQueryExecuteResult) {
             OdbcQueryExecuteResult res = (OdbcQueryExecuteResult) res0;
 
-            log.debug("Resulting query ID: " + res.getQueryId());
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.getQueryId());
 
             writer.writeLong(res.getQueryId());
 
@@ -301,7 +313,8 @@ public class OdbcParser implements GridNioParser {
         else if (res0 instanceof OdbcQueryFetchResult) {
             OdbcQueryFetchResult res = (OdbcQueryFetchResult) res0;
 
-            log.debug("Resulting query ID: " + res.queryId());
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.queryId());
 
             writer.writeLong(res.queryId());
 
@@ -330,7 +343,8 @@ public class OdbcParser implements GridNioParser {
         else if (res0 instanceof OdbcQueryCloseResult) {
             OdbcQueryCloseResult res = (OdbcQueryCloseResult) res0;
 
-            log.debug("Resulting query ID: " + res.getQueryId());
+            if (log.isDebugEnabled())
+                log.debug("Resulting query ID: " + res.getQueryId());
 
             writer.writeLong(res.getQueryId());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1ee5650/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
index b1044cd..d19aef8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
@@ -18,76 +18,36 @@
 package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.OdbcConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
-import org.apache.ignite.internal.util.GridSpinReadWriteLock;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.IgnitePortProtocol;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteOrder;
 
 /**
  * ODBC processor.
  */
 public class OdbcProcessor extends GridProcessorAdapter {
     /** OBCD TCP Server. */
-    private OdbcTcpServer srv;
+    private GridNioServer<OdbcRequest> srv;
 
     /** Busy lock. */
-    private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock();
-
-    /** Command handler. */
-    private OdbcCommandHandler handler;
-
-    /** Protocol handler. */
-    private final OdbcProtocolHandler protoHnd = new OdbcProtocolHandler() {
-        /** {@inheritDoc} */
-        @Override public OdbcResponse handle(OdbcRequest req) throws IgniteCheckedException {
-            return handle0(req);
-        }
-    };
-
-    /**
-     * Handle request.
-     *
-     * @param req Request.
-     * @return Response.
-     */
-    private OdbcResponse handle0(final OdbcRequest req) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock())
-            throw new IgniteCheckedException("Failed to handle request (received request while stopping grid).");
-
-        try {
-            if (log.isDebugEnabled())
-                log.debug("Received request from client: " + req);
-
-            OdbcResponse rsp;
-
-            try {
-                rsp = handler == null ? null : handler.handle(req);
-
-                if (rsp == null)
-                    throw new IgniteCheckedException("Failed to find registered handler for command: " + req.command());
-            }
-            catch (Exception e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to handle request [req=" + req + ", e=" + e + "]");
-
-                rsp = new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage());
-            }
-
-            return rsp;
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
+    private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
     /**
      * @param ctx Kernal context.
      */
     public OdbcProcessor(GridKernalContext ctx) {
         super(ctx);
-
-        srv = new OdbcTcpServer(ctx);
     }
 
     /** {@inheritDoc} */
@@ -98,17 +58,44 @@ public class OdbcProcessor extends GridProcessorAdapter {
             if (marsh != null && !(marsh instanceof BinaryMarshaller))
                 throw new IgniteCheckedException("ODBC may only be used with BinaryMarshaller.");
 
-            // Register handler.
-            handler = new OdbcCommandHandler(ctx);
+            OdbcConfiguration cfg = ctx.config().getOdbcConfiguration();
+
+            assert cfg != null;
+
+            GridNioServerListener<OdbcRequest> listener = new OdbcCommandHandler(ctx, busyLock);
+
+            GridNioParser parser = new OdbcParser(ctx);
+
+            try {
+                InetAddress host = resolveOdbcTcpHost(ctx.config());
+
+                int port = cfg.getPort();
+
+                if (startTcpServer(host, port, listener, parser, cfg)) {
+                    if (log.isDebugEnabled())
+                        log.debug("ODBC Server has started on TCP port " + port);
+
+                    return;
+                }
 
-            srv.start(protoHnd);
+                U.warn(log, "Failed to start ODBC server (possibly all ports in range are in use) " +
+                        "[port=" + port + ", host=" + host + ']');
+            }
+            catch (IOException e) {
+                U.warn(log, "Failed to start ODBC server: " + e.getMessage(),
+                        "Failed to start ODBC server. Check odbcTcpHost configuration property.");
+            }
         }
     }
 
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         if (isOdbcEnabled()) {
-            srv.stop();
+            if (srv != null) {
+                ctx.ports().deregisterPorts(getClass());
+
+                srv.stop();
+            }
         }
     }
 
@@ -123,7 +110,7 @@ public class OdbcProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
         if (isOdbcEnabled()) {
-            busyLock.writeLock();
+            busyLock.block();
 
             if (log.isDebugEnabled())
                 log.debug("ODBC processor stopped.");
@@ -138,4 +125,78 @@ public class OdbcProcessor extends GridProcessorAdapter {
     public boolean isOdbcEnabled() {
         return ctx.config().getOdbcConfiguration() != null;
     }
+
+    /**
+     * Resolves host for server using grid configuration.
+     *
+     * @param cfg Grid configuration.
+     * @return Host address.
+     * @throws IOException If failed to resolve host.
+     */
+    private static InetAddress resolveOdbcTcpHost(IgniteConfiguration cfg) throws IOException {
+        String host = null;
+
+        OdbcConfiguration odbcCfg = cfg.getOdbcConfiguration();
+
+        if (odbcCfg != null)
+            host = odbcCfg.getHost();
+
+        if (host == null)
+            host = cfg.getLocalHost();
+
+        return U.resolveLocalHost(host);
+    }
+
+    /**
+     * Tries to start server with given parameters.
+     *
+     * @param hostAddr Host on which server should be bound.
+     * @param port Port on which server should be bound.
+     * @param listener Server message listener.
+     * @param parser Server message parser.
+     * @param cfg Configuration for other parameters.
+     * @return {@code True} if server successfully started, {@code false} if port is used and
+     *      server was unable to start.
+     */
+    private boolean startTcpServer(InetAddress hostAddr, int port, GridNioServerListener<OdbcRequest> listener,
+                                   GridNioParser parser, OdbcConfiguration cfg) {
+        try {
+            GridNioFilter codec = new GridNioCodecFilter(parser, log, false);
+
+            GridNioFilter[] filters;
+
+            filters = new GridNioFilter[] { codec };
+
+            srv = GridNioServer.<OdbcRequest>builder()
+                    .address(hostAddr)
+                    .port(port)
+                    .listener(listener)
+                    .logger(log)
+                    .selectorCount(cfg.getSelectorCount())
+                    .gridName(ctx.gridName())
+                    .tcpNoDelay(cfg.isNoDelay())
+                    .directBuffer(cfg.isDirectBuffer())
+                    .byteOrder(ByteOrder.nativeOrder())
+                    .socketSendBufferSize(cfg.getSendBufferSize())
+                    .socketReceiveBufferSize(cfg.getReceiveBufferSize())
+                    .sendQueueLimit(cfg.getSendQueueLimit())
+                    .filters(filters)
+                    .directMode(false)
+                    .build();
+
+            srv.idleTimeout(cfg.getIdleTimeout());
+
+            srv.start();
+
+            ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass());
+
+            return true;
+        }
+        catch (IgniteCheckedException e) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to start ODBC server on port " + port + ": " + e.getMessage());
+
+            return false;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1ee5650/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolHandler.java
deleted file mode 100644
index 962054a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolHandler.java
+++ /dev/null
@@ -1,35 +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.apache.ignite.internal.IgniteInternalFuture;
-
-/**
- * ODBC command protocol handler.
- */
-public interface OdbcProtocolHandler {
-    /**
-     * Handle request.
-     *
-     * @param req Request.
-     * @return Response.
-     * @throws IgniteCheckedException In case of error.
-     */
-    OdbcResponse handle(OdbcRequest req) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1ee5650/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpNioListener.java
deleted file mode 100644
index ce7f8a6..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpNioListener.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.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.util.nio.GridNioFuture;
-import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
-import org.apache.ignite.internal.util.nio.GridNioSession;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Listener for ODBC driver connection.
- */
-public class OdbcTcpNioListener extends GridNioServerListenerAdapter<OdbcRequest> {
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /** Protocol handler. */
-    private final OdbcProtocolHandler hnd;
-
-    /**
-     * @param log Logger.
-     * @param hnd Protocol handler.
-     */
-    OdbcTcpNioListener(IgniteLogger log, OdbcProtocolHandler hnd) {
-        this.log = log;
-        this.hnd = hnd;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onConnected(GridNioSession ses) {
-        log.debug("Driver connected");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
-        log.debug("Driver disconnected");
-
-        if (e != null) {
-            if (e instanceof RuntimeException)
-                U.error(log, "Failed to process request from remote client: " + ses, e);
-            else
-                U.warn(log, "Closed client session due to exception [ses=" + ses + ", msg=" + e.getMessage() + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onMessage(GridNioSession ses, OdbcRequest msg) {
-        assert msg != null;
-
-        log.debug("Query: " + msg.command());
-
-        OdbcResponse res;
-
-        try {
-            res = hnd.handle(msg);
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to process client request: " + msg, e);
-
-            res = new OdbcResponse(OdbcResponse.STATUS_FAILED,
-                    "Failed to process client request: " + e.getMessage());
-        }
-
-        log.debug("Resulting success status: " + res.status());
-
-        GridNioFuture<?> sf = ses.send(res);
-
-        // Check if send failed.
-        if (sf.isDone()) {
-            try {
-                sf.get();
-            }
-            catch (Exception e) {
-                U.error(log, "Failed to process client request [ses=" + ses + ", msg=" + msg + ']', e);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f1ee5650/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpServer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpServer.java
deleted file mode 100644
index 159b04a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTcpServer.java
+++ /dev/null
@@ -1,177 +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.apache.ignite.IgniteLogger;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.OdbcConfiguration;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.util.nio.*;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.spi.IgnitePortProtocol;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.nio.ByteOrder;
-
-/**
- * TCP server that handles communication with ODBC driver.
- */
-public class OdbcTcpServer {
-    /** Server. */
-    private GridNioServer<OdbcRequest> srv;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /** Context. */
-    private final GridKernalContext ctx;
-
-    /**
-     * @param ctx Kernel context.
-     */
-    public OdbcTcpServer(GridKernalContext ctx) {
-        assert ctx != null;
-        assert ctx.config().getConnectorConfiguration() != null;
-
-        this.ctx = ctx;
-
-        log = ctx.log(getClass());
-    }
-
-    /**
-     * Start ODBC TCP server.
-     *
-     * @param hnd ODBC protocol handler.
-     * @throws IgniteCheckedException
-     */
-    public void start(final OdbcProtocolHandler hnd) throws IgniteCheckedException {
-        OdbcConfiguration cfg = ctx.config().getOdbcConfiguration();
-
-        assert cfg != null;
-
-        GridNioServerListener<OdbcRequest> listener = new OdbcTcpNioListener(log, hnd);
-
-        GridNioParser parser = new OdbcParser(ctx);
-
-        try {
-            InetAddress host = resolveOdbcTcpHost(ctx.config());
-
-            int port = cfg.getPort();
-
-            if (startTcpServer(host, port, listener, parser, cfg)) {
-                log.debug("ODBC Server has started on TCP port " + port);
-
-                return;
-            }
-
-            U.warn(log, "Failed to start ODBC server (possibly all ports in range are in use) " +
-                    "[port=" + port + ", host=" + host + ']');
-        }
-        catch (IOException e) {
-            U.warn(log, "Failed to start ODBC server: " + e.getMessage(),
-                    "Failed to start ODBC server. Check odbcTcpHost configuration property.");
-        }
-    }
-
-    /**
-     * Stop ODBC TCP server.
-     */
-    public void stop() {
-        if (srv != null) {
-            ctx.ports().deregisterPorts(getClass());
-
-            srv.stop();
-        }
-    }
-
-    /**
-     * Resolves host for server using grid configuration.
-     *
-     * @param cfg Grid configuration.
-     * @return Host address.
-     * @throws IOException If failed to resolve host.
-     */
-    private static InetAddress resolveOdbcTcpHost(IgniteConfiguration cfg) throws IOException {
-        String host = null;
-
-        OdbcConfiguration odbcCfg = cfg.getOdbcConfiguration();
-
-        if (odbcCfg != null)
-            host = odbcCfg.getHost();
-
-        if (host == null)
-            host = cfg.getLocalHost();
-
-        return U.resolveLocalHost(host);
-    }
-
-    /**
-     * Tries to start server with given parameters.
-     *
-     * @param hostAddr Host on which server should be bound.
-     * @param port Port on which server should be bound.
-     * @param listener Server message listener.
-     * @param parser Server message parser.
-     * @param cfg Configuration for other parameters.
-     * @return {@code True} if server successfully started, {@code false} if port is used and
-     *      server was unable to start.
-     */
-    private boolean startTcpServer(InetAddress hostAddr, int port, GridNioServerListener<OdbcRequest> listener,
-        GridNioParser parser, OdbcConfiguration cfg) {
-        try {
-            GridNioFilter codec = new GridNioCodecFilter(parser, log, false);
-
-            GridNioFilter[] filters;
-
-            filters = new GridNioFilter[] { codec };
-
-            srv = GridNioServer.<OdbcRequest>builder()
-                    .address(hostAddr)
-                    .port(port)
-                    .listener(listener)
-                    .logger(log)
-                    .selectorCount(cfg.getSelectorCount())
-                    .gridName(ctx.gridName())
-                    .tcpNoDelay(cfg.isNoDelay())
-                    .directBuffer(cfg.isDirectBuffer())
-                    .byteOrder(ByteOrder.nativeOrder())
-                    .socketSendBufferSize(cfg.getSendBufferSize())
-                    .socketReceiveBufferSize(cfg.getReceiveBufferSize())
-                    .sendQueueLimit(cfg.getSendQueueLimit())
-                    .filters(filters)
-                    .directMode(false)
-                    .build();
-
-            srv.idleTimeout(cfg.getIdleTimeout());
-
-            srv.start();
-
-            ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass());
-
-            return true;
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to start ODBC server on port " + port + ": " + e.getMessage());
-
-            return false;
-        }
-    }
-}