You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/08/17 10:41:47 UTC

[1/3] ignite git commit: IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.

Repository: ignite
Updated Branches:
  refs/heads/master 37e58bade -> 0e8031444


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
index f132366..0b26dce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
@@ -27,10 +27,10 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * SQL listener query execute request.
+ * JDBC query execute request.
  */
 public class JdbcQueryExecuteRequest extends JdbcRequest {
-    /** Cache name. */
+    /** Schema name. */
     private String schemaName;
 
     /** Fetch size. */
@@ -49,7 +49,7 @@ public class JdbcQueryExecuteRequest extends JdbcRequest {
 
     /**
      */
-    public JdbcQueryExecuteRequest() {
+    JdbcQueryExecuteRequest() {
         super(QRY_EXEC);
     }
 
@@ -100,7 +100,7 @@ public class JdbcQueryExecuteRequest extends JdbcRequest {
     }
 
     /**
-     * @return Cache name.
+     * @return Schema name.
      */
     @Nullable public String schemaName() {
         return schemaName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
index a935215..fdebdb8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
@@ -21,9 +21,10 @@ import java.util.List;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query execute result.
+ * JDBC query execute result.
  */
 public class JdbcQueryExecuteResult extends JdbcResult {
     /** Query ID. */
@@ -44,7 +45,7 @@ public class JdbcQueryExecuteResult extends JdbcResult {
     /**
      * Condtructor.
      */
-    public JdbcQueryExecuteResult() {
+    JdbcQueryExecuteResult() {
         super(QRY_EXEC);
     }
 
@@ -53,7 +54,7 @@ public class JdbcQueryExecuteResult extends JdbcResult {
      * @param items Query result rows.
      * @param last Flag indicates the query has no unfetched results.
      */
-    public JdbcQueryExecuteResult(long queryId, List<List<Object>> items, boolean last) {
+    JdbcQueryExecuteResult(long queryId, List<List<Object>> items, boolean last) {
         super(QRY_EXEC);
 
         this.queryId = queryId;
@@ -147,4 +148,9 @@ public class JdbcQueryExecuteResult extends JdbcResult {
             updateCnt = reader.readLong();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryExecuteResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
index 2e1f551..776c3bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query fetch request.
+ * JDBC query fetch request.
  */
 public class JdbcQueryFetchRequest extends JdbcRequest {
     /** Query ID. */
@@ -35,7 +35,7 @@ public class JdbcQueryFetchRequest extends JdbcRequest {
     /**
      * Constructor.
      */
-    public JdbcQueryFetchRequest() {
+    JdbcQueryFetchRequest() {
         super(QRY_FETCH);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
index 6735c6b..ac4a603 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
@@ -21,9 +21,10 @@ import java.util.List;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query fetch result.
+ * JDBC query fetch result.
  */
 public class JdbcQueryFetchResult extends JdbcResult {
     /** Query result rows. */
@@ -35,7 +36,7 @@ public class JdbcQueryFetchResult extends JdbcResult {
     /**
      * Default constructor is used for deserialization.
      */
-    public JdbcQueryFetchResult() {
+    JdbcQueryFetchResult() {
         super(QRY_FETCH);
     }
 
@@ -43,7 +44,7 @@ public class JdbcQueryFetchResult extends JdbcResult {
      * @param items Query result rows.
      * @param last Flag indicating the query has no unfetched results.
      */
-    public JdbcQueryFetchResult(List<List<Object>> items, boolean last){
+    JdbcQueryFetchResult(List<List<Object>> items, boolean last){
         super(QRY_FETCH);
 
         this.items = items;
@@ -81,4 +82,9 @@ public class JdbcQueryFetchResult extends JdbcResult {
 
         items = JdbcUtils.readItems(reader);
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryFetchResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
index d14c9df..bdef321 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
@@ -23,47 +23,47 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query metadata request.
+ * JDBC query metadata request.
  */
 public class JdbcQueryMetadataRequest extends JdbcRequest {
     /** Query ID. */
-    private long queryId;
+    private long qryId;
 
     /**
      * Constructor.
      */
-    public JdbcQueryMetadataRequest() {
+    JdbcQueryMetadataRequest() {
         super(QRY_META);
     }
 
     /**
-     * @param queryId Query ID.
+     * @param qryId Query ID.
      */
-    public JdbcQueryMetadataRequest(long queryId) {
+    public JdbcQueryMetadataRequest(long qryId) {
         super(QRY_META);
 
-        this.queryId = queryId;
+        this.qryId = qryId;
     }
 
     /**
      * @return Query ID.
      */
     public long queryId() {
-        return queryId;
+        return qryId;
     }
 
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
         super.writeBinary(writer);
 
-        writer.writeLong(queryId);
+        writer.writeLong(qryId);
     }
 
     /** {@inheritDoc} */
     @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
         super.readBinary(reader);
 
-        queryId = reader.readLong();
+        qryId = reader.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
index cc193e3..c8c0991 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
@@ -24,9 +24,10 @@ import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query metadata result.
+ * JDBC query metadata result.
  */
 public class JdbcQueryMetadataResult extends JdbcResult {
     /** Fields metadata. */
@@ -35,7 +36,7 @@ public class JdbcQueryMetadataResult extends JdbcResult {
     /**
      * Default constructor is used for deserialization.
      */
-    public JdbcQueryMetadataResult() {
+    JdbcQueryMetadataResult() {
         super(QRY_META);
     }
 
@@ -43,14 +44,14 @@ public class JdbcQueryMetadataResult extends JdbcResult {
      * @param queryId Query ID.
      * @param meta Query metadata.
      */
-    public JdbcQueryMetadataResult(long queryId, List<JdbcColumnMeta> meta){
+    JdbcQueryMetadataResult(long queryId, List<JdbcColumnMeta> meta){
         super(QRY_META);
 
         this.meta = meta;
     }
 
     /**
-     * @return Query result rows.
+     * @return Query result metadata.
      */
     public List<JdbcColumnMeta> meta() {
         return meta;
@@ -90,4 +91,9 @@ public class JdbcQueryMetadataResult extends JdbcResult {
             }
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryMetadataResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
index 0e144cc..4ef75f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
@@ -24,24 +24,43 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
 
 /**
- * SQL listener command request.
+ * JDBC request.
  */
 public class JdbcRequest extends SqlListenerRequest implements JdbcRawBinarylizable {
-    /** Execute sql query. */
-    public static final byte QRY_EXEC = 2;
+    /** Execute sql query request. */
+    static final byte QRY_EXEC = 2;
 
-    /** Fetch query results. */
-    public static final byte QRY_FETCH = 3;
+    /** Fetch query results request. */
+    static final byte QRY_FETCH = 3;
 
-    /** Close query. */
-    public static final byte QRY_CLOSE = 4;
+    /** Close query request. */
+    static final byte QRY_CLOSE = 4;
 
-    /** Get columns meta query. */
-    public static final byte QRY_META = 5;
+    /** Get query columns metadata request. */
+    static final byte QRY_META = 5;
 
     /** Batch queries. */
     public static final byte BATCH_EXEC = 6;
 
+    /** Get tables metadata request. */
+    static final byte META_TABLES = 7;
+
+    /** Get columns metadata request. */
+    static final byte META_COLUMNS = 8;
+
+    /** Get indexes metadata request. */
+    static final byte META_INDEXES = 9;
+
+    /** Get SQL query parameters metadata request. */
+    static final byte META_PARAMS = 10;
+
+    /** Get primary keys metadata request. */
+    static final byte META_PRIMARY_KEYS = 11;
+
+    /** Get schemas metadata request. */
+    static final byte META_SCHEMAS = 12;
+
+
     /** Request type. */
     private byte type;
 
@@ -105,6 +124,36 @@ public class JdbcRequest extends SqlListenerRequest implements JdbcRawBinaryliza
 
                 break;
 
+            case META_TABLES:
+                req = new JdbcMetaTablesRequest();
+
+                break;
+
+            case META_COLUMNS:
+                req = new JdbcMetaColumnsRequest();
+
+                break;
+
+            case META_INDEXES:
+                req = new JdbcMetaIndexesRequest();
+
+                break;
+
+            case META_PARAMS:
+                req = new JdbcMetaParamsRequest();
+
+                break;
+
+            case META_PRIMARY_KEYS:
+                req = new JdbcMetaPrimaryKeysRequest();
+
+                break;
+
+            case META_SCHEMAS:
+                req = new JdbcMetaSchemasRequest();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + reqType + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 60c08f9..7e58f99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -17,8 +17,15 @@
 
 package org.apache.ignite.internal.processors.odbc.jdbc;
 
+import java.sql.ParameterMetaData;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
@@ -26,10 +33,15 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.odbc.OdbcQueryGetColumnsMetaRequest;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.typedef.F;
@@ -37,13 +49,19 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_COLUMNS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_INDEXES;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_PARAMS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_PRIMARY_KEYS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_SCHEMAS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_TABLES;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_CLOSE;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_EXEC;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_FETCH;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_META;
 
 /**
- * SQL query handler.
+ * JDBC request handler.
  */
 public class JdbcRequestHandler implements SqlListenerRequestHandler {
     /** Query ID sequence. */
@@ -92,7 +110,7 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
      * @param autoCloseCursors Flag to automatically close server cursors.
      */
     public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors,
-        boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, 
+        boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly,
         boolean autoCloseCursors) {
         this.ctx = ctx;
         this.busyLock = busyLock;
@@ -134,6 +152,24 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
 
                 case BATCH_EXEC:
                     return executeBatch((JdbcBatchExecuteRequest)req);
+
+                case META_TABLES:
+                    return getTablesMeta((JdbcMetaTablesRequest)req);
+
+                case META_COLUMNS:
+                    return getColumnsMeta((JdbcMetaColumnsRequest)req);
+
+                case META_INDEXES:
+                    return getIndexesMeta((JdbcMetaIndexesRequest)req);
+
+                case META_PARAMS:
+                    return getParametersMeta((JdbcMetaParamsRequest)req);
+
+                case META_PRIMARY_KEYS:
+                    return getPrimaryKeys((JdbcMetaPrimaryKeysRequest)req);
+
+                case META_SCHEMAS:
+                    return getSchemas((JdbcMetaSchemasRequest)req);
             }
 
             return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, "Unsupported JDBC request [req=" + req + ']');
@@ -148,6 +184,20 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
         return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeHandshake(BinaryWriterExImpl writer) {
+        // Handshake OK.
+        writer.writeBoolean(true);
+
+        // Write server version.
+        writer.writeByte(IgniteVersionUtils.VER.major());
+        writer.writeByte(IgniteVersionUtils.VER.minor());
+        writer.writeByte(IgniteVersionUtils.VER.maintenance());
+        writer.writeString(IgniteVersionUtils.VER.stage());
+        writer.writeLong(IgniteVersionUtils.VER.revisionTimestamp());
+        writer.writeByteArray(IgniteVersionUtils.VER.revisionHash());
+    }
+
     /**
      * {@link JdbcQueryExecuteRequest} command handler.
      *
@@ -318,7 +368,7 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
      * @return Response.
      */
     private SqlListenerResponse executeBatch(JdbcBatchExecuteRequest req) {
-        String schemaName = req.schema();
+        String schemaName = req.schemaName();
 
         if (F.isEmpty(schemaName))
             schemaName = QueryUtils.DFLT_SCHEMA;
@@ -365,4 +415,221 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
                 SqlListenerResponse.STATUS_FAILED, e.toString()));
         }
     }
+
+    /**
+     * @param req Get tables metadata request.
+     * @return Response.
+     */
+    private JdbcResponse getTablesMeta(JdbcMetaTablesRequest req) {
+        try {
+            List<JdbcTableMeta> meta = new ArrayList<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    JdbcTableMeta tableMeta = new JdbcTableMeta(table.schemaName(), table.tableName(), "TABLE");
+
+                    if (!meta.contains(tableMeta))
+                        meta.add(tableMeta);
+                }
+            }
+
+            JdbcMetaTablesResult res = new JdbcMetaTablesResult(meta);
+
+            return new JdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get tables metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetColumnsMetaRequest} command handler.
+     *
+     * @param req Get columns metadata request.
+     * @return Response.
+     */
+    private JdbcResponse getColumnsMeta(JdbcMetaColumnsRequest req) {
+        try {
+            Collection<JdbcColumnMeta> meta = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    for (Map.Entry<String, Class<?>> field : table.fields().entrySet()) {
+                        if (!matches(field.getKey(), req.columnName()))
+                            continue;
+
+                        JdbcColumnMeta columnMeta = new JdbcColumnMeta(table.schemaName(), table.tableName(),
+                            field.getKey(), field.getValue());
+
+                        if (!meta.contains(columnMeta))
+                            meta.add(columnMeta);
+                    }
+                }
+            }
+
+            JdbcMetaColumnsResult res = new JdbcMetaColumnsResult(meta);
+
+            return new JdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get columns metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getIndexesMeta(JdbcMetaIndexesRequest req) {
+        try {
+            Collection<JdbcIndexMeta> meta = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    for (GridQueryIndexDescriptor idxDesc : table.indexes().values())
+                        meta.add(new JdbcIndexMeta(table.schemaName(), table.tableName(), idxDesc));
+                }
+            }
+
+            return new JdbcResponse(new JdbcMetaIndexesResult(meta));
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get parameters metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getParametersMeta(JdbcMetaParamsRequest req) {
+        try {
+            ParameterMetaData paramMeta = ctx.query().prepareNativeStatement(req.schemaName(), req.sql())
+                .getParameterMetaData();
+
+            int size = paramMeta.getParameterCount();
+
+            List<JdbcParameterMeta> meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                meta.add(new JdbcParameterMeta(paramMeta, i + 1));
+
+            JdbcMetaParamsResult res = new JdbcMetaParamsResult(meta);
+
+            return new JdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get parameters metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getPrimaryKeys(JdbcMetaPrimaryKeysRequest req) {
+        try {
+            Collection<JdbcPrimaryKeyMeta> meta = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    List<String> fields = new ArrayList<>();
+
+                    for (String field : table.fields().keySet()) {
+                        if (table.property(field).key())
+                            fields.add(field);
+                    }
+
+
+                    final String keyName = table.keyFieldName() == null ?
+                        "PK_" + table.schemaName() + "_" + table.tableName() :
+                        table.keyFieldName();
+
+                    if (fields.isEmpty()) {
+                        meta.add(new JdbcPrimaryKeyMeta(table.schemaName(), table.tableName(), keyName,
+                            Collections.singletonList("_KEY")));
+                    }
+                    else
+                        meta.add(new JdbcPrimaryKeyMeta(table.schemaName(), table.tableName(), keyName, fields));
+                }
+            }
+
+            return new JdbcResponse(new JdbcMetaPrimaryKeysResult(meta));
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get parameters metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getSchemas(JdbcMetaSchemasRequest req) {
+        try {
+            String schemaPtrn = req.schemaName();
+
+            Set<String> schemas = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (matches(table.schemaName(), schemaPtrn))
+                        schemas.add(table.schemaName());
+                }
+            }
+
+            return new JdbcResponse(new JdbcMetaSchemasResult(schemas));
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get schemas metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * Checks whether string matches SQL pattern.
+     *
+     * @param str String.
+     * @param ptrn Pattern.
+     * @return Whether string matches pattern.
+     */
+    private static boolean matches(String str, String ptrn) {
+        return str != null && (F.isEmpty(ptrn) ||
+            str.matches(ptrn.replace("%", ".*").replace("_", ".")));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
index 48affe9..202905b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
@@ -23,21 +23,39 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 
 /**
- * SQL listener response.
+ * JDBC response result.
  */
 public class JdbcResult implements JdbcRawBinarylizable {
     /** Execute sql result. */
-    public static final byte QRY_EXEC = 2;
+    static final byte QRY_EXEC = 2;
 
     /** Fetch query results. */
-    public static final byte QRY_FETCH = 3;
+    static final byte QRY_FETCH = 3;
 
-    /** Get columns meta query result. */
-    public static final byte QRY_META = 4;
+    /** Query result's columns metadata result. */
+    static final byte QRY_META = 5;
 
     /** Batch queries. */
     public static final byte BATCH_EXEC = 6;
 
+    /** Tables metadata result. */
+    static final byte META_TABLES = 7;
+
+    /** Columns metadata result. */
+    static final byte META_COLUMNS = 8;
+
+    /** Indexes metadata result. */
+    static final byte META_INDEXES = 9;
+
+    /** SQL query parameters metadata result. */
+    static final byte META_PARAMS = 10;
+
+    /** Primary keys metadata result. */
+    static final byte META_PRIMARY_KEYS = 11;
+
+    /** Database schemas metadata result. */
+    static final byte META_SCHEMAS = 12;
+
     /** Success status. */
     private byte type;
 
@@ -91,6 +109,36 @@ public class JdbcResult implements JdbcRawBinarylizable {
 
                 break;
 
+            case META_TABLES:
+                res = new JdbcMetaTablesResult();
+
+                break;
+
+            case META_COLUMNS:
+                res = new JdbcMetaColumnsResult();
+
+                break;
+
+            case META_INDEXES:
+                res = new JdbcMetaIndexesResult();
+
+                break;
+
+            case META_PARAMS:
+                res = new JdbcMetaParamsResult();
+
+                break;
+
+            case META_PRIMARY_KEYS:
+                res = new JdbcMetaPrimaryKeysResult();
+
+                break;
+
+            case META_SCHEMAS:
+                res = new JdbcMetaSchemasResult();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java
new file mode 100644
index 0000000..b954e97
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java
@@ -0,0 +1,82 @@
+/*
+ * 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.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC table metadata.
+ */
+public class JdbcTableMeta implements JdbcRawBinarylizable {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcTableMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param tblType Table type.
+     */
+    JdbcTableMeta(String schemaName, String tblName, String tblType) {
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        schemaName = reader.readString();
+        tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcTableMeta.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
index 65efbf5..d556419 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.odbc.jdbc;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
@@ -29,7 +30,7 @@ import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
  */
 public class JdbcUtils {
     /**
-     * @param writer Binari writer.
+     * @param writer Binary writer.
      * @param items Query results items.
      */
     public static void writeItems(BinaryWriterExImpl writer, List<List<Object>> items) {
@@ -70,4 +71,38 @@ public class JdbcUtils {
         } else
             return Collections.emptyList();
     }
+
+    /**
+     * @param writer Binary writer.
+     * @param lst List to write.
+     */
+    public static void writeStringCollection(BinaryWriterExImpl writer, Collection<String> lst) {
+        if (lst == null)
+            writer.writeInt(0);
+        else {
+            writer.writeInt(lst.size());
+
+            for (String s : lst)
+                writer.writeString(s);
+        }
+    }
+
+    /**
+     * @param reader Binary reader.
+     * @return List of string.
+     */
+    public static List<String> readStringList(BinaryReaderExImpl reader) {
+        int size = reader.readInt();
+
+        if (size > 0) {
+            List<String> lst = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i)
+                lst.add(reader.readString());
+
+            return lst;
+        }
+        else
+            return Collections.emptyList();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index bb54b59..692043c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -33,6 +33,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
@@ -156,6 +157,11 @@ public class OdbcRequestHandler implements SqlListenerRequestHandler {
         return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeHandshake(BinaryWriterExImpl writer) {
+        writer.writeBoolean(true);
+    }
+
     /**
      * {@link OdbcQueryExecuteRequest} command handler.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index baafb1e..1d154d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -688,7 +688,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 if (!F.isEmpty(qryEntities)) {
                     for (QueryEntity qryEntity : qryEntities) {
-                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(cacheName, cctx, qryEntity,
+                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(cacheName, schemaName, cctx, qryEntity,
                             mustDeserializeClss, escape);
 
                         cands.add(cand);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index c1a9e1e..c149335 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -33,6 +33,13 @@ public interface GridQueryTypeDescriptor {
     public String name();
 
     /**
+     * Gets schema name for type (database schema means here).
+     *
+     * @return Schema name.
+     */
+    public String schemaName();
+
+    /**
      * Gets table name for type.
      *
      * @return Table name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index c0da83f..79b90e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -42,6 +42,9 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     /** */
     private String name;
 
+    /** Schema name. */
+    private String schemaName;
+
     /** */
     private String tblName;
 
@@ -120,6 +123,11 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
         return name;
     }
 
+    /** {@inheritDoc} */
+    @Override public String schemaName() {
+        return schemaName;
+    }
+
     /**
      * Sets type name.
      *
@@ -363,6 +371,13 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
         fields.put(name, prop.type());
     }
 
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean valueTextIndex() {
         return valTextIdx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 320b25a..26fc776 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -343,6 +343,7 @@ public class QueryUtils {
      * Create type candidate for query entity.
      *
      * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param cctx Cache context.
      * @param qryEntity Query entity.
      * @param mustDeserializeClss Classes which must be deserialized.
@@ -350,8 +351,8 @@ public class QueryUtils {
      * @return Type candidate.
      * @throws IgniteCheckedException If failed.
      */
-    public static QueryTypeCandidate typeForQueryEntity(String cacheName, GridCacheContext cctx, QueryEntity qryEntity,
-        List<Class<?>> mustDeserializeClss, boolean escape) throws IgniteCheckedException {
+    public static QueryTypeCandidate typeForQueryEntity(String cacheName, String schemaName, GridCacheContext cctx,
+        QueryEntity qryEntity, List<Class<?>> mustDeserializeClss, boolean escape) throws IgniteCheckedException {
         GridKernalContext ctx = cctx.kernalContext();
         CacheConfiguration<?,?> ccfg = cctx.config();
 
@@ -361,6 +362,8 @@ public class QueryUtils {
 
         QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl(cacheName);
 
+        desc.schemaName(schemaName);
+
         desc.aliases(qryEntity.getAliases());
 
         // Key and value classes still can be available if they are primitive or JDK part.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 99246eb..e1ab8e6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -104,13 +104,13 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
     }
 
     /** */
-    private static TypeDesc typeAA = new TypeDesc("A", "A", Collections.<String, Class<?>>emptyMap(), null);
+    private static TypeDesc typeAA = new TypeDesc("A", "A", "A", Collections.<String, Class<?>>emptyMap(), null);
 
     /** */
-    private static TypeDesc typeAB = new TypeDesc("A", "B", Collections.<String, Class<?>>emptyMap(), textIdx);
+    private static TypeDesc typeAB = new TypeDesc("A", "A", "B", Collections.<String, Class<?>>emptyMap(), textIdx);
 
     /** */
-    private static TypeDesc typeBA = new TypeDesc("B", "A", Collections.<String, Class<?>>emptyMap(), null);
+    private static TypeDesc typeBA = new TypeDesc("B", "B", "A", Collections.<String, Class<?>>emptyMap(), null);
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
@@ -467,6 +467,9 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         private final String cacheName;
 
         /** */
+        private final String schemaName;
+
+        /** */
         private final Map<String, Class<?>> valFields;
 
         /** */
@@ -474,13 +477,15 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         /**
          * @param cacheName Cache name.
+         * @param schemaName Schema name.
          * @param name Type name.
          * @param valFields Fields.
          * @param textIdx Fulltext index.
          */
-        private TypeDesc(String cacheName, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
+        private TypeDesc(String cacheName, String schemaName, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
             this.name = name;
             this.cacheName = cacheName;
+            this.schemaName = schemaName;
             this.valFields = Collections.unmodifiableMap(valFields);
             this.textIdx = textIdx;
         }
@@ -496,6 +501,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public String schemaName() {
+            return schemaName;
+        }
+
+        /** {@inheritDoc} */
         @Override public String tableName() {
             return null;
         }


[2/3] ignite git commit: IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java
new file mode 100644
index 0000000..8647258
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java
@@ -0,0 +1,115 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcParameterMeta;
+
+/**
+ * JDBC SQL query's parameters metadata.
+ */
+public class JdbcThinParameterMetadata implements ParameterMetaData {
+    /** Parameters metadata. */
+    private final List<JdbcParameterMeta> meta;
+
+    /**
+     * @param meta Parameters metadata.
+     */
+    public JdbcThinParameterMetadata(List<JdbcParameterMeta> meta) {
+        assert meta != null;
+
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getParameterCount() throws SQLException {
+        return meta.size();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("MagicConstant")
+    @Override public int isNullable(int param) throws SQLException {
+        return parameter(param).isNullable();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isSigned(int param) throws SQLException {
+        return parameter(param).isSigned();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getPrecision(int param) throws SQLException {
+        return parameter(param).precision();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getScale(int param) throws SQLException {
+        return parameter(param).scale();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getParameterType(int param) throws SQLException {
+        return parameter(param).type();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getParameterTypeName(int param) throws SQLException {
+        return parameter(param).typeName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getParameterClassName(int param) throws SQLException {
+        return parameter(param).typeClass();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("MagicConstant")
+    @Override public int getParameterMode(int param) throws SQLException {
+        return parameter(param).mode();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("Parameters metadata is not a wrapper for " + iface.getName());
+
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface != null && iface.isAssignableFrom(JdbcThinParameterMetadata.class);
+    }
+
+    /**
+     * Bounds checks the parameter index.
+     *
+     * @param param Parameter index.
+     * @return Parameter.
+     * @throws SQLException If failed.
+     */
+    private JdbcParameterMeta parameter(int param) throws SQLException {
+        if (param <= 0 || param > meta.size())
+            throw new SQLException("Invalid parameter number");
+
+        return meta.get(param - 1);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
index 455c80f..e6dfa59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.jdbc.thin;
 
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
@@ -39,7 +40,9 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery;
 
 /**
@@ -52,6 +55,9 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
     /** Query arguments. */
     protected ArrayList<Object> args;
 
+    /** Parameters metadata. */
+    private JdbcThinParameterMetadata metaData;
+
     /**
      * Creates new prepared statement.
      *
@@ -322,8 +328,27 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
     /** {@inheritDoc} */
     @Override public ParameterMetaData getParameterMetaData() throws SQLException {
         ensureNotClosed();
+        try {
+            if (conn.isClosed())
+                throw new SQLException("Connection is closed.");
 
-        throw new SQLFeatureNotSupportedException("Meta data for prepared statement is not supported.");
+            if (metaData != null)
+                return metaData;
+
+            JdbcMetaParamsResult res = conn.io().parametersMeta(conn.getSchema(), sql);
+
+            metaData = new JdbcThinParameterMetadata(res.meta());
+
+            return metaData;
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
index 5c61e23..c4be5bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
@@ -100,6 +100,34 @@ public class JdbcThinResultSet implements ResultSet {
     /** Update count. */
     private long updCnt;
 
+    /** Jdbc metadata. Cache the JDBC object on the first access */
+    private JdbcThinResultSetMetadata jdbcMeta;
+
+    /**
+     * Constructs static result set.
+     *
+     * @param fields Fields.
+     * @param meta Columns metadata.
+     */
+    JdbcThinResultSet(List<List<Object>> fields, List<JdbcColumnMeta> meta) {
+        stmt = null;
+        fetchSize = 0;
+        qryId = -1L;
+        finished = true;
+        isQuery = true;
+        updCnt = -1;
+
+        this.rows = fields;
+
+        rowsIter = fields.iterator();
+
+        this.meta = meta;
+
+        metaInit = true;
+
+        initColumnOrder();
+    }
+
     /**
      * Creates new result set.
      *
@@ -180,7 +208,7 @@ public class JdbcThinResultSet implements ResultSet {
 
     /** {@inheritDoc} */
     @Override public void close() throws SQLException {
-        if (closed || stmt.connection().isClosed())
+        if (closed || stmt == null || stmt.connection().isClosed())
             return;
 
         try {
@@ -497,7 +525,10 @@ public class JdbcThinResultSet implements ResultSet {
     @Override public ResultSetMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
-        return new JdbcThinResultSetMetadata(meta());
+        if (jdbcMeta == null)
+            jdbcMeta = new JdbcThinResultSetMetadata(meta());
+
+        return jdbcMeta;
     }
 
     /** {@inheritDoc} */
@@ -1648,7 +1679,6 @@ public class JdbcThinResultSet implements ResultSet {
     }
 
     /**
-     * Init column order map.
      * @throws SQLException On error.
      * @return Column order map.
      */
@@ -1659,6 +1689,15 @@ public class JdbcThinResultSet implements ResultSet {
         if(!metaInit)
             meta();
 
+        initColumnOrder();
+
+        return colOrder;
+    }
+
+    /**
+     * Init column order map.
+     */
+    private void initColumnOrder() {
         colOrder = new HashMap<>(meta.size());
 
         for (int i = 0; i < meta.size(); ++i) {
@@ -1667,8 +1706,6 @@ public class JdbcThinResultSet implements ResultSet {
             if(!colOrder.containsKey(colName))
                 colOrder.put(colName, i);
         }
-
-        return colOrder;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
index b01350a..3772b83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
@@ -46,8 +46,8 @@ public class JdbcThinStatement implements Statement {
     /** Default queryPage size. */
     private static final int DFLT_PAGE_SIZE = SqlQuery.DFLT_PAGE_SIZE;
 
-    /** Ignite endpoint and I/O protocol implementation. */
-    private JdbcThinConnection conn;
+    /** JDBC Connection implementation. */
+    protected JdbcThinConnection conn;
 
     /** Closed flag. */
     private boolean closed;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
index f54d5fd..e124921 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
@@ -34,6 +34,18 @@ import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaIndexesRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaIndexesResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaPrimaryKeysRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaPrimaryKeysResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaSchemasRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaSchemasResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaTablesRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaTablesResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryCloseRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest;
@@ -47,6 +59,7 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult;
 import org.apache.ignite.internal.util.ipc.loopback.IpcClientTcpEndpoint;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteProductVersion;
 
 /**
  * JDBC IO layer implementation based on blocking IPC streams.
@@ -59,7 +72,7 @@ public class JdbcThinTcpIo {
     private static final int HANDSHAKE_MSG_SIZE = 13;
 
     /** Initial output for query message. */
-    private static final int QUERY_EXEC_MSG_INIT_CAP = 256;
+    private static final int DYNAMIC_SIZE_MSG_CAP = 256;
 
     /** Maximum batch query count. */
     private static final int MAX_BATCH_QRY_CNT = 32;
@@ -115,6 +128,9 @@ public class JdbcThinTcpIo {
     /** Closed flag. */
     private boolean closed;
 
+    /** Ignite server version. */
+    private IgniteProductVersion igniteVer;
+
     /**
      * Constructor.
      *
@@ -202,8 +218,20 @@ public class JdbcThinTcpIo {
 
         boolean accepted = reader.readBoolean();
 
-        if (accepted)
+        if (accepted) {
+            byte maj = reader.readByte();
+            byte min = reader.readByte();
+            byte maintenance = reader.readByte();
+
+            String stage = reader.readString();
+
+            long ts = reader.readLong();
+            byte[] hash = reader.readByteArray();
+
+            igniteVer = new IgniteProductVersion(maj, min, maintenance, stage, ts, hash);
+
             return;
+        }
 
         short maj = reader.readShort();
         short min = reader.readShort();
@@ -231,7 +259,7 @@ public class JdbcThinTcpIo {
         String sql, List<Object> args)
         throws IOException, IgniteCheckedException {
         return sendRequest(new JdbcQueryExecuteRequest(cache, fetchSize, maxRows, sql,
-            args == null ? null : args.toArray(new Object[args.size()])), QUERY_EXEC_MSG_INIT_CAP);
+            args == null ? null : args.toArray(new Object[args.size()])), DYNAMIC_SIZE_MSG_CAP);
     }
 
     /**
@@ -295,21 +323,89 @@ public class JdbcThinTcpIo {
     }
 
     /**
-     * @param schema Schema.
+     * @param schemaName Schema.
      * @param batch Batch queries.
      * @return Result.
      * @throws IOException On error.
      * @throws IgniteCheckedException On error.
      */
-    public JdbcBatchExecuteResult batchExecute(String schema, List<JdbcQuery> batch)
+    public JdbcBatchExecuteResult batchExecute(String schemaName, List<JdbcQuery> batch)
         throws IOException, IgniteCheckedException {
         int cnt = Math.min(MAX_BATCH_QRY_CNT, batch.size());
 
-        return sendRequest(new JdbcBatchExecuteRequest(schema, batch), QUERY_EXEC_MSG_INIT_CAP * cnt);
+        return sendRequest(new JdbcBatchExecuteRequest(schemaName, batch), DYNAMIC_SIZE_MSG_CAP * cnt);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaTablesResult tablesMeta(String schemaPtrn, String tablePtrn)
+        throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaTablesRequest(schemaPtrn, tablePtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @param columnPtrn Column name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaColumnsResult columnsMeta(String schemaPtrn, String tablePtrn, String columnPtrn)
+        throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaColumnsRequest(schemaPtrn, tablePtrn, columnPtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaIndexesResult indexMeta(String schemaPtrn, String tablePtrn) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaIndexesRequest(schemaPtrn, tablePtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param sql SQL query.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaParamsResult parametersMeta(String schemaPtrn, String sql) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaParamsRequest(schemaPtrn, sql), DYNAMIC_SIZE_MSG_CAP);
     }
 
     /**
-     * @param req ODBC request.
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaPrimaryKeysResult primaryKeysMeta(String schemaPtrn, String tablePtrn) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaPrimaryKeysRequest(schemaPtrn, tablePtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaSchemasResult schemasMeta(String schemaPtrn) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaSchemasRequest(schemaPtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param req JDBC request bytes.
      * @throws IOException On error.
      */
     private void send(byte[] req) throws IOException {
@@ -434,4 +530,11 @@ public class JdbcThinTcpIo {
     public boolean tcpNoDelay() {
         return tcpNoDelay;
     }
+
+    /**
+     * @return Ignnite server version.
+     */
+    IgniteProductVersion igniteVersion() {
+        return igniteVer;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
index a879796..5a49e3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
@@ -198,9 +198,11 @@ public class SqlListenerNioListener extends GridNioServerListenerAdapter<byte[]>
 
         String errMsg = null;
 
+        SqlListenerConnectionContext connCtx = null;
+
         if (SUPPORTED_VERS.contains(ver)) {
             // Prepare context.
-            SqlListenerConnectionContext connCtx = prepareContext(ver, reader);
+            connCtx = prepareContext(ver, reader);
 
             ses.addMeta(CONN_CTX_META_KEY, connCtx);
         }
@@ -213,9 +215,10 @@ public class SqlListenerNioListener extends GridNioServerListenerAdapter<byte[]>
         // Send response.
         BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(8), null, null);
 
-        if (errMsg == null)
-            writer.writeBoolean(true);
+        if (connCtx != null)
+            connCtx.handler().writeHandshake(writer);
         else {
+            // Failed handshake response
             writer.writeBoolean(false);
             writer.writeShort(CURRENT_VER.major());
             writer.writeShort(CURRENT_VER.minor());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
index 98dc039..348054f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.odbc;
 
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+
 /**
  * SQL listener request handler.
  */
@@ -36,4 +38,11 @@ public interface SqlListenerRequestHandler {
      * @return Error response.
      */
     public SqlListenerResponse handleException(Exception e);
+
+    /**
+     * Write successful handshake response.
+     *
+     * @param writer Binary writer.
+     */
+    public void writeHandshake(BinaryWriterExImpl writer);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
index 9f71bff..25e1049 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
@@ -31,8 +31,8 @@ import org.jetbrains.annotations.Nullable;
  * JDBC batch execute request.
  */
 public class JdbcBatchExecuteRequest extends JdbcRequest {
-    /** Cache name. */
-    private String schema;
+    /** Schema name. */
+    private String schemaName;
 
     /** Sql query. */
     @GridToStringInclude(sensitive = true)
@@ -46,23 +46,23 @@ public class JdbcBatchExecuteRequest extends JdbcRequest {
     }
 
     /**
-     * @param schema Schema.
+     * @param schemaName Schema name.
      * @param queries Queries.
      */
-    public JdbcBatchExecuteRequest(String schema, List<JdbcQuery> queries) {
+    public JdbcBatchExecuteRequest(String schemaName, List<JdbcQuery> queries) {
         super(BATCH_EXEC);
 
         assert !F.isEmpty(queries);
 
-        this.schema = schema;
+        this.schemaName = schemaName;
         this.queries = queries;
     }
 
     /**
-     * @return Schema.
+     * @return Schema name.
      */
-    @Nullable public String schema() {
-        return schema;
+    @Nullable public String schemaName() {
+        return schemaName;
     }
 
     /**
@@ -76,7 +76,7 @@ public class JdbcBatchExecuteRequest extends JdbcRequest {
     @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
         super.writeBinary(writer);
 
-        writer.writeString(schema);
+        writer.writeString(schemaName);
         writer.writeInt(queries.size());
 
         for (JdbcQuery q : queries)
@@ -87,7 +87,7 @@ public class JdbcBatchExecuteRequest extends JdbcRequest {
     @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
         super.readBinary(reader);
 
-        schema = reader.readString();
+        schemaName = reader.readString();
 
         int n = reader.readInt();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
index 7977c22..917e60a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.odbc.jdbc;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * JDBC batch execute result.
@@ -93,4 +94,9 @@ public class JdbcBatchExecuteResult extends JdbcResult {
         errMsg = reader.readString();
         updateCnts = reader.readIntArray();
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcBatchExecuteResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
index 07cbabe..9f145e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
@@ -21,19 +21,21 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.jdbc.thin.JdbcThinUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener column metadata.
+ * JDBC column metadata.
  */
 public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /** Cache name. */
     private String schemaName;
 
     /** Table name. */
-    private String tableName;
+    private String tblName;
 
     /** Column name. */
-    private String columnName;
+    private String colName;
 
     /** Data type. */
     private int dataType;
@@ -47,16 +49,17 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /**
      * Default constructor is used for serialization.
      */
-    public JdbcColumnMeta() {
+    JdbcColumnMeta() {
+        // No-op.
     }
 
     /**
      * @param info Field metadata.
      */
-    public JdbcColumnMeta(GridQueryFieldMetadata info) {
+    JdbcColumnMeta(GridQueryFieldMetadata info) {
         this.schemaName = info.schemaName();
-        this.tableName = info.typeName();
-        this.columnName = info.fieldName();
+        this.tblName = info.typeName();
+        this.colName = info.fieldName();
 
         dataType = JdbcThinUtils.type(info.fieldTypeName());
         dataTypeName = JdbcThinUtils.typeName(info.fieldTypeName());
@@ -64,6 +67,24 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     }
 
     /**
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param colName Column.
+     * @param cls Type.
+     */
+    public JdbcColumnMeta(String schemaName, String tblName, String colName, Class<?> cls) {
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+        this.colName = colName;
+
+        String type = cls.getName();
+
+        dataType = JdbcThinUtils.type(type);
+        dataTypeName = JdbcThinUtils.typeName(type);
+        dataTypeClass = type;
+    }
+
+    /**
      * @return Schema name.
      */
     public String schemaName() {
@@ -74,14 +95,14 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
      * @return Table name.
      */
     public String tableName() {
-        return tableName;
+        return tblName;
     }
 
     /**
      * @return Column name.
      */
     public String columnName() {
-        return columnName;
+        return colName;
     }
 
     /**
@@ -108,8 +129,8 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriterExImpl writer) {
         writer.writeString(schemaName);
-        writer.writeString(tableName);
-        writer.writeString(columnName);
+        writer.writeString(tblName);
+        writer.writeString(colName);
 
         writer.writeInt(dataType);
         writer.writeString(dataTypeName);
@@ -119,11 +140,39 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /** {@inheritDoc} */
     @Override public void readBinary(BinaryReaderExImpl reader) {
         schemaName = reader.readString();
-        tableName = reader.readString();
-        columnName = reader.readString();
+        tblName = reader.readString();
+        colName = reader.readString();
 
         dataType = reader.readInt();
         dataTypeName = reader.readString();
         dataTypeClass = reader.readString();
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        JdbcColumnMeta meta = (JdbcColumnMeta)o;
+
+        return F.eq(schemaName, meta.schemaName) && F.eq(tblName, meta.tblName) && F.eq(colName, meta.colName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName != null ? schemaName.hashCode() : 0;
+
+        result = 31 * result + (tblName != null ? tblName.hashCode() : 0);
+        result = 31 * result + colName.hashCode();
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcColumnMeta.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java
new file mode 100644
index 0000000..d33f887
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java
@@ -0,0 +1,192 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC index metadata.
+ */
+public class JdbcIndexMeta implements JdbcRawBinarylizable {
+    /** Index schema name. */
+    private String schemaName;
+
+    /** Index table name. */
+    private String tblName;
+
+    /** Index name. */
+    private String idxName;
+
+    /** Index type. */
+    private QueryIndexType type;
+
+    /** Index fields */
+    private List<String> fields;
+
+    /** Index fields is ascending. */
+    private List<Boolean> fieldsAsc;
+
+    /**
+     * Default constructor is used for binary serialization.
+     */
+    JdbcIndexMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param idx Index info.
+     */
+    JdbcIndexMeta(String schemaName, String tblName, GridQueryIndexDescriptor idx) {
+        assert tblName != null;
+        assert idx != null;
+        assert idx.fields() != null;
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+
+        idxName = idx.name();
+        type = idx.type();
+        fields = new ArrayList(idx.fields());
+
+        fieldsAsc = new ArrayList<>(fields.size());
+
+        for (int i = 0; i < fields.size(); ++i)
+            fieldsAsc.add(!idx.descending(fields.get(i)));
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Index name.
+     */
+    public String indexName() {
+        return idxName;
+    }
+
+    /**
+     * @return Index type.
+     */
+    public QueryIndexType type() {
+        return type;
+    }
+
+    /**
+     * @return Index fields
+     */
+    public List<String> fields() {
+        return fields;
+    }
+
+    /**
+     * @return Index fields is ascending.
+     */
+    public List<Boolean> fieldsAsc() {
+        return fieldsAsc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+        writer.writeString(idxName);
+        writer.writeByte((byte)type.ordinal());
+
+        JdbcUtils.writeStringCollection(writer, fields);
+
+        if (fieldsAsc == null)
+            writer.writeInt(0);
+        else {
+            writer.writeInt(fieldsAsc.size());
+
+            for (Boolean b : fieldsAsc)
+                writer.writeBoolean(b.booleanValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        schemaName = reader.readString();
+        tblName = reader.readString();
+        idxName = reader.readString();
+        type = QueryIndexType.fromOrdinal(reader.readByte());
+
+        fields = JdbcUtils.readStringList(reader);
+
+        int size = reader.readInt();
+
+        if (size > 0) {
+            fieldsAsc = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i)
+                fieldsAsc .add(reader.readBoolean());
+        }
+        else
+            fieldsAsc = Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        JdbcIndexMeta meta = (JdbcIndexMeta)o;
+
+        return F.eq(schemaName, meta.schemaName) && F.eq(tblName, meta.tblName) && F.eq(idxName, meta.idxName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName != null ? schemaName.hashCode() : 0;
+
+        result = 31 * result + tblName.hashCode();
+        result = 31 * result + idxName.hashCode();
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcIndexMeta.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java
new file mode 100644
index 0000000..fca1bf7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * JDBC get columns metadata request.
+ */
+public class JdbcMetaColumnsRequest extends JdbcRequest {
+    /** Schema name pattern. */
+    private String schemaName;
+
+    /** Table name pattern. */
+    private String tblName;
+
+    /** Column name pattern. */
+    private String colName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaColumnsRequest() {
+        super(META_COLUMNS);
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param colName Column name.
+     */
+    public JdbcMetaColumnsRequest(String schemaName, String tblName, String colName) {
+        super(META_COLUMNS);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+        this.colName = colName;
+    }
+
+    /**
+     * @return Schema name pattern.
+     */
+    @Nullable public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name pattern.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Column name pattern.
+     */
+    public String columnName() {
+        return colName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+        writer.writeString(colName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        tblName = reader.readString();
+        colName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaColumnsRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
new file mode 100644
index 0000000..da270de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
@@ -0,0 +1,99 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC columns metadata result.
+ */
+public class JdbcMetaColumnsResult extends JdbcResult {
+    /** Columns metadata. */
+    private List<JdbcColumnMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaColumnsResult() {
+        super(META_COLUMNS);
+    }
+
+    /**
+     * @param meta Columns metadata.
+     */
+    JdbcMetaColumnsResult(Collection<JdbcColumnMeta> meta) {
+        super(META_COLUMNS);
+
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /**
+     * @return Columns metadata.
+     */
+    public List<JdbcColumnMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcColumnMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcColumnMeta m = new JdbcColumnMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaColumnsResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java
new file mode 100644
index 0000000..d4a53d8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * JDBC indexes metadata request.
+ */
+public class JdbcMetaIndexesRequest extends JdbcRequest {
+    /** Schema name pattern. */
+    private String schemaName;
+
+    /** Table name pattern. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaIndexesRequest() {
+        super(META_INDEXES);
+    }
+
+    /**
+     * @param schemaName Cache name.
+     * @param tblName Table name.
+     */
+    public JdbcMetaIndexesRequest(String schemaName, String tblName) {
+        super(META_INDEXES);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    @Nullable public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaIndexesRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java
new file mode 100644
index 0000000..2316dfc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java
@@ -0,0 +1,98 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC indexes metadata result.
+ */
+public class JdbcMetaIndexesResult extends JdbcResult {
+    /** Indexes metadata. */
+    private List<JdbcIndexMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaIndexesResult() {
+        super(META_INDEXES);
+    }
+
+    /**
+     * @param meta Indexes metadata.
+     */
+    JdbcMetaIndexesResult(Collection<JdbcIndexMeta> meta) {
+        super(META_INDEXES);
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /**
+     * @return Indexes metadata.
+     */
+    public List<JdbcIndexMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcIndexMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcIndexMeta m = new JdbcIndexMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaIndexesResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java
new file mode 100644
index 0000000..6b955f9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC SQL query parameters metadata request.
+ */
+public class JdbcMetaParamsRequest extends JdbcRequest {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Query. */
+    private String sql;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaParamsRequest() {
+        super(META_PARAMS);
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param sql SQL Query.
+     */
+    public JdbcMetaParamsRequest(String schemaName, String sql) {
+        super(META_PARAMS);
+
+        this.schemaName = schemaName;
+        this.sql = sql;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return SQL Query.
+     */
+    public String sql() {
+        return sql;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(sql);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        sql = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaParamsRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java
new file mode 100644
index 0000000..7563e01
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java
@@ -0,0 +1,97 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC SQL query parameters metadata result.
+ */
+public class JdbcMetaParamsResult extends JdbcResult {
+    /** Parameters meta results. */
+    private List<JdbcParameterMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaParamsResult() {
+        super(META_PARAMS);
+    }
+
+    /**
+     * @param meta Column metadata.
+     */
+    JdbcMetaParamsResult(List<JdbcParameterMeta> meta) {
+        super(META_PARAMS);
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcParameterMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcParameterMeta m = new JdbcParameterMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /**
+     * @return SQL query parameters metadata.
+     */
+    public List<JdbcParameterMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaParamsResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java
new file mode 100644
index 0000000..957225a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * JDBC get primary keys metadata request.
+ */
+public class JdbcMetaPrimaryKeysRequest extends JdbcRequest {
+    /** Schema name pattern. */
+    private String schemaName;
+
+    /** Table name pattern. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaPrimaryKeysRequest() {
+        super(META_PRIMARY_KEYS);
+    }
+
+    /**
+     * @param schemaName Cache name.
+     * @param tblName Table name.
+     */
+    public JdbcMetaPrimaryKeysRequest(String schemaName, String tblName) {
+        super(META_PRIMARY_KEYS);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name pattern.
+     */
+    @Nullable public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name pattern.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaPrimaryKeysRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java
new file mode 100644
index 0000000..bd0dd90
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java
@@ -0,0 +1,99 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC primary keys metadata result.
+ */
+public class JdbcMetaPrimaryKeysResult extends JdbcResult {
+    /** Query result rows. */
+    private List<JdbcPrimaryKeyMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaPrimaryKeysResult() {
+        super(META_PRIMARY_KEYS);
+    }
+
+    /**
+     * @param meta Column metadata.
+     */
+    JdbcMetaPrimaryKeysResult(Collection<JdbcPrimaryKeyMeta> meta) {
+        super(META_PRIMARY_KEYS);
+
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcPrimaryKeyMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcPrimaryKeyMeta m = new JdbcPrimaryKeyMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /**
+     * @return Primary keys metadata.
+     */
+    public List<JdbcPrimaryKeyMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaPrimaryKeysResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java
new file mode 100644
index 0000000..43bbe5d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata request.
+ */
+public class JdbcMetaSchemasRequest extends JdbcRequest {
+    /** Schema search pattern. */
+    private String schemaName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaSchemasRequest() {
+        super(META_SCHEMAS);
+    }
+
+    /**
+     * @param schemaName Schema search pattern.
+     */
+    public JdbcMetaSchemasRequest(String schemaName) {
+        super(META_SCHEMAS);
+
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Schema search pattern.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        this.schemaName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaSchemasRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java
new file mode 100644
index 0000000..48b6aae
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.odbc.jdbc;
+
+import java.util.Collection;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata result.
+ */
+public class JdbcMetaSchemasResult extends JdbcResult {
+    /** Found schemas. */
+    private Collection<String> schemas;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaSchemasResult() {
+        super(META_SCHEMAS);
+    }
+
+    /**
+     * @param schemas Found schemas.
+     */
+    JdbcMetaSchemasResult(Collection<String> schemas) {
+        super(META_SCHEMAS);
+        this.schemas = schemas;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        JdbcUtils.writeStringCollection(writer, schemas);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemas = JdbcUtils.readStringList(reader);
+    }
+
+    /**
+     * @return Found schemas.
+     */
+    public Collection<String> schemas() {
+        return schemas;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaSchemasResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java
new file mode 100644
index 0000000..740b656
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata request.
+ */
+public class JdbcMetaTablesRequest extends JdbcRequest {
+    /** Schema search pattern. */
+    private String schemaName;
+
+    /** Table search pattern. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaTablesRequest() {
+        super(META_TABLES);
+    }
+
+    /**
+     * @param schemaName Schema search pattern.
+     * @param tblName Table search pattern.
+     */
+    public JdbcMetaTablesRequest(String schemaName, String tblName) {
+        super(META_TABLES);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema search pattern.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table search pattern.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        this.schemaName = reader.readString();
+        this.tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaTablesRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java
new file mode 100644
index 0000000..585667e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java
@@ -0,0 +1,97 @@
+/*
+ * 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.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata result.
+ */
+public class JdbcMetaTablesResult extends JdbcResult {
+    /** Tables metadata. */
+    private List<JdbcTableMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaTablesResult() {
+        super(META_TABLES);
+    }
+
+    /**
+     * @param meta Tables metadata.
+     */
+    JdbcMetaTablesResult(List<JdbcTableMeta> meta) {
+        super(META_TABLES);
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcTableMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcTableMeta m = new JdbcTableMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /**
+     * @return Tables metadata.
+     */
+    public List<JdbcTableMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaTablesResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
new file mode 100644
index 0000000..dd3b18b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
@@ -0,0 +1,165 @@
+/*
+ * 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.jdbc;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC SQL query parameter metadata.
+ *
+ * {@see java.sql.ParameterMetaData}.
+ */
+public class JdbcParameterMeta implements JdbcRawBinarylizable {
+    /** Null value is allow for the param. */
+    private int isNullable;
+
+    /** Signed flag. */
+    private boolean signed;
+
+    /** Precision. */
+    private int precision;
+
+    /** Scale. */
+    private int scale;
+
+    /** SQL type ID. */
+    private int type;
+
+    /** SQL type name. */
+    private String typeName;
+
+    /** Java type class name. */
+    private String typeClass;
+
+    /** Mode. */
+    private int mode;
+
+
+    /**
+     * Default constructor is used for binary serialization.
+     */
+    public JdbcParameterMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param meta Param metadata.
+     * @param order Param order.
+     * @throws SQLException On errror.
+     */
+    public JdbcParameterMeta(ParameterMetaData meta, int order) throws SQLException {
+        isNullable = meta.isNullable(order);
+        signed = meta.isSigned(order);
+        precision = meta.getPrecision(order);
+        scale = meta.getScale(order);
+        type = meta.getParameterType(order);
+        typeName = meta.getParameterTypeName(order);
+        typeClass = meta.getParameterClassName(order);
+        mode = meta.getParameterMode(order);
+    }
+
+    /**
+     * @return Nullable mode.
+     */
+    public int isNullable() {
+        return isNullable;
+    }
+
+    /**
+     * @return Signed flag.
+     */
+    public boolean isSigned() {
+        return signed;
+    }
+
+    /**
+     * @return Precision.
+     */
+    public int precision() {
+        return precision;
+    }
+
+    /**
+     * @return Scale.
+     */
+    public int scale() {
+        return scale;
+    }
+
+    /**
+     * @return SQL type.
+     */
+    public int type() {
+        return type;
+    }
+
+    /**
+     * @return SQL type name.
+     */
+    public String typeName() {
+        return typeName;
+    }
+
+    /**
+     * @return Java type class name.
+     */
+    public String typeClass() {
+        return typeClass;
+    }
+
+    /**
+     * @return Mode.
+     */
+    public int mode() {
+        return mode;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeInt(isNullable);
+        writer.writeBoolean(signed);
+        writer.writeInt(precision);
+        writer.writeInt(scale);
+        writer.writeInt(type);
+        writer.writeString(typeName);
+        writer.writeString(typeClass);
+        writer.writeInt(mode);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        isNullable = reader.readInt();
+        signed = reader.readBoolean();
+        precision = reader.readInt();
+        scale = reader.readInt();
+        type = reader.readInt();
+        typeName = reader.readString();
+        typeClass = reader.readString();
+        mode = reader.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcParameterMeta.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java
new file mode 100644
index 0000000..6b9bf70
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java
@@ -0,0 +1,131 @@
+/*
+ * 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.jdbc;
+
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * JDBC primary key metadata.
+ */
+public class JdbcPrimaryKeyMeta implements JdbcRawBinarylizable {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Primary key name. */
+    private String name;
+
+    /** Primary key fields. */
+    private List<String> fields;
+
+    /**
+     * Default constructor is used for binary serialization.
+     */
+    JdbcPrimaryKeyMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param name Name.
+     * @param fields Primary key fields.
+     */
+    JdbcPrimaryKeyMeta(String schemaName, String tblName, String name, List<String> fields) {
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+        this.name = name;
+        this.fields = fields;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Primary key name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return Key fields.
+     */
+    public List<String> fields() {
+        return fields;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+        writer.writeString(name);
+
+        JdbcUtils.writeStringCollection(writer, fields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        schemaName = reader.readString();
+        tblName = reader.readString();
+        name = reader.readString();
+
+        fields = JdbcUtils.readStringList(reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        JdbcPrimaryKeyMeta meta = (JdbcPrimaryKeyMeta)o;
+
+        return F.eq(schemaName, meta.schemaName) && F.eq(tblName, meta.tblName) && F.eq(name, meta.name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName != null ? schemaName.hashCode() : 0;
+
+        result = 31 * result + tblName.hashCode();
+        result = 31 * result + name.hashCode();
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
index 411d1e0..872889c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query close request.
+ * JDBC query close request.
  */
 public class JdbcQueryCloseRequest extends JdbcRequest {
     /** Query ID. */
@@ -31,7 +31,7 @@ public class JdbcQueryCloseRequest extends JdbcRequest {
 
     /**
      */
-    public JdbcQueryCloseRequest() {
+    JdbcQueryCloseRequest() {
         super(QRY_CLOSE);
     }
 


[3/3] ignite git commit: IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.

Posted by vo...@apache.org.
IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.


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

Branch: refs/heads/master
Commit: 0e8031444b2f0d68fd3fb5a9ba03ca4d6a0c4e2d
Parents: 37e58ba
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Aug 17 13:41:37 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 13:41:37 2017 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  337 +++-
 .../internal/jdbc/thin/JdbcThinConnection.java  |   20 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     | 1587 ++++++++++++++++++
 .../jdbc/thin/JdbcThinParameterMetadata.java    |  115 ++
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   27 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |   47 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |    4 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |  117 +-
 .../processors/odbc/SqlListenerNioListener.java |    9 +-
 .../odbc/SqlListenerRequestHandler.java         |    9 +
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |   20 +-
 .../odbc/jdbc/JdbcBatchExecuteResult.java       |    6 +
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |   75 +-
 .../processors/odbc/jdbc/JdbcIndexMeta.java     |  192 +++
 .../odbc/jdbc/JdbcMetaColumnsRequest.java       |  102 ++
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |   99 ++
 .../odbc/jdbc/JdbcMetaIndexesRequest.java       |   88 +
 .../odbc/jdbc/JdbcMetaIndexesResult.java        |   98 ++
 .../odbc/jdbc/JdbcMetaParamsRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaParamsResult.java         |   97 ++
 .../odbc/jdbc/JdbcMetaPrimaryKeysRequest.java   |   88 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysResult.java    |   99 ++
 .../odbc/jdbc/JdbcMetaSchemasRequest.java       |   73 +
 .../odbc/jdbc/JdbcMetaSchemasResult.java        |   73 +
 .../odbc/jdbc/JdbcMetaTablesRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaTablesResult.java         |   97 ++
 .../processors/odbc/jdbc/JdbcParameterMeta.java |  165 ++
 .../odbc/jdbc/JdbcPrimaryKeyMeta.java           |  131 ++
 .../odbc/jdbc/JdbcQueryCloseRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryExecuteRequest.java      |    8 +-
 .../odbc/jdbc/JdbcQueryExecuteResult.java       |   12 +-
 .../odbc/jdbc/JdbcQueryFetchRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryFetchResult.java         |   12 +-
 .../odbc/jdbc/JdbcQueryMetadataRequest.java     |   18 +-
 .../odbc/jdbc/JdbcQueryMetadataResult.java      |   14 +-
 .../processors/odbc/jdbc/JdbcRequest.java       |   67 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |  273 ++-
 .../processors/odbc/jdbc/JdbcResult.java        |   58 +-
 .../processors/odbc/jdbc/JdbcTableMeta.java     |   82 +
 .../processors/odbc/jdbc/JdbcUtils.java         |   37 +-
 .../odbc/odbc/OdbcRequestHandler.java           |    6 +
 .../processors/query/GridQueryProcessor.java    |    2 +-
 .../query/GridQueryTypeDescriptor.java          |    7 +
 .../query/QueryTypeDescriptorImpl.java          |   15 +
 .../internal/processors/query/QueryUtils.java   |    7 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   18 +-
 46 files changed, 4472 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 2dae107..01b2e8a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -21,16 +21,27 @@ import java.io.Serializable;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.Statement;
+import java.sql.Types;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Set;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.affinity.AffinityKey;
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -66,15 +77,18 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * @param qryEntity Query entity.
      * @return Cache configuration.
      */
-    protected CacheConfiguration cacheConfiguration() {
+    protected CacheConfiguration cacheConfiguration(QueryEntity qryEntity) {
         CacheConfiguration<?,?> cache = defaultCacheConfiguration();
 
         cache.setCacheMode(PARTITIONED);
         cache.setBackups(1);
         cache.setWriteSynchronizationMode(FULL_SYNC);
 
+        cache.setQueryEntities(Collections.singletonList(qryEntity));
+
         return cache;
     }
 
@@ -84,22 +98,49 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
 
         startGridsMultiThreaded(3);
 
-        IgniteCache<String, Organization> orgCache = jcache(grid(0), cacheConfiguration(), "org",
-            String.class, Organization.class);
+        IgniteCache<String, Organization> orgCache = jcache(grid(0),
+            cacheConfiguration(new QueryEntity(String.class.getName(), Organization.class.getName())
+                .addQueryField("id", Integer.class.getName(), null)
+                .addQueryField("name", String.class.getName(), null)
+                .setIndexes(Arrays.asList(
+                    new QueryIndex("id"),
+                    new QueryIndex("name", false, "org_name_index")
+                ))), "org");
 
         assert orgCache != null;
 
         orgCache.put("o1", new Organization(1, "A"));
         orgCache.put("o2", new Organization(2, "B"));
 
-        IgniteCache<AffinityKey, Person> personCache = jcache(grid(0), cacheConfiguration(), "pers",
-            AffinityKey.class, Person.class);
+        LinkedHashMap<String, Boolean> persFields = new LinkedHashMap<>();
+
+        persFields.put("name", true);
+        persFields.put("age", false);
+
+        IgniteCache<AffinityKey, Person> personCache = jcache(grid(0), cacheConfiguration(
+            new QueryEntity(AffinityKey.class.getName(), Person.class.getName())
+                .addQueryField("name", String.class.getName(), null)
+                .addQueryField("age", Integer.class.getName(), null)
+                .addQueryField("orgId", Integer.class.getName(), null)
+                .setIndexes(Arrays.asList(
+                    new QueryIndex("orgId"),
+                    new QueryIndex().setFields(persFields)))
+            ), "pers");
 
         assert personCache != null;
 
         personCache.put(new AffinityKey<>("p1", "o1"), new Person("John White", 25, 1));
         personCache.put(new AffinityKey<>("p2", "o1"), new Person("Joe Black", 35, 1));
         personCache.put(new AffinityKey<>("p3", "o2"), new Person("Mike Green", 40, 2));
+
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE TEST (ID INT, NAME VARCHAR(50), VAL VARCHAR(50), PRIMARY KEY (ID, NAME))");
+            stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50))");
+            stmt.execute("CREATE INDEX \"MyTestIndex quoted\" on \"Quoted\" (\"Id\" DESC)");
+            stmt.execute("CREATE INDEX IDX ON TEST (ID ASC)");
+        }
     }
 
     /** {@inheritDoc} */
@@ -147,8 +188,6 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testGetTables() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5233");
-
         try (Connection conn = DriverManager.getConnection(URL)) {
             DatabaseMetaData meta = conn.getMetaData();
 
@@ -184,15 +223,43 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
+    public void testGetAllTables() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getTables(null, null, null, null);
+
+            Set<String> expectedTbls = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION",
+                "pers.PERSON",
+                "PUBLIC.TEST",
+                "PUBLIC.Quoted"));
+
+            Set<String> actualTbls = new HashSet<>(expectedTbls.size());
+
+            while(rs.next()) {
+                actualTbls.add(rs.getString("TABLE_SCHEM") + '.'
+                    + rs.getString("TABLE_NAME"));
+            }
+
+            assert expectedTbls.equals(actualTbls) : "expectedTbls=" + expectedTbls +
+                ", actualTbls" + actualTbls;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetColumns() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5233");
+        final boolean primitivesInformationIsLostAfterStore = ignite(0).configuration().getMarshaller()
+            instanceof BinaryMarshaller;
 
         try (Connection conn = DriverManager.getConnection(URL)) {
             conn.setSchema("pers");
 
             DatabaseMetaData meta = conn.getMetaData();
 
-            ResultSet rs = meta.getColumns("", "pers", "Person", "%");
+            ResultSet rs = meta.getColumns("", "pers", "PERSON", "%");
 
             assert rs != null;
 
@@ -216,7 +283,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 } else if ("AGE".equals(name) || "ORGID".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == INTEGER;
                     assert "INTEGER".equals(rs.getString("TYPE_NAME"));
-                    assert rs.getInt("NULLABLE") == 0;
+                    assertEquals(primitivesInformationIsLostAfterStore ? 1 : 0, rs.getInt("NULLABLE"));
                 }
                 if ("_KEY".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == OTHER;
@@ -235,7 +302,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
             assert names.isEmpty();
             assert cnt == 3;
 
-            rs = meta.getColumns("", "org", "Organization", "%");
+            rs = meta.getColumns("", "org", "ORGANIZATION", "%");
 
             assert rs != null;
 
@@ -280,22 +347,243 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
-    public void testMetadataResultSetClose() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5233");
+    public void testGetAllColumns() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getColumns(null, null, null, null);
+
+            Set<String> expectedCols = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION.ID",
+                "org.ORGANIZATION.NAME",
+                "pers.PERSON.ORGID",
+                "pers.PERSON.AGE",
+                "pers.PERSON.NAME",
+                "PUBLIC.TEST.ID",
+                "PUBLIC.TEST.NAME",
+                "PUBLIC.TEST.VAL",
+                "PUBLIC.Quoted.Id",
+                "PUBLIC.Quoted.Name"));
+
+            Set<String> actualCols = new HashSet<>(expectedCols.size());
+
+            while(rs.next()) {
+                actualCols.add(rs.getString("TABLE_SCHEM") + '.'
+                    + rs.getString("TABLE_NAME") + "."
+                    + rs.getString("COLUMN_NAME"));
+            }
+
+            assert expectedCols.equals(actualCols) : "expectedCols=" + expectedCols +
+                ", actualCols" + actualCols;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvalidCatalog() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getSchemas("q", null);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getTables("q", null, null, null);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getColumns("q", null, null, null);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getIndexInfo("q", null, null, false, false);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getPrimaryKeys("q", null, null);
+
+            assert !rs.next() : "Results must be empty";
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIndexMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL);
+             ResultSet rs = conn.getMetaData().getIndexInfo(null, "pers", "PERSON", false, false)) {
+
+            int cnt = 0;
+
+            while (rs.next()) {
+                String idxName = rs.getString("INDEX_NAME");
+                String field = rs.getString("COLUMN_NAME");
+                String ascOrDesc = rs.getString("ASC_OR_DESC");
+
+                assert rs.getShort("TYPE") == DatabaseMetaData.tableIndexOther;
+
+                if ("PERSON_ORGID_ASC_IDX".equals(idxName)) {
+                    assert "ORGID".equals(field);
+                    assert "A".equals(ascOrDesc);
+                }
+                else if ("PERSON_NAME_ASC_AGE_DESC_IDX".equals(idxName)) {
+                    if ("NAME".equals(field))
+                        assert "A".equals(ascOrDesc);
+                    else if ("AGE".equals(field))
+                        assert "D".equals(ascOrDesc);
+                    else
+                        fail("Unexpected field: " + field);
+                }
+                else
+                    fail("Unexpected index: " + idxName);
+
+                cnt++;
+            }
+
+            assert cnt == 3;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllIndexes() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getIndexInfo(null, null, null, false, false);
+
+            Set<String> expectedIdxs = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION.ORGANIZATION_ID_ASC_IDX",
+                "org.ORGANIZATION.ORG_NAME_INDEX",
+                "pers.PERSON.PERSON_ORGID_ASC_IDX",
+                "pers.PERSON.PERSON_NAME_ASC_AGE_DESC_IDX",
+                "PUBLIC.TEST.IDX",
+                "PUBLIC.Quoted.MyTestIndex quoted"));
+
+            Set<String> actualIdxs = new HashSet<>(expectedIdxs.size());
+
+            while(rs.next()) {
+                actualIdxs.add(rs.getString("TABLE_SCHEM") +
+                    '.' + rs.getString("TABLE_NAME") +
+                    '.' + rs.getString("INDEX_NAME"));
+            }
+
+            assert expectedIdxs.equals(actualIdxs) : "expectedIdxs=" + expectedIdxs +
+                ", actualIdxs" + actualIdxs;
+        }
+    }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryKeyMetadata() throws Exception {
         try (Connection conn = DriverManager.getConnection(URL);
-             ResultSet tbls = conn.getMetaData().getTables(null, null, "%", null)) {
-            int colCnt = tbls.getMetaData().getColumnCount();
+             ResultSet rs = conn.getMetaData().getPrimaryKeys(null, "pers", "PERSON")) {
+
+            int cnt = 0;
+
+            while (rs.next()) {
+                assert "_KEY".equals(rs.getString("COLUMN_NAME"));
+
+                cnt++;
+            }
+
+            assert cnt == 1;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllPrimaryKeys() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, null);
+
+            Set<String> expectedPks = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION.PK_org_ORGANIZATION._KEY",
+                "pers.PERSON.PK_pers_PERSON._KEY",
+                "PUBLIC.TEST.PK_PUBLIC_TEST.ID",
+                "PUBLIC.TEST.PK_PUBLIC_TEST.NAME",
+                "PUBLIC.Quoted.PK_PUBLIC_Quoted.Id"));
+
+            Set<String> actualPks = new HashSet<>(expectedPks.size());
+
+            while(rs.next()) {
+                actualPks.add(rs.getString("TABLE_SCHEM") +
+                    '.' + rs.getString("TABLE_NAME") +
+                    '.' + rs.getString("PK_NAME") +
+                    '.' + rs.getString("COLUMN_NAME"));
+            }
+
+            assert expectedPks.equals(actualPks) : "expectedPks=" + expectedPks +
+                ", actualPks" + actualPks;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testParametersMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.setSchema("pers");
+
+            PreparedStatement stmt = conn.prepareStatement("select orgId from Person p where p.name > ? and p.orgId > ?");
+
+            ParameterMetaData meta = stmt.getParameterMetaData();
+
+            assert meta != null;
+
+            assert meta.getParameterCount() == 2;
+
+            assert meta.getParameterType(1) == Types.VARCHAR;
+            assert meta.isNullable(1) == ParameterMetaData.parameterNullableUnknown;
+            assert meta.getPrecision(1) == Integer.MAX_VALUE;
+
+            assert meta.getParameterType(2) == Types.INTEGER;
+            assert meta.isNullable(2) == ParameterMetaData.parameterNullableUnknown;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSchemasMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getSchemas();
+
+            Set<String> expectedSchemas = new HashSet<>(Arrays.asList("PUBLIC", "pers", "org"));
 
-            while (tbls.next()) {
-                for (int i = 0; i < colCnt; i++)
-                    tbls.getObject(i + 1);
+            Set<String> schemas = new HashSet<>();
+
+            while (rs.next()) {
+                schemas.add(rs.getString(1));
+
+                assert rs.getString(2) == null;
             }
+
+            assert expectedSchemas.equals(schemas) : "Unexpected schemas: " + schemas +
+                ". Expected schemas: " + expectedSchemas;
         }
-        catch (Exception e) {
-            log.error("Unexpected exception", e);
+    }
 
-            fail();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEmptySchemasMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getSchemas(null, "qqq");
+
+            assert !rs.next() : "Empty result set is expected";
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testVersions() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            assert conn.getMetaData().getDatabaseProductVersion().equals(IgniteVersionUtils.VER.toString());
+            assert conn.getMetaData().getDriverVersion().equals(IgniteVersionUtils.VER.toString());
         }
     }
 
@@ -305,15 +593,12 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     @SuppressWarnings("UnusedDeclaration")
     private static class Person implements Serializable {
         /** Name. */
-        @QuerySqlField(index = false)
         private final String name;
 
         /** Age. */
-        @QuerySqlField
         private final int age;
 
         /** Organization ID. */
-        @QuerySqlField
         private final int orgId;
 
         /**
@@ -338,11 +623,9 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     @SuppressWarnings("UnusedDeclaration")
     private static class Organization implements Serializable {
         /** ID. */
-        @QuerySqlField
         private final int id;
 
         /** Name. */
-        @QuerySqlField(index = false)
         private final String name;
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index 89ef2fc..8836cd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -64,6 +64,9 @@ public class JdbcThinConnection implements Connection {
     /** Logger. */
     private static final Logger LOG = Logger.getLogger(JdbcThinConnection.class.getName());
 
+    /** Connection URL. */
+    private String url;
+
     /** Schema name. */
     private String schemaName;
 
@@ -88,6 +91,9 @@ public class JdbcThinConnection implements Connection {
     /** Ignite endpoint. */
     private JdbcThinTcpIo cliIo;
 
+    /** Jdbc metadata. Cache the JDBC object on the first access */
+    private JdbcThinDatabaseMetadata metadata;
+
     /**
      * Creates new connection.
      *
@@ -99,6 +105,8 @@ public class JdbcThinConnection implements Connection {
         assert url != null;
         assert props != null;
 
+        this.url = url;
+
         holdability = HOLD_CURSORS_OVER_COMMIT;
         autoCommit = true;
         txIsolation = Connection.TRANSACTION_NONE;
@@ -274,7 +282,10 @@ public class JdbcThinConnection implements Connection {
     @Override public DatabaseMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
-        return null;
+        if (metadata == null)
+            metadata = new JdbcThinDatabaseMetadata(this);
+
+        return metadata;
     }
 
     /** {@inheritDoc} */
@@ -665,4 +676,11 @@ public class JdbcThinConnection implements Connection {
                 ", value=" + strVal + ']');
         }
     }
+
+    /**
+     * @return Connection URL.
+     */
+    public String url() {
+        return url;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
new file mode 100644
index 0000000..583bcec
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
@@ -0,0 +1,1587 @@
+/*
+ * 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.jdbc.thin;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.RowIdLifetime;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.jdbc2.JdbcUtils;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcColumnMeta;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcIndexMeta;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaIndexesResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaPrimaryKeysResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaSchemasResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaTablesResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcPrimaryKeyMeta;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcTableMeta;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static java.sql.Connection.TRANSACTION_NONE;
+import static java.sql.ResultSet.CONCUR_READ_ONLY;
+import static java.sql.ResultSet.HOLD_CURSORS_OVER_COMMIT;
+import static java.sql.ResultSet.TYPE_FORWARD_ONLY;
+import static java.sql.RowIdLifetime.ROWID_UNSUPPORTED;
+
+/**
+ * JDBC database metadata implementation.
+ */
+@SuppressWarnings("RedundantCast")
+public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
+    /** Connection. */
+    private final JdbcThinConnection conn;
+
+    /**
+     * @param conn Connection.
+     */
+    JdbcThinDatabaseMetadata(JdbcThinConnection conn) {
+        this.conn = conn;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean allProceduresAreCallable() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean allTablesAreSelectable() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getURL() throws SQLException {
+        return conn.url();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getUserName() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isReadOnly() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedHigh() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedLow() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedAtStart() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedAtEnd() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDatabaseProductName() throws SQLException {
+        return "Ignite";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDatabaseProductVersion() throws SQLException {
+        return conn.io().igniteVersion().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDriverName() throws SQLException {
+        return "Ignite JDBC Thin Driver";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDriverVersion() throws SQLException {
+        return IgniteVersionUtils.VER.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDriverMajorVersion() {
+        return IgniteVersionUtils.VER.major();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDriverMinorVersion() {
+        return IgniteVersionUtils.VER.minor();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean usesLocalFiles() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean usesLocalFilePerTable() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMixedCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesUpperCaseIdentifiers() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesLowerCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesMixedCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getIdentifierQuoteString() throws SQLException {
+        return "\"";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQLKeywords() throws SQLException {
+        return "LIMIT,MINUS,ROWNUM,SYSDATE,SYSTIME,SYSTIMESTAMP,TODAY";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getNumericFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getStringFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSystemFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTimeDateFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSearchStringEscape() throws SQLException {
+        return "\\";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getExtraNameCharacters() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsAlterTableWithAddColumn() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsAlterTableWithDropColumn() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsColumnAliasing() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullPlusNonNullIsNull() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsConvert() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsConvert(int fromType, int toType) throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsTableCorrelationNames() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsDifferentTableCorrelationNames() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsExpressionsInOrderBy() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOrderByUnrelated() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGroupBy() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGroupByUnrelated() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGroupByBeyondSelect() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsLikeEscapeClause() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMultipleResultSets() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMultipleTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsNonNullableColumns() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMinimumSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCoreSQLGrammar() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsExtendedSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsANSI92EntryLevelSQL() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsANSI92IntermediateSQL() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsANSI92FullSQL() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsIntegrityEnhancementFacility() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOuterJoins() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsFullOuterJoins() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsLimitedOuterJoins() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSchemaTerm() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getProcedureTerm() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCatalogTerm() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isCatalogAtStart() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCatalogSeparator() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInDataManipulation() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInProcedureCalls() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInTableDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInIndexDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInDataManipulation() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInProcedureCalls() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInTableDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsPositionedDelete() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsPositionedUpdate() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSelectForUpdate() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsStoredProcedures() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInComparisons() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInExists() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInIns() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInQuantifieds() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCorrelatedSubqueries() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsUnion() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsUnionAll() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxBinaryLiteralLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxCharLiteralLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInGroupBy() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInIndex() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInOrderBy() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInSelect() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInTable() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxConnections() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxCursorNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxIndexLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxSchemaNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxProcedureNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxCatalogNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxRowSize() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxStatementLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxStatements() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxTableNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxTablesInSelect() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxUserNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDefaultTransactionIsolation() throws SQLException {
+        return TRANSACTION_NONE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsTransactionIsolationLevel(int level) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsDataDefinitionAndDataManipulationTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsDataManipulationTransactionsOnly() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getProcedures(String catalog, String schemaPtrn,
+        String procedureNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PROCEDURE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_TYPE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getProcedureColumns(String catalog, String schemaPtrn, String procedureNamePtrn,
+        String colNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PROCEDURE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PRECISION", Integer.class),
+            new JdbcColumnMeta(null, null, "LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "RADIX", Short.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_DEF", String.class),
+            new JdbcColumnMeta(null, null, "SQL_DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "SQL_DATETIME_SUB", Integer.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+            ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getTables(String catalog, String schemaPtrn, String tblNamePtrn,
+        String[] tblTypes) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_TYPE", String.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "SELF_REFERENCING_COL_NAME", String.class),
+            new JdbcColumnMeta(null, null, "REF_GENERATION", String.class));
+
+        boolean tblTypeMatch = false;
+
+        if (tblTypes == null)
+            tblTypeMatch = true;
+        else {
+            for (String type : tblTypes) {
+                if ("TABLE".equals(type)) {
+                    tblTypeMatch = true;
+
+                    break;
+                }
+            }
+        }
+
+        if (!validCatalogPattern(catalog) || !tblTypeMatch)
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaTablesResult res = conn.io().tablesMeta(schemaPtrn, tblNamePtrn);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (JdbcTableMeta tblMeta : res.meta())
+                rows.add(tableRow(tblMeta));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param tblMeta Table metadata.
+     * @return Table metadata row.
+     */
+    private List<Object> tableRow(JdbcTableMeta tblMeta) {
+        List<Object> row = new ArrayList<>(10);
+
+        row.add(null);
+        row.add(tblMeta.schemaName());
+        row.add(tblMeta.tableName());
+        row.add("TABLE");
+        row.add(null);
+        row.add(null);
+        row.add(null);
+        row.add(null);
+        row.add(null);
+        row.add(null);
+
+        return row;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSchemas() throws SQLException {
+        return getSchemas(null, "%");
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ArraysAsListWithZeroOrOneArgument")
+    @Override public ResultSet getCatalogs() throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(),
+            Arrays.asList(new JdbcColumnMeta(null, null, "TABLE_CAT", String.class)));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ArraysAsListWithZeroOrOneArgument")
+    @Override public ResultSet getTableTypes() throws SQLException {
+        return new JdbcThinResultSet(Collections.singletonList(Collections.<Object>singletonList("TABLE")),
+            Arrays.asList(new JdbcColumnMeta(null, null, "TABLE_TYPE", String.class)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getColumns(String catalog, String schemaPtrn, String tblNamePtrn,
+        String colNamePtrn) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Short.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_DEF", String.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_CATLOG", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_SCHEMA", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_TABLE", String.class),
+            new JdbcColumnMeta(null, null, "SOURCE_DATA_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "IS_AUTOINCREMENT", String.class));
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaColumnsResult res = conn.io().columnsMeta(schemaPtrn, tblNamePtrn, colNamePtrn);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (int i = 0; i < res.meta().size(); ++i)
+                rows.add(columnRow(res.meta().get(i), i + 1));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param colMeta Column metadata.
+     * @param pos Ordinal position.
+     * @return Column metadata row.
+     */
+    private List<Object> columnRow(JdbcColumnMeta colMeta, int pos) {
+        List<Object> row = new ArrayList<>(20);
+
+        row.add((String)null);
+        row.add(colMeta.schemaName());
+        row.add(colMeta.tableName());
+        row.add(colMeta.columnName());
+        row.add(colMeta.dataType());
+        row.add(colMeta.dataTypeName());
+        row.add((Integer)null);
+        row.add((Integer)null);
+        row.add(10);
+        row.add(JdbcUtils.nullable(colMeta.columnName(), colMeta.dataTypeClass()) ? 1 : 0 );
+        row.add((String)null);
+        row.add((String)null);
+        row.add(Integer.MAX_VALUE);
+        row.add(pos);
+        row.add("YES");
+        row.add((String)null);
+        row.add((String)null);
+        row.add((String)null);
+        row.add((Short)null);
+        row.add("NO");
+
+        return row;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getColumnPrivileges(String catalog, String schema, String tbl,
+        String colNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "GRANTOR", String.class),
+            new JdbcColumnMeta(null, null, "GRANTEE", String.class),
+            new JdbcColumnMeta(null, null, "PRIVILEGE", String.class),
+            new JdbcColumnMeta(null, null, "IS_GRANTABLE", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getTablePrivileges(String catalog, String schemaPtrn,
+        String tblNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "GRANTOR", String.class),
+            new JdbcColumnMeta(null, null, "GRANTEE", String.class),
+            new JdbcColumnMeta(null, null, "PRIVILEGE", String.class),
+            new JdbcColumnMeta(null, null, "IS_GRANTABLE", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getBestRowIdentifier(String catalog, String schema, String tbl, int scope,
+        boolean nullable) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "SCOPE", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "BUFFER_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Short.class),
+            new JdbcColumnMeta(null, null, "PSEUDO_COLUMN", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getVersionColumns(String catalog, String schema, String tbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "SCOPE", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "BUFFER_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Short.class),
+            new JdbcColumnMeta(null, null, "PSEUDO_COLUMN", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getPrimaryKeys(String catalog, String schema, String tbl) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class));
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaPrimaryKeysResult res = conn.io().primaryKeysMeta(schema, tbl);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (JdbcPrimaryKeyMeta pkMeta : res.meta())
+                rows.addAll(primaryKeyRows(pkMeta));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param pkMeta Primary key metadata.
+     * @return Result set rows for primary key.
+     */
+    private List<List<Object>> primaryKeyRows(JdbcPrimaryKeyMeta pkMeta) {
+        List<List<Object>> rows = new ArrayList<>(pkMeta.fields().size());
+
+        for (int i = 0; i < pkMeta.fields().size(); ++i) {
+            List<Object> row = new ArrayList<>(6);
+
+            row.add((String)null); // table catalog
+            row.add(pkMeta.schemaName());
+            row.add(pkMeta.tableName());
+            row.add(pkMeta.fields().get(i));
+            row.add((Integer)i + 1); // sequence number
+            row.add(pkMeta.name());
+
+            rows.add(row);
+        }
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getImportedKeys(String catalog, String schema, String tbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "UPDATE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "DELETE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "FK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DEFERRABILITY", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getExportedKeys(String catalog, String schema, String tbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "UPDATE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "DELETE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "FK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DEFERRABILITY", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTbl,
+        String foreignCatalog, String foreignSchema, String foreignTbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "UPDATE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "DELETE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "FK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DEFERRABILITY", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getTypeInfo() throws SQLException {
+        List<List<Object>> types = new ArrayList<>(21);
+
+        types.add(Arrays.<Object>asList("BOOLEAN", Types.BOOLEAN, 1, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "BOOLEAN", 0, 0,
+            Types.BOOLEAN, 0, 10));
+
+        types.add(Arrays.<Object>asList("TINYINT", Types.TINYINT, 3, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "TINYINT", 0, 0,
+            Types.TINYINT, 0, 10));
+
+        types.add(Arrays.<Object>asList("SMALLINT", Types.SMALLINT, 5, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "SMALLINT", 0, 0,
+            Types.SMALLINT, 0, 10));
+
+        types.add(Arrays.<Object>asList("INTEGER", Types.INTEGER, 10, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "INTEGER", 0, 0,
+            Types.INTEGER, 0, 10));
+
+        types.add(Arrays.<Object>asList("BIGINT", Types.BIGINT, 19, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "BIGINT", 0, 0,
+            Types.BIGINT, 0, 10));
+
+        types.add(Arrays.<Object>asList("FLOAT", Types.FLOAT, 17, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "FLOAT", 0, 0,
+            Types.FLOAT, 0, 10));
+
+        types.add(Arrays.<Object>asList("REAL", Types.REAL, 7, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "REAL", 0, 0,
+            Types.REAL, 0, 10));
+
+        types.add(Arrays.<Object>asList("DOUBLE", Types.DOUBLE, 17, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "DOUBLE", 0, 0,
+            Types.DOUBLE, 0, 10));
+
+        types.add(Arrays.<Object>asList("NUMERIC", Types.NUMERIC, Integer.MAX_VALUE, null, null, "PRECISION,SCALE",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "NUMERIC", 0, 0,
+            Types.NUMERIC, 0, 10));
+
+        types.add(Arrays.<Object>asList("DECIMAL", Types.DECIMAL, Integer.MAX_VALUE, null, null, "PRECISION,SCALE",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "DECIMAL", 0, 0,
+            Types.DECIMAL, 0, 10));
+
+        types.add(Arrays.<Object>asList("DATE", Types.DATE, 8, "DATE '", "'", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "DATE", 0, 0,
+            Types.DATE, 0, null));
+
+        types.add(Arrays.<Object>asList("TIME", Types.TIME, 6, "TIME '", "'", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "TIME", 0, 0,
+            Types.TIME, 0, null));
+
+        types.add(Arrays.<Object>asList("TIMESTAMP", Types.TIMESTAMP, 23, "TIMESTAMP '", "'", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "TIMESTAMP", 0, 10,
+            Types.TIMESTAMP, 0, null));
+
+        types.add(Arrays.<Object>asList("CHAR", Types.CHAR, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, true, (short)typeSearchable, false, false, false, "CHAR", 0, 0,
+            Types.CHAR, 0, null));
+
+        types.add(Arrays.<Object>asList("VARCHAR", Types.VARCHAR, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, true, (short)typeSearchable, false, false, false, "VARCHAR", 0, 0,
+            Types.VARCHAR, 0, null));
+
+        types.add(Arrays.<Object>asList("LONGVARCHAR", Types.LONGVARCHAR, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, true, (short)typeSearchable, false, false, false, "LONGVARCHAR", 0, 0,
+            Types.LONGVARCHAR, 0, null));
+
+        types.add(Arrays.<Object>asList("BINARY", Types.BINARY, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "BINARY", 0, 0,
+            Types.BINARY, 0, null));
+
+        types.add(Arrays.<Object>asList("VARBINARY", Types.VARBINARY, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "VARBINARY", 0, 0,
+            Types.VARBINARY, 0, null));
+
+        types.add(Arrays.<Object>asList("LONGVARBINARY", Types.LONGVARBINARY, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "LONGVARBINARY", 0, 0,
+            Types.LONGVARBINARY, 0, null));
+
+        types.add(Arrays.<Object>asList("OTHER", Types.OTHER, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "OTHER", 0, 0,
+            Types.OTHER, 0, null));
+
+        types.add(Arrays.<Object>asList("ARRAY", Types.ARRAY, 0, "(", "')", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "ARRAY", 0, 0,
+            Types.ARRAY, 0, null));
+
+        return new JdbcThinResultSet(types, Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "PRECISION", Integer.class),
+            new JdbcColumnMeta(null, null, "LITERAL_PREFIX", String.class),
+            new JdbcColumnMeta(null, null, "LITERAL_SUFFIX", String.class),
+            new JdbcColumnMeta(null, null, "CREATE_PARAMS", String.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "CASE_SENSITIVE", Boolean.class),
+            new JdbcColumnMeta(null, null, "SEARCHABLE", Short.class),
+            new JdbcColumnMeta(null, null, "UNSIGNED_ATTRIBUTE", Boolean.class),
+            new JdbcColumnMeta(null, null, "FIXED_PREC_SCALE", Boolean.class),
+            new JdbcColumnMeta(null, null, "AUTO_INCREMENT", Boolean.class),
+            new JdbcColumnMeta(null, null, "LOCAL_TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "MINIMUM_SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "MAXIMUM_SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "SQL_DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "SQL_DATETIME_SUB", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Integer.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getIndexInfo(String catalog, String schema, String tbl, boolean unique,
+        boolean approximate) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "NON_UNIQUE", Boolean.class),
+            new JdbcColumnMeta(null, null, "INDEX_QUALIFIER", String.class),
+            new JdbcColumnMeta(null, null, "INDEX_NAME", String.class),
+            new JdbcColumnMeta(null, null, "TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "ASC_OR_DESC", String.class),
+            new JdbcColumnMeta(null, null, "CARDINALITY", Integer.class),
+            new JdbcColumnMeta(null, null, "PAGES", Integer.class),
+            new JdbcColumnMeta(null, null, "FILTER_CONDITION", String.class));
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaIndexesResult res = conn.io().indexMeta(schema, tbl);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (JdbcIndexMeta idxMeta : res.meta())
+                rows.addAll(indexRows(idxMeta));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param idxMeta Index metadata.
+     * @return List of result rows correspond to index.
+     */
+    private List<List<Object>> indexRows(JdbcIndexMeta idxMeta) {
+        List<List<Object>> rows = new ArrayList<>(idxMeta.fields().size());
+
+        for (int i = 0; i < idxMeta.fields().size(); ++i) {
+            List<Object> row = new ArrayList<>(13);
+
+            row.add((String)null); // table catalog
+            row.add(idxMeta.schemaName());
+            row.add(idxMeta.tableName());
+            row.add(true); // non unique
+            row.add(null); // index qualifier (index catalog)
+            row.add(idxMeta.indexName());
+            row.add((short)tableIndexOther); // type
+            row.add((Integer)i); // field ordinal position in index
+            row.add(idxMeta.fields().get(i));
+            row.add(idxMeta.fieldsAsc().get(i) ? "A" : "D");
+            row.add((Integer)0); // cardinality
+            row.add((Integer)0); // pages
+            row.add((String)null); // filer condition
+
+            rows.add(row);
+        }
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsResultSetType(int type) throws SQLException {
+        return type == TYPE_FORWARD_ONLY;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsResultSetConcurrency(int type, int concurrency) throws SQLException {
+        return supportsResultSetType(type) && concurrency == CONCUR_READ_ONLY;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ownUpdatesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ownDeletesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ownInsertsAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean othersUpdatesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean othersDeletesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean othersInsertsAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean updatesAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean deletesAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean insertsAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsBatchUpdates() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getUDTs(String catalog, String schemaPtrn, String typeNamePtrn,
+        int[] types) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "CLASS_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "BASE_TYPE", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Connection getConnection() throws SQLException {
+        return conn;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSavepoints() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsNamedParameters() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMultipleOpenResults() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGetGeneratedKeys() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSuperTypes(String catalog, String schemaPtrn,
+        String typeNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTYPE_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSuperTables(String catalog, String schemaPtrn,
+        String tblNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTABLE_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getAttributes(String catalog, String schemaPtrn, String typeNamePtrn,
+        String attributeNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "ATTR_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "ATTR_TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "ATTR_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Integer.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Integer.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "ATTR_DEF", String.class),
+            new JdbcColumnMeta(null, null, "SQL_DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "SQL_DATETIME_SUB", Integer.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_CATALOG", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_SCHEMA", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_TABLE", String.class),
+            new JdbcColumnMeta(null, null, "SOURCE_DATA_TYPE", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsResultSetHoldability(int holdability) throws SQLException {
+        return holdability == HOLD_CURSORS_OVER_COMMIT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getResultSetHoldability() throws SQLException {
+        return HOLD_CURSORS_OVER_COMMIT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDatabaseMajorVersion() throws SQLException {
+        return conn.io().igniteVersion().major();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDatabaseMinorVersion() throws SQLException {
+        return conn.io().igniteVersion().minor();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getJDBCMajorVersion() throws SQLException {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getJDBCMinorVersion() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getSQLStateType() throws SQLException {
+        return DatabaseMetaData.sqlStateSQL99;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean locatorsUpdateCopy() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsStatementPooling() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowIdLifetime getRowIdLifetime() throws SQLException {
+        return ROWID_UNSUPPORTED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSchemas(String catalog, String schemaPtrn) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_CATALOG", String.class)
+        );
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaSchemasResult res = conn.io().schemasMeta(schemaPtrn);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (String schema : res.schemas()) {
+                List<Object> row = new ArrayList<>(2);
+
+                row.add(schema);
+                row.add(null);
+
+                rows.add(row);
+            }
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getClientInfoProperties() throws SQLException {
+        // TODO: IGNITE-5425.
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "NAME", String.class),
+            new JdbcColumnMeta(null, null, "MAX_LEN", Integer.class),
+            new JdbcColumnMeta(null, null, "DEFAULT_VALUE", String.class),
+            new JdbcColumnMeta(null, null, "DESCRIPTION", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getFunctions(String catalog, String schemaPtrn,
+        String functionNamePtrn) throws SQLException {
+        // TODO: IGNITE-6028
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "FUNCTION_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_NAME", String.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_TYPE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getFunctionColumns(String catalog, String schemaPtrn, String functionNamePtrn,
+        String colNamePtrn) throws SQLException {
+        // TODO: IGNITE-6028
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "FUNCTION_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PRECISION", Integer.class),
+            new JdbcColumnMeta(null, null, "LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "RADIX", Short.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("Database meta data is not a wrapper for " + iface.getName());
+
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface != null && iface.isAssignableFrom(JdbcThinDatabaseMetadata.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getPseudoColumns(String catalog, String schemaPtrn, String tblNamePtrn,
+        String colNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Integer.class),
+            new JdbcColumnMeta(null, null, "COLUMN_USAGE", Integer.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class)
+        ));
+    }
+
+    /**
+     * @param catalog Catalog pattern.
+     * @return {@code true} If patter is valid for Ignite (null, empty, or '%' wildcard).
+     *  Otherwise returns {@code false}.
+     */
+    private static boolean validCatalogPattern(String catalog) {
+        return F.isEmpty(catalog) || "%".equals(catalog);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean generatedKeyAlwaysReturned() throws SQLException {
+        return false;
+    }
+}
\ No newline at end of file