Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id F3C0219E39 for ; Wed, 30 Mar 2016 12:29:02 +0000 (UTC) Received: (qmail 17735 invoked by uid 500); 30 Mar 2016 12:29:02 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 17698 invoked by uid 500); 30 Mar 2016 12:29:02 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 17636 invoked by uid 99); 30 Mar 2016 12:29:02 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Mar 2016 12:29:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7FCC6E07EF; Wed, 30 Mar 2016 12:29:02 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Wed, 30 Mar 2016 12:29:03 -0000 Message-Id: <6c46eba5691b4ec1bcd69e7cb5bf8d07@git.apache.org> In-Reply-To: <5c54ea18a6cb485fa625aa3339641fe9@git.apache.org> References: <5c54ea18a6cb485fa625aa3339641fe9@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/3] ignite git commit: IGNITE-1786: ODBC: Implemented ODBC support on Java side. IGNITE-1786: ODBC: Implemented ODBC support on Java side. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/bae86159 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/bae86159 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/bae86159 Branch: refs/heads/ignite-1786 Commit: bae86159c437b66e3cf79d295efa6e35f633d43a Parents: bbe5258 Author: isapego Authored: Wed Mar 30 15:28:24 2016 +0300 Committer: vozerov-gridgain Committed: Wed Mar 30 15:28:24 2016 +0300 ---------------------------------------------------------------------- .../configuration/IgniteConfiguration.java | 25 ++ .../ignite/configuration/OdbcConfiguration.java | 319 ++++++++++++++++ .../ignite/internal/GridKernalContext.java | 8 + .../ignite/internal/GridKernalContextImpl.java | 12 + .../apache/ignite/internal/IgniteKernal.java | 2 + .../processors/odbc/OdbcBufferedParser.java | 81 +++++ .../processors/odbc/OdbcColumnMeta.java | 110 ++++++ .../processors/odbc/OdbcHandshakeRequest.java | 49 +++ .../processors/odbc/OdbcHandshakeResult.java | 66 ++++ .../processors/odbc/OdbcMessageParser.java | 277 ++++++++++++++ .../processors/odbc/OdbcNioListener.java | 171 +++++++++ .../processors/odbc/OdbcNioServerBuffer.java | 114 ++++++ .../internal/processors/odbc/OdbcProcessor.java | 116 ++++++ .../processors/odbc/OdbcQueryCloseRequest.java | 49 +++ .../processors/odbc/OdbcQueryCloseResult.java | 40 ++ .../odbc/OdbcQueryExecuteRequest.java | 78 ++++ .../processors/odbc/OdbcQueryExecuteResult.java | 54 +++ .../processors/odbc/OdbcQueryFetchRequest.java | 61 ++++ .../processors/odbc/OdbcQueryFetchResult.java | 66 ++++ .../odbc/OdbcQueryGetColumnsMetaRequest.java | 74 ++++ .../odbc/OdbcQueryGetColumnsMetaResult.java | 42 +++ .../odbc/OdbcQueryGetTablesMetaRequest.java | 85 +++++ .../odbc/OdbcQueryGetTablesMetaResult.java | 42 +++ .../internal/processors/odbc/OdbcRequest.java | 58 +++ .../processors/odbc/OdbcRequestHandler.java | 362 +++++++++++++++++++ .../internal/processors/odbc/OdbcResponse.java | 96 +++++ .../internal/processors/odbc/OdbcTableMeta.java | 85 +++++ .../internal/processors/odbc/OdbcUtils.java | 56 +++ 28 files changed, 2598 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index 758a2b4..e06978f 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -411,6 +411,9 @@ public class IgniteConfiguration { /** Client access configuration. */ private ConnectorConfiguration connectorCfg = new ConnectorConfiguration(); + /** ODBC configuration. */ + private OdbcConfiguration odbcCfg; + /** Warmup closure. Will be invoked before actual grid start. */ private IgniteInClosure warmupClos; @@ -510,6 +513,7 @@ public class IgniteConfiguration { mgmtPoolSize = cfg.getManagementThreadPoolSize(); netTimeout = cfg.getNetworkTimeout(); nodeId = cfg.getNodeId(); + odbcCfg = cfg.getOdbcConfiguration(); p2pEnabled = cfg.isPeerClassLoadingEnabled(); p2pLocClsPathExcl = cfg.getPeerClassLoadingLocalClassPathExclude(); p2pMissedCacheSize = cfg.getPeerClassLoadingMissedResourcesCacheSize(); @@ -2298,6 +2302,27 @@ public class IgniteConfiguration { } /** + * Gets configuration for ODBC. + * + * @return ODBC configuration. + */ + public OdbcConfiguration getOdbcConfiguration() { + return odbcCfg; + } + + /** + * Sets configuration for ODBC. + * + * @param odbcCfg ODBC configuration. + * @return {@code this} for chaining. + */ + public IgniteConfiguration setOdbcConfiguration(OdbcConfiguration odbcCfg) { + this.odbcCfg = odbcCfg; + + return this; + } + + /** * Gets configurations for services to be deployed on the grid. * * @return Configurations for services to be deployed on the grid. http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java new file mode 100644 index 0000000..8f0a0fb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java @@ -0,0 +1,319 @@ +/* + * 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.configuration; + +import java.net.Socket; + +/** + * ODBC configuration. + */ +public class OdbcConfiguration { + /** Default TCP server port. */ + public static final int DFLT_TCP_PORT = 11443; + + /** Default TCP_NODELAY flag. */ + public static final boolean DFLT_TCP_NODELAY = true; + + /** Default TCP direct buffer flag. */ + public static final boolean DFLT_TCP_DIRECT_BUF = false; + + /** Default ODBC idle timeout. */ + public static final int DFLT_IDLE_TIMEOUT = 7000; + + /** Default socket send and receive buffer size. */ + public static final int DFLT_SOCK_BUF_SIZE = 32 * 1024; + + /** Default max number of open cursors per connection. */ + public static final int DFLT_MAX_OPEN_CURSORS = 128; + + /** TCP port. */ + private int port = DFLT_TCP_PORT; + + /** TCP host. */ + private String host; + + /** TCP no delay flag. */ + private boolean noDelay = DFLT_TCP_NODELAY; + + /** ODBC TCP direct buffer flag. */ + private boolean directBuf = DFLT_TCP_DIRECT_BUF; + + /** ODBC TCP send buffer size. */ + private int sndBufSize = DFLT_SOCK_BUF_SIZE; + + /** ODBC TCP receive buffer size. */ + private int rcvBufSize = DFLT_SOCK_BUF_SIZE; + + /** ODBC TCP send queue limit. */ + private int sndQueueLimit; + + /** ODBC TCP selector count. */ + private int selectorCnt = Math.min(4, Runtime.getRuntime().availableProcessors()); + + /** Idle timeout. */ + private long idleTimeout = DFLT_IDLE_TIMEOUT; + + /** Max number of opened cursors per connection. */ + private int maxOpenCursors = DFLT_MAX_OPEN_CURSORS; + + /** + * Creates ODBC server configuration with all default values. + */ + public OdbcConfiguration() { + // No-op. + } + + /** + * Creates ODBC server configuration by copying all properties from + * given configuration. + * + * @param cfg ODBC server configuration. + */ + public OdbcConfiguration(OdbcConfiguration cfg) { + assert cfg != null; + + directBuf = cfg.isDirectBuffer(); + host = cfg.getHost(); + idleTimeout = cfg.getIdleTimeout(); + maxOpenCursors = cfg.getMaxOpenCursors(); + noDelay = cfg.isNoDelay(); + port = cfg.getPort(); + rcvBufSize = cfg.getReceiveBufferSize(); + selectorCnt = cfg.getSelectorCount(); + sndBufSize = cfg.getSendBufferSize(); + sndQueueLimit = cfg.getSendQueueLimit(); + } + + /** + * Gets port for TCP ODBC server. + *

+ * Default is {@link #DFLT_TCP_PORT}. + * + * @return TCP port. + */ + public int getPort() { + return port; + } + + /** + * Sets port for TCP ODBC server. + * + * @param port TCP port. + */ + public void setPort(int port) { + this.port = port; + } + + /** + * Gets host for TCP ODBC server. This can be either an + * IP address or a domain name. + *

+ * If not defined, system-wide local address will be used + * (see {@link IgniteConfiguration#getLocalHost()}. + *

+ * You can also use {@code 0.0.0.0} value to bind to all + * locally-available IP addresses. + * + * @return TCP host. + */ + public String getHost() { + return host; + } + + /** + * Sets host for TCP ODBC server. + * + * @param host TCP host. + */ + public void setHost(String host) { + this.host = host; + } + + /** + * Gets flag indicating whether {@code TCP_NODELAY} option should be set for accepted client connections. + * Setting this option reduces network latency and should be set to {@code true} in majority of cases. + * For more information, see {@link Socket#setTcpNoDelay(boolean)} + *

+ * If not specified, default value is {@link #DFLT_TCP_NODELAY}. + * + * @return Whether {@code TCP_NODELAY} option should be enabled. + */ + public boolean isNoDelay() { + return noDelay; + } + + /** + * Sets whether {@code TCP_NODELAY} option should be set for all accepted ODBC client connections. + * + * @param noDelay {@code True} if option should be enabled. + * @see #isNoDelay() + */ + public void setNoDelay(boolean noDelay) { + this.noDelay = noDelay; + } + + /** + * Gets flag indicating whether ODBC TCP server should use direct buffers. A direct buffer is a buffer + * that is allocated and accessed using native system calls, without using JVM heap. Enabling direct + * buffer may improve performance and avoid memory issues (long GC pauses due to huge buffer + * size). + *

+ * If not specified, default value is {@link #DFLT_TCP_DIRECT_BUF}. + * + * @return Whether direct buffer should be used. + */ + public boolean isDirectBuffer() { + return directBuf; + } + + /** + * Sets whether to use direct buffer for ODBC TCP server. + * + * @param directBuf {@code True} if option should be enabled. + * @see #isDirectBuffer() + */ + public void setDirectBuffer(boolean directBuf) { + this.directBuf = directBuf; + } + + /** + * Gets ODBC TCP server send buffer size. + *

+ * If not specified, default value is {@link #DFLT_SOCK_BUF_SIZE}. + * + * @return ODBC TCP server send buffer size (0 for default). + */ + public int getSendBufferSize() { + return sndBufSize; + } + + /** + * Sets ODBC TCP server send buffer size. + * + * @param sndBufSize Send buffer size. + * @see #getSendBufferSize() + */ + public void setSendBufferSize(int sndBufSize) { + this.sndBufSize = sndBufSize; + } + + /** + * Gets ODBC TCP server receive buffer size. + *

+ * If not specified, default value is {@link #DFLT_SOCK_BUF_SIZE}. + * + * @return ODBC TCP server receive buffer size (0 for default). + */ + public int getReceiveBufferSize() { + return rcvBufSize; + } + + /** + * Sets ODBC TCP server receive buffer size. + * + * @param rcvBufSize Receive buffer size. + * @see #getReceiveBufferSize() + */ + public void setReceiveBufferSize(int rcvBufSize) { + this.rcvBufSize = rcvBufSize; + } + + /** + * Gets ODBC TCP server send queue limit. If the limit exceeds, all successive writes will + * block until the queue has enough capacity. + * + * @return ODBC TCP server send queue limit (0 for unlimited). + */ + public int getSendQueueLimit() { + return sndQueueLimit; + } + + /** + * Sets ODBC TCP server send queue limit. + * + * @param sndQueueLimit ODBC TCP server send queue limit (0 for unlimited). + * @see #getSendQueueLimit() + */ + public void setSendQueueLimit(int sndQueueLimit) { + this.sndQueueLimit = sndQueueLimit; + } + + /** + * Gets number of selector threads in ODBC TCP server. Higher value for this parameter + * may increase throughput, but also increases context switching. + * + * @return Number of selector threads for ODBC TCP server. + */ + public int getSelectorCount() { + return selectorCnt; + } + + /** + * Sets number of selector threads for ODBC TCP server. + * + * @param selectorCnt Number of selector threads for ODBC TCP server. + * @see #getSelectorCount() + */ + public void setSelectorCount(int selectorCnt) { + this.selectorCnt = selectorCnt; + } + + /** + * Gets idle timeout for ODBC TCP server. + *

+ * This setting is used to reject half-opened sockets. If no packets + * come within idle timeout, the connection is closed. + *

+ * If not specified, default value is {@link #DFLT_IDLE_TIMEOUT}. + * + * @return Idle timeout in milliseconds. + */ + public long getIdleTimeout() { + return idleTimeout; + } + + /** + * Sets idle timeout for ODBC TCP server. + * + * @param idleTimeout Idle timeout in milliseconds. + * @see #getIdleTimeout() + */ + public void setIdleTimeout(long idleTimeout) { + this.idleTimeout = idleTimeout; + } + + /** + * Gets maximum number of opened cursors per connection. + *

+ * Defaults to {@link #DFLT_MAX_OPEN_CURSORS}. + * + * @return Maximum number of opened cursors. + */ + public int getMaxOpenCursors() { + return maxOpenCursors; + } + + /** + * Sets maximum number of opened cursors per connection. See {@link #getMaxOpenCursors()}. + * + * @param maxOpenCursors Maximum number of opened cursors. + */ + public void setMaxOpenCursors(int maxOpenCursors) { + this.maxOpenCursors = maxOpenCursors; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index c0b50a2..b95d595 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsHelper; import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor; +import org.apache.ignite.internal.processors.odbc.OdbcProcessor; import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.processors.platform.PlatformProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; @@ -311,6 +312,13 @@ public interface GridKernalContext extends Iterable { public GridQueryProcessor query(); /** + * Gets ODBC processor. + * + * @return ODBC processor. + */ + public OdbcProcessor odbc(); + + /** * @return Plugin processor. */ public IgnitePluginProcessor plugins(); http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 22fd96c..753dbe8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -65,6 +65,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor; import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor; +import org.apache.ignite.internal.processors.odbc.OdbcProcessor; import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.processors.platform.PlatformProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; @@ -160,6 +161,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable /** */ @GridToStringInclude + private OdbcProcessor odbcProc; + + /** */ + @GridToStringInclude private GridQueryProcessor qryProc; /** */ @@ -506,6 +511,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable pluginProc = (IgnitePluginProcessor)comp; else if (comp instanceof GridQueryProcessor) qryProc = (GridQueryProcessor)comp; + else if (comp instanceof OdbcProcessor) + odbcProc = (OdbcProcessor)comp; else if (comp instanceof DataStructuresProcessor) dataStructuresProc = (DataStructuresProcessor)comp; else if (comp instanceof ClusterProcessor) @@ -749,6 +756,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable } /** {@inheritDoc} */ + @Override public OdbcProcessor odbc() { + return odbcProc; + } + + /** {@inheritDoc} */ @Override public DataStructuresProcessor dataStructures() { return dataStructuresProc; } http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 8df89f3..21fbc47 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -117,6 +117,7 @@ import org.apache.ignite.internal.processors.job.GridJobProcessor; import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor; import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor; import org.apache.ignite.internal.processors.nodevalidation.OsDiscoveryNodeValidationProcessor; +import org.apache.ignite.internal.processors.odbc.OdbcProcessor; import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.processors.platform.PlatformNoopProcessor; import org.apache.ignite.internal.processors.platform.PlatformProcessor; @@ -849,6 +850,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx)); startProcessor(new GridCacheProcessor(ctx)); startProcessor(new GridQueryProcessor(ctx)); + startProcessor(new OdbcProcessor(ctx)); startProcessor(new GridServiceProcessor(ctx)); startProcessor(new GridTaskSessionProcessor(ctx)); startProcessor(new GridJobProcessor(ctx)); http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcBufferedParser.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcBufferedParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcBufferedParser.java new file mode 100644 index 0000000..0d43aec --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcBufferedParser.java @@ -0,0 +1,81 @@ +/* + * 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.util.nio.GridNioParser; +import org.apache.ignite.internal.util.nio.GridNioSession; +import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * This class implements stream parser based on {@link OdbcNioServerBuffer}. + *

+ * The rule for this parser is that every message sent over the stream is prepended with + * 4-byte integer header containing message size. So, the stream structure is as follows: + *

+ *     +--+--+--+--+--+--+...+--+--+--+--+--+--+--+...+--+
+ *     | MSG_SIZE  |   MESSAGE  | MSG_SIZE  |   MESSAGE  |
+ *     +--+--+--+--+--+--+...+--+--+--+--+--+--+--+...+--+
+ * 
+ */ +public class OdbcBufferedParser implements GridNioParser { + /** Buffer metadata key. */ + private static final int BUF_META_KEY = GridNioSessionMetaKey.nextUniqueKey(); + + /** {@inheritDoc} */ + @Override public byte[] decode(GridNioSession ses, ByteBuffer buf) throws IOException, IgniteCheckedException { + OdbcNioServerBuffer nioBuf = ses.meta(BUF_META_KEY); + + // Decode for a given session is called per one thread, so there should not be any concurrency issues. + // However, we make some additional checks. + if (nioBuf == null) { + nioBuf = new OdbcNioServerBuffer(); + + OdbcNioServerBuffer old = ses.addMeta(BUF_META_KEY, nioBuf); + + assert old == null; + } + + return nioBuf.read(buf); + } + + /** {@inheritDoc} */ + @Override public ByteBuffer encode(GridNioSession ses, Object msg) throws IOException, IgniteCheckedException { + byte[] msg0 = (byte[])msg; + + ByteBuffer res = ByteBuffer.allocate(msg0.length + 4); + + res.order(ByteOrder.LITTLE_ENDIAN); + + res.putInt(msg0.length); + res.put(msg0); + + res.flip(); + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return OdbcBufferedParser.class.getSimpleName(); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcColumnMeta.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcColumnMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcColumnMeta.java new file mode 100644 index 0000000..0edfabc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcColumnMeta.java @@ -0,0 +1,110 @@ +/* + * 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.binary.BinaryRawWriter; +import org.apache.ignite.internal.binary.BinaryUtils; +import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; + +/** + * ODBC column-related metadata. + */ +public class OdbcColumnMeta { + /** Cache name. */ + private final String schemaName; + + /** Table name. */ + private final String tableName; + + /** Column name. */ + private final String columnName; + + /** Data type. */ + private final Class dataType; + + /** + * @param schemaName Cache name. + * @param tableName Table name. + * @param columnName Column name. + * @param dataType Data type. + */ + public OdbcColumnMeta(String schemaName, String tableName, String columnName, Class dataType) { + this.schemaName = OdbcUtils.addQuotationMarksIfNeeded(schemaName); + this.tableName = tableName; + this.columnName = columnName; + this.dataType = dataType; + } + + /** + * @param info Field metadata. + */ + public OdbcColumnMeta(GridQueryFieldMetadata info) { + this.schemaName = OdbcUtils.addQuotationMarksIfNeeded(info.schemaName()); + this.tableName = info.typeName(); + this.columnName = info.fieldName(); + + Class type; + + try { + type = Class.forName(info.fieldTypeName()); + } + catch (Exception ignored) { + type = Object.class; + } + + this.dataType = type; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int hash = schemaName.hashCode(); + + hash = 31 * hash + tableName.hashCode(); + hash = 31 * hash + columnName.hashCode(); + hash = 31 * hash + dataType.hashCode(); + + return hash; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (o instanceof OdbcColumnMeta) { + OdbcColumnMeta other = (OdbcColumnMeta) o; + + return this == other || schemaName.equals(other.schemaName) && tableName.equals(other.tableName) && + columnName.equals(other.columnName) && dataType.equals(other.dataType); + } + + return false; + } + + /** + * Write in a binary format. + * + * @param writer Binary writer. + */ + public void write(BinaryRawWriter writer) { + writer.writeString(schemaName); + writer.writeString(tableName); + writer.writeString(columnName); + + byte typeId = BinaryUtils.typeByClass(dataType); + + writer.writeByte(typeId); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java new file mode 100644 index 0000000..5e09041 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ODBC handshake request. + */ +public class OdbcHandshakeRequest extends OdbcRequest { + /** Protocol version. */ + private final long ver; + + /** + * @param ver Protocol version. + */ + public OdbcHandshakeRequest(long ver) { + super(HANDSHAKE); + + this.ver = ver; + } + + /** + * @return Protocol version. + */ + public long version() { + return ver; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcHandshakeRequest.class, this); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java new file mode 100644 index 0000000..bf1c61e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java @@ -0,0 +1,66 @@ +/* + * 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; + +/** + * ODBC handshake result. + */ +public class OdbcHandshakeResult { + /** Handshake accepted. */ + private final boolean accepted; + + /** Apache Ignite version when protocol version has been introduced. */ + private final String protoVerSince; + + /** Current Apache Ignite version. */ + private final String curVer; + + /** + * @param accepted Handshake accepted. + * @param protoVerSince Apache Ignite version when protocol version has been introduced. + * @param curVer Current Apache Ignite version. + */ + public OdbcHandshakeResult(boolean accepted, @Nullable String protoVerSince, @Nullable String curVer) { + this.accepted = accepted; + this.protoVerSince = protoVerSince; + this.curVer = curVer; + } + + /** + * @return Query ID. + */ + public boolean accepted() { + return accepted; + } + + /** + * @return Apache Ignite version when protocol version has been introduced. + */ + @Nullable public String protoVerSince() { + return protoVerSince; + } + + /** + * @return Current Apache Ignite version. + */ + @Nullable public String currentVer() { + return curVer; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java new file mode 100644 index 0000000..d217390 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.binary.GridBinaryMarshaller; +import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream; +import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; +import org.apache.ignite.internal.binary.streams.BinaryInputStream; +import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; + +import java.util.Collection; + +/** + * ODBC message parser. + */ +public class OdbcMessageParser { + /** Current ODBC communication protocol version. */ + public static final long PROTO_VER = 1; + + /** Apache Ignite version when ODBC communication protocol version has been introduced. */ + public static final String PROTO_VER_SINCE = "1.6.0"; + + /** Initial output stream capacity. */ + private static final int INIT_CAP = 1024; + + /** Marshaller. */ + private final GridBinaryMarshaller marsh; + + /** Logger. */ + private final IgniteLogger log; + + /** Protocol version confirmation flag. */ + private boolean verConfirmed = false; + + /** + * @param ctx Context. + */ + public OdbcMessageParser(final GridKernalContext ctx) { + CacheObjectBinaryProcessorImpl cacheObjProc = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects(); + + this.marsh = cacheObjProc.marshaller(); + + this.log = ctx.log(getClass()); + } + + /** + * Decode OdbcRequest from byte array. + * + * @param msg Message. + * @return Assembled ODBC request. + */ + public OdbcRequest decode(byte[] msg) { + assert msg != null; + + BinaryInputStream stream = new BinaryHeapInputStream(msg); + + BinaryReaderExImpl reader = new BinaryReaderExImpl(null, stream, null); + + byte cmd = reader.readByte(); + + // This is a special case because we can not decode protocol messages until + // we has not confirmed that the remote client uses the same protocol version. + if (!verConfirmed) { + if (cmd == OdbcRequest.HANDSHAKE) + return new OdbcHandshakeRequest(reader.readLong()); + else + throw new IgniteException("Unexpected ODBC command (first message is not a handshake request): [cmd=" + + cmd + ']'); + } + + OdbcRequest res; + + switch (cmd) { + case OdbcRequest.EXECUTE_SQL_QUERY: { + String cache = reader.readString(); + String sql = reader.readString(); + int argsNum = reader.readInt(); + + Object[] params = new Object[argsNum]; + + for (int i = 0; i < argsNum; ++i) + params[i] = reader.readObjectDetached(); + + res = new OdbcQueryExecuteRequest(cache, sql, params); + + break; + } + + case OdbcRequest.FETCH_SQL_QUERY: { + long queryId = reader.readLong(); + int pageSize = reader.readInt(); + + res = new OdbcQueryFetchRequest(queryId, pageSize); + + break; + } + + case OdbcRequest.CLOSE_SQL_QUERY: { + long queryId = reader.readLong(); + + res = new OdbcQueryCloseRequest(queryId); + + break; + } + + case OdbcRequest.GET_COLUMNS_META: { + String cache = reader.readString(); + String table = reader.readString(); + String column = reader.readString(); + + res = new OdbcQueryGetColumnsMetaRequest(cache, table, column); + + break; + } + + case OdbcRequest.GET_TABLES_META: { + String catalog = reader.readString(); + String schema = reader.readString(); + String table = reader.readString(); + String tableType = reader.readString(); + + res = new OdbcQueryGetTablesMetaRequest(catalog, schema, table, tableType); + + break; + } + + default: + throw new IgniteException("Unknown ODBC command: [cmd=" + cmd + ']'); + } + + return res; + } + + /** + * Encode OdbcResponse to byte array. + * + * @param msg Message. + * @return Byte array. + */ + public byte[] encode(OdbcResponse msg) { + assert msg != null; + + // Creating new binary writer + BinaryWriterExImpl writer = marsh.writer(new BinaryHeapOutputStream(INIT_CAP)); + + // Writing status. + writer.writeByte((byte) msg.status()); + + if (msg.status() != OdbcResponse.STATUS_SUCCESS) { + writer.writeString(msg.error()); + + return writer.array(); + } + + Object res0 = msg.response(); + + if (res0 instanceof OdbcHandshakeResult) { + OdbcHandshakeResult res = (OdbcHandshakeResult) res0; + + if (log.isDebugEnabled()) + log.debug("Handshake result: " + (res.accepted() ? "accepted" : "rejected")); + + verConfirmed = res.accepted(); + + if (res.accepted()) { + verConfirmed = true; + + writer.writeBoolean(true); + } + else { + writer.writeBoolean(false); + writer.writeString(res.protoVerSince()); + writer.writeString(res.currentVer()); + } + } + else if (res0 instanceof OdbcQueryExecuteResult) { + OdbcQueryExecuteResult res = (OdbcQueryExecuteResult) res0; + + if (log.isDebugEnabled()) + log.debug("Resulting query ID: " + res.getQueryId()); + + writer.writeLong(res.getQueryId()); + + Collection metas = res.getColumnsMetadata(); + + assert metas != null; + + writer.writeInt(metas.size()); + + for (OdbcColumnMeta meta : metas) + meta.write(writer); + } + else if (res0 instanceof OdbcQueryFetchResult) { + OdbcQueryFetchResult res = (OdbcQueryFetchResult) res0; + + if (log.isDebugEnabled()) + log.debug("Resulting query ID: " + res.queryId()); + + writer.writeLong(res.queryId()); + + Collection items0 = res.items(); + + assert items0 != null; + + writer.writeBoolean(res.last()); + + writer.writeInt(items0.size()); + + for (Object row0 : items0) { + if (row0 != null) { + Collection row = (Collection)row0; + + writer.writeInt(row.size()); + + for (Object obj : row) + writer.writeObjectDetached(obj); + } + } + } + else if (res0 instanceof OdbcQueryCloseResult) { + OdbcQueryCloseResult res = (OdbcQueryCloseResult) res0; + + if (log.isDebugEnabled()) + log.debug("Resulting query ID: " + res.getQueryId()); + + writer.writeLong(res.getQueryId()); + } + else if (res0 instanceof OdbcQueryGetColumnsMetaResult) { + OdbcQueryGetColumnsMetaResult res = (OdbcQueryGetColumnsMetaResult) res0; + + Collection columnsMeta = res.meta(); + + assert columnsMeta != null; + + writer.writeInt(columnsMeta.size()); + + for (OdbcColumnMeta columnMeta : columnsMeta) + columnMeta.write(writer); + } + else if (res0 instanceof OdbcQueryGetTablesMetaResult) { + OdbcQueryGetTablesMetaResult res = (OdbcQueryGetTablesMetaResult) res0; + + Collection tablesMeta = res.meta(); + + assert tablesMeta != null; + + writer.writeInt(tablesMeta.size()); + + for (OdbcTableMeta tableMeta : tablesMeta) + tableMeta.writeBinary(writer); + } + else + assert false : "Should not reach here."; + + return writer.array(); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java new file mode 100644 index 0000000..28b2b5a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java @@ -0,0 +1,171 @@ +/* + * 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.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +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; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * ODBC message listener. + */ +public class OdbcNioListener extends GridNioServerListenerAdapter { + /** Connection-related metadata key. */ + private static final int CONNECTION_DATA_META_KEY = GridNioSessionMetaKey.nextUniqueKey(); + + /** Request ID generator. */ + private static final AtomicLong REQ_ID_GEN = new AtomicLong(); + + /** Busy lock. */ + private final GridSpinBusyLock busyLock; + + /** Kernal context. */ + private final GridKernalContext ctx; + + /** Logger. */ + private final IgniteLogger log; + + /** + * @param ctx Context. + * @param busyLock Shutdown busy lock. + */ + public OdbcNioListener(GridKernalContext ctx, 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("ODBC client connected: " + ses.remoteAddress()); + + ses.addMeta(CONNECTION_DATA_META_KEY, new ConnectionData(ctx, busyLock)); + } + + /** {@inheritDoc} */ + @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) { + if (log.isDebugEnabled()) { + if (e == null) + log.debug("ODBC client disconnected: " + ses.remoteAddress()); + else + log.debug("ODBC client disconnected due to an error [addr=" + ses.remoteAddress() + ", err=" + e + ']'); + } + } + + /** {@inheritDoc} */ + @Override public void onMessage(GridNioSession ses, byte[] msg) { + assert msg != null; + + long reqId = REQ_ID_GEN.incrementAndGet(); + + ConnectionData connData = ses.meta(CONNECTION_DATA_META_KEY); + + assert connData != null; + + OdbcMessageParser parser = connData.getParser(); + + OdbcRequest req; + + try { + req = parser.decode(msg); + } + catch (Exception e) { + log.error("Failed to parse message [id=" + reqId + ", err=" + e + ']'); + + ses.close(); + + return; + } + + assert req != null; + + try { + long startTime = 0; + + if (log.isDebugEnabled()) { + startTime = System.nanoTime(); + + log.debug("ODBC request received [id=" + reqId + ", addr=" + ses.remoteAddress() + + ", req=" + req + ']'); + } + + OdbcRequestHandler handler = connData.getHandler(); + + OdbcResponse resp = handler.handle(req); + + if (log.isDebugEnabled()) { + long dur = (System.nanoTime() - startTime) / 1000; + + log.debug("ODBC request processed [id=" + reqId + ", dur(mcs)=" + dur + + ", resp=" + resp.status() + ']'); + } + + byte[] outMsg = parser.encode(resp); + + ses.send(outMsg); + } + catch (Exception e) { + log.error("Failed to process ODBC request [id=" + reqId + ", err=" + e + ']'); + + ses.send(parser.encode(new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage()))); + } + } + + /** + * Connection-related data. + */ + private static class ConnectionData { + /** Request handler. */ + private final OdbcRequestHandler handler; + + /** Message parser. */ + private final OdbcMessageParser parser; + + /** + * @param ctx Context. + * @param busyLock Shutdown busy lock. + */ + public ConnectionData(GridKernalContext ctx, GridSpinBusyLock busyLock) { + handler = new OdbcRequestHandler(ctx, busyLock); + parser = new OdbcMessageParser(ctx); + } + + /** + * Handler getter. + * @return Request handler for the connection. + */ + public OdbcRequestHandler getHandler() { + return handler; + } + + /** + * Parser getter + * @return Message parser for the connection. + */ + public OdbcMessageParser getParser() { + return parser; + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioServerBuffer.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioServerBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioServerBuffer.java new file mode 100644 index 0000000..d1e6dd9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioServerBuffer.java @@ -0,0 +1,114 @@ +/* + * 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 OdbcNioServerBuffer { + /** 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/bae86159/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 new file mode 100644 index 0000000..87be686 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java @@ -0,0 +1,116 @@ +/* + * 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.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.GridSpinBusyLock; +import org.apache.ignite.internal.util.nio.GridNioCodecFilter; +import org.apache.ignite.internal.util.nio.GridNioServer; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.spi.IgnitePortProtocol; + +import java.net.InetAddress; +import java.nio.ByteOrder; + +/** + * ODBC processor. + */ +public class OdbcProcessor extends GridProcessorAdapter { + /** Busy lock. */ + private final GridSpinBusyLock busyLock = new GridSpinBusyLock(); + + /** OBCD TCP Server. */ + private GridNioServer srv; + + /** + * @param ctx Kernal context. + */ + public OdbcProcessor(GridKernalContext ctx) { + super(ctx); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + OdbcConfiguration odbcCfg = ctx.config().getOdbcConfiguration(); + + if (odbcCfg != null) { + try { + Marshaller marsh = ctx.config().getMarshaller(); + + if (marsh != null && !(marsh instanceof BinaryMarshaller)) + throw new IgniteCheckedException("ODBC can only be used with BinaryMarshaller (please set it " + + "through IgniteConfiguration.setMarshaller())"); + + String hostStr = odbcCfg.getHost(); + + if (hostStr == null) + hostStr = ctx.config().getLocalHost(); + + InetAddress host = U.resolveLocalHost(hostStr); + + int port = odbcCfg.getPort(); + + srv = GridNioServer.builder() + .address(host) + .port(port) + .listener(new OdbcNioListener(ctx, busyLock)) + .logger(log) + .selectorCount(odbcCfg.getSelectorCount()) + .gridName(ctx.gridName()) + .tcpNoDelay(odbcCfg.isNoDelay()) + .directBuffer(odbcCfg.isDirectBuffer()) + .byteOrder(ByteOrder.nativeOrder()) + .socketSendBufferSize(odbcCfg.getSendBufferSize()) + .socketReceiveBufferSize(odbcCfg.getReceiveBufferSize()) + .sendQueueLimit(odbcCfg.getSendQueueLimit()) + .filters(new GridNioCodecFilter(new OdbcBufferedParser(), log, false)) + .directMode(false) + .idleTimeout(odbcCfg.getIdleTimeout()) + .build(); + + srv.start(); + + ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass()); + + log.info("ODBC processor has started on TCP port " + port); + } + catch (Exception e) { + throw new IgniteCheckedException("Failed to start ODBC processor.", e); + } + } + } + + /** {@inheritDoc} */ + @Override public void onKernalStop(boolean cancel) { + if (srv != null) { + busyLock.block(); + + srv.stop(); + + ctx.ports().deregisterPorts(getClass()); + + if (log.isDebugEnabled()) + log.debug("ODBC processor stopped."); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseRequest.java new file mode 100644 index 0000000..8c081ef --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseRequest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ODBC query close request. + */ +public class OdbcQueryCloseRequest extends OdbcRequest { + /** Query ID. */ + private final long queryId; + + /** + * @param queryId Query ID. + */ + public OdbcQueryCloseRequest(long queryId) { + super(CLOSE_SQL_QUERY); + + this.queryId = queryId; + } + + /** + * @return Query ID. + */ + public long queryId() { + return queryId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryCloseRequest.class, this); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseResult.java new file mode 100644 index 0000000..9a85e43 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryCloseResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +/** + * ODBC query result. + */ +public class OdbcQueryCloseResult { + /** Query ID. */ + private final long queryId; + + /** + * @param queryId Query ID. + */ + public OdbcQueryCloseResult(long queryId){ + this.queryId = queryId; + } + + /** + * @return Query ID. + */ + public long getQueryId() { + return queryId; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java new file mode 100644 index 0000000..1bcd41f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; + +/** + * ODBC query execute request. + */ +public class OdbcQueryExecuteRequest extends OdbcRequest { + /** Cache name. */ + private final String cacheName; + + /** Sql query. */ + private final String sqlQry; + + /** Sql query arguments. */ + @GridToStringExclude + private final Object[] args; + + /** + * @param cacheName Cache name. + * @param sqlQry SQL query. + * @param args Arguments list. + */ + public OdbcQueryExecuteRequest(String cacheName, String sqlQry, Object[] args) { + super(EXECUTE_SQL_QUERY); + + this.cacheName = cacheName.isEmpty() ? null : cacheName; + this.sqlQry = sqlQry; + this.args = args; + } + + /** + * @return Sql query. + */ + public String sqlQuery() { + return sqlQry; + } + + /** + * @return Sql query arguments. + */ + public Object[] arguments() { + return args; + } + + /** + * @return Cache name. + */ + @Nullable public String cacheName() { + return cacheName; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryExecuteRequest.class, this, "args", Arrays.toString(args)); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java new file mode 100644 index 0000000..a403812 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteResult.java @@ -0,0 +1,54 @@ +/* + * 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.Collection; + +/** + * Query execute result. + */ +public class OdbcQueryExecuteResult { + /** Query ID. */ + private final long queryId; + + /** Fields metadata. */ + private final Collection columnsMeta; + + /** + * @param queryId Query ID. + * @param columnsMeta Columns metadata. + */ + public OdbcQueryExecuteResult(long queryId, Collection columnsMeta) { + this.queryId = queryId; + this.columnsMeta = columnsMeta; + } + + /** + * @return Query ID. + */ + public long getQueryId() { + return queryId; + } + + /** + * @return Columns metadata. + */ + public Collection getColumnsMetadata() { + return columnsMeta; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchRequest.java new file mode 100644 index 0000000..58bc43a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchRequest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ODBC query fetch request. + */ +public class OdbcQueryFetchRequest extends OdbcRequest { + /** Query ID. */ + private final long queryId; + + /** Page size - maximum number of rows to return. */ + private final int pageSize; + + /** + * @param queryId Query ID. + * @param pageSize Page size. + */ + public OdbcQueryFetchRequest(long queryId, int pageSize) { + super(FETCH_SQL_QUERY); + + this.queryId = queryId; + this.pageSize = pageSize; + } + + /** + * @return Page size. + */ + public int pageSize() { + return pageSize; + } + + /** + * @return Query ID. + */ + public long queryId() { + return queryId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryFetchRequest.class, this); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchResult.java new file mode 100644 index 0000000..f72d7e1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryFetchResult.java @@ -0,0 +1,66 @@ +/* + * 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.Collection; + +/** + * Query fetch result. + */ +public class OdbcQueryFetchResult { + /** Query ID. */ + private final long queryId; + + /** Query result rows. */ + private final Collection items; + + /** Flag indicating the query has no unfetched results. */ + private final boolean last; + + /** + * @param queryId Query ID. + * @param items Query result rows. + * @param last Flag indicating the query has no unfetched results. + */ + public OdbcQueryFetchResult(long queryId, Collection items, boolean last){ + this.queryId = queryId; + this.items = items; + this.last = last; + } + + /** + * @return Query ID. + */ + public long queryId() { + return queryId; + } + + /** + * @return Query result rows. + */ + public Collection items() { + return items; + } + + /** + * @return Flag indicating the query has no unfetched results. + */ + public boolean last() { + return last; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaRequest.java new file mode 100644 index 0000000..875be89 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaRequest.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +/** + * ODBC query get columns meta request. + */ +public class OdbcQueryGetColumnsMetaRequest extends OdbcRequest { + /** Cache name. */ + private final String cacheName; + + /** Table name. */ + private final String tableName; + + /** Column name. */ + private final String columnName; + + /** + * @param cacheName Cache name. + * @param tableName Table name. + * @param columnName Column name. + */ + public OdbcQueryGetColumnsMetaRequest(String cacheName, String tableName, String columnName) { + super(GET_COLUMNS_META); + + this.cacheName = cacheName; + this.tableName = tableName; + this.columnName = columnName; + } + + /** + * @return Cache name. + */ + @Nullable public String cacheName() { + return cacheName; + } + + /** + * @return Table name. + */ + public String tableName() { + return tableName; + } + + /** + * @return Column name. + */ + public String columnName() { + return columnName; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryGetColumnsMetaRequest.class, this); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaResult.java new file mode 100644 index 0000000..256b15b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetColumnsMetaResult.java @@ -0,0 +1,42 @@ +/* + * 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.Collection; + +/** + * Query get columns meta result. + */ +public class OdbcQueryGetColumnsMetaResult { + /** Query result rows. */ + private final Collection meta; + + /** + * @param meta Column metadata. + */ + public OdbcQueryGetColumnsMetaResult(Collection meta) { + this.meta = meta; + } + + /** + * @return Query result rows. + */ + public Collection meta() { + return meta; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaRequest.java new file mode 100644 index 0000000..a56aad2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaRequest.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ODBC query get tables meta request. + */ +public class OdbcQueryGetTablesMetaRequest extends OdbcRequest { + /** Catalog search pattern. */ + private final String catalog; + + /** Schema search pattern. */ + private final String schema; + + /** Table search pattern. */ + private final String table; + + /** Table type search pattern. */ + private final String tableType; + + /** + * @param catalog Catalog search pattern. + * @param schema Schema search pattern. + * @param table Table search pattern. + * @param tableType Table type search pattern. + */ + public OdbcQueryGetTablesMetaRequest(String catalog, String schema, String table, String tableType) { + super(GET_TABLES_META); + + this.catalog = catalog; + this.schema = schema; + this.table = table; + this.tableType = tableType; + } + + /** + * @return catalog search pattern. + */ + public String catalog() { + return catalog; + } + + /** + * @return Schema search pattern. + */ + public String schema() { + return schema; + } + + /** + * @return Table search pattern. + */ + public String table() { + return table; + } + + /** + * @return Table type search pattern. + */ + public String tableType() { + return tableType; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcQueryGetTablesMetaRequest.class, this); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaResult.java new file mode 100644 index 0000000..27bebd6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryGetTablesMetaResult.java @@ -0,0 +1,42 @@ +/* + * 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.Collection; + +/** + * Query get columns meta result. + */ +public class OdbcQueryGetTablesMetaResult { + /** Query result rows. */ + private final Collection meta; + + /** + * @param meta Column metadata. + */ + public OdbcQueryGetTablesMetaResult(Collection meta) { + this.meta = meta; + } + + /** + * @return Query result rows. + */ + public Collection meta() { + return meta; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/bae86159/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java new file mode 100644 index 0000000..ebecb60 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +/** + * ODBC command request. + */ +public class OdbcRequest { + /** Handshake request. */ + public static final int HANDSHAKE = 1; + + /** Execute sql query. */ + public static final int EXECUTE_SQL_QUERY = 2; + + /** Fetch query results. */ + public static final int FETCH_SQL_QUERY = 3; + + /** Close query. */ + public static final int CLOSE_SQL_QUERY = 4; + + /** Get columns meta query. */ + public static final int GET_COLUMNS_META = 5; + + /** Get columns meta query. */ + public static final int GET_TABLES_META = 6; + + /** Command. */ + private final int cmd; + + /** + * @param cmd Command type. + */ + public OdbcRequest(int cmd) { + this.cmd = cmd; + } + + /** + * @return Command. + */ + public int command() { + return cmd; + } +}