IGNITE-15187 JDBC driver for 3.0: ResultSet metadata (#345)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlCursor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlCursor.java
index b5bb641..31007b9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlCursor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlCursor.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.calcite;
 
+import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadata;
 import org.apache.ignite.internal.util.Cursor;
 
 /**
@@ -29,4 +30,9 @@
      * @return Query type.
      */
     SqlQueryType getQueryType();
+
+    /**
+     * @return Column metadata.
+     */
+    FieldsMetadata getColumnMetadata();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
index 6ebc28c..43d371f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
@@ -17,6 +17,8 @@
 package org.apache.ignite.internal.processors.query.calcite;
 
 import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -83,6 +85,12 @@
         if (type == String.class)
             return UUID.randomUUID().toString();
 
+        if (type == BigDecimal.class)
+            return BigDecimal.valueOf(ThreadLocalRandom.current().nextDouble());
+
+        if (type == BigInteger.class)
+            return BigDecimal.valueOf(ThreadLocalRandom.current().nextDouble()).unscaledValue();
+
         throw new IllegalStateException("Can't generate value of type " + type.getTypeName());
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index 032b3cd..c99eff6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -522,12 +522,12 @@
                 res.add(ectx.rowHandler().get(i, row));
 
             return res;
-        }), plan.type());
+        }), plan);
     }
 
     /** */
     private SqlCursor<List<?>> executeExplain(ExplainPlan plan) {
-        SqlCursor<List<?>> cur = Commons.createCursor(singletonList(singletonList(plan.plan())), plan.type());
+        SqlCursor<List<?>> cur = Commons.createCursor(singletonList(singletonList(plan.plan())), plan);
         // TODO: fix this
 //        cur.fieldsMeta(plan.fieldsMeta().queryFieldsMetadata(pctx.typeFactory()));
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
index b60a7a2..a1d363c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
@@ -65,6 +65,10 @@
 import org.apache.ignite.internal.processors.query.calcite.SqlQueryType;
 import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactoryImpl;
 import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCostFactory;
+import org.apache.ignite.internal.processors.query.calcite.prepare.AbstractMultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ExplainPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadata;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
 import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlan;
 import org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable;
@@ -134,14 +138,19 @@
     /** */
     private Commons(){}
 
-    public static <T> SqlCursor<T> createCursor(Iterable<T> iterable, QueryPlan.Type plan) {
+    public static <T> SqlCursor<T> createCursor(Iterable<T> iterable, QueryPlan plan) {
         return createCursor(iterable.iterator(), plan);
     }
 
-    public static <T> SqlCursor<T> createCursor(Iterator<T> iter, QueryPlan.Type type) {
+    public static <T> SqlCursor<T> createCursor(Iterator<T> iter, QueryPlan plan) {
         return new SqlCursor<>() {
             @Override public SqlQueryType getQueryType() {
-                return SqlQueryType.mapPlanTypeToSqlType(type);
+                return SqlQueryType.mapPlanTypeToSqlType(plan.type());
+            }
+
+            @Override public FieldsMetadata getColumnMetadata() {
+                return plan instanceof AbstractMultiStepPlan ? ((MultiStepPlan)plan).fieldsMetadata()
+                    : ((ExplainPlan)plan).fieldsMeta();
             }
 
             @Override public void remove() {
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/JdbcQueryEventHandler.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/JdbcQueryEventHandler.java
index f2f1750..c8f5540 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/JdbcQueryEventHandler.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/JdbcQueryEventHandler.java
@@ -27,6 +27,7 @@
 import org.apache.ignite.client.proto.query.event.JdbcMetaSchemasResult;
 import org.apache.ignite.client.proto.query.event.JdbcMetaTablesRequest;
 import org.apache.ignite.client.proto.query.event.JdbcMetaTablesResult;
+import org.apache.ignite.client.proto.query.event.JdbcQueryMetadataRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseResult;
 import org.apache.ignite.client.proto.query.event.QueryExecuteRequest;
@@ -101,4 +102,12 @@
      * @return Result.
      */
     JdbcMetaPrimaryKeysResult primaryKeysMeta(JdbcMetaPrimaryKeysRequest req);
+
+    /**
+     * {@link JdbcQueryMetadataRequest} command handler.
+     *
+     * @param req Jdbc query metadata request.
+     * @return Result.
+     */
+    JdbcMetaColumnsResult queryMetadata(JdbcQueryMetadataRequest req);
 }
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcColumnMeta.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcColumnMeta.java
index 54ef323..8e62626 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcColumnMeta.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcColumnMeta.java
@@ -70,6 +70,9 @@
     /** Scale. */
     private int scale;
 
+    /** Data type class. */
+    private String dataTypeCls;
+
     /**
      * Default constructor is used for serialization.
      */
@@ -101,13 +104,30 @@
      */
     public JdbcColumnMeta(String schemaName, String tblName, String colName, Class<?> cls, int precision, int scale,
         boolean nullable) {
+        this(schemaName, tblName, colName, cls.getName(), precision, scale, nullable);
+    }
+
+    /**
+     * Constructor with nullable flag.
+     *
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param colName Column.
+     * @param javaTypeName Java type name.
+     * @param nullable Nullable flag.
+     * @param precision Column precision.
+     * @param scale Column scale.
+     */
+    public JdbcColumnMeta(String schemaName, String tblName, String colName, String javaTypeName, int precision, int scale,
+        boolean nullable) {
         this.schemaName = schemaName;
         this.tblName = tblName;
         this.colName = colName;
         this.nullable = nullable;
 
-        this.dataType = type(cls.getName());
-        this.dataTypeName = typeName(cls.getName());
+        this.dataType = type(javaTypeName);
+        this.dataTypeName = typeName(javaTypeName);
+        this.dataTypeCls = javaTypeName;
         this.precision = precision;
         this.scale = scale;
 
@@ -195,6 +215,15 @@
         return nullable;
     }
 
+    /**
+     * Gets data type class.
+     *
+     * @return Data type class.
+     */
+    public String dataTypeClass() {
+        return dataTypeCls;
+    }
+
     /** {@inheritDoc} */
     @Override public void writeBinary(ClientMessagePacker packer) {
         super.writeBinary(packer);
@@ -202,12 +231,13 @@
         if (!hasResults)
             return;
 
-        packer.packString(schemaName);
-        packer.packString(tblName);
+        ClientMessageUtils.writeStringNullable(packer, schemaName);
+        ClientMessageUtils.writeStringNullable(packer, tblName);
         packer.packString(colName);
 
         packer.packInt(dataType);
         packer.packString(dataTypeName);
+        packer.packString(dataTypeCls);
         packer.packBoolean(nullable);
         packer.packInt(precision);
         packer.packInt(scale);
@@ -220,12 +250,13 @@
         if (!hasResults)
             return;
 
-        schemaName = unpacker.unpackString();
-        tblName = unpacker.unpackString();
+        schemaName = ClientMessageUtils.readStringNullable(unpacker);
+        tblName = ClientMessageUtils.readStringNullable(unpacker);
         colName = unpacker.unpackString();
 
         dataType = unpacker.unpackInt();
         dataTypeName = unpacker.unpackString();
+        dataTypeCls = unpacker.unpackString();
         nullable = unpacker.unpackBoolean();
         precision = unpacker.unpackInt();
         scale = unpacker.unpackInt();
@@ -247,6 +278,7 @@
             && Objects.equals(schemaName, meta.schemaName)
             && Objects.equals(tblName, meta.tblName)
             && Objects.equals(colName, meta.colName)
+            && Objects.equals(dataTypeCls, meta.dataTypeCls)
             && Objects.equals(dataTypeName, meta.dataTypeName);
     }
 
@@ -256,6 +288,7 @@
         result = 31 * result + (schemaName != null ? schemaName.hashCode() : 0);
         result = 31 * result + (tblName != null ? tblName.hashCode() : 0);
         result = 31 * result + (colName != null ? colName.hashCode() : 0);
+        result = 31 * result + (dataTypeCls != null ? dataTypeCls.hashCode() : 0);
         result = 31 * result + dataType;
         result = 31 * result + (dataTypeName != null ? dataTypeName.hashCode() : 0);
         result = 31 * result + precision;
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcMetaColumnsResult.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcMetaColumnsResult.java
index fed2195..5baabef 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcMetaColumnsResult.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcMetaColumnsResult.java
@@ -43,6 +43,16 @@
     /**
      * Constructor.
      *
+     * @param status Status code.
+     * @param err Error message.
+     */
+    public JdbcMetaColumnsResult(int status, String err) {
+        super(status, err);
+    }
+
+    /**
+     * Constructor.
+     *
      * @param meta Columns metadata.
      */
     public JdbcMetaColumnsResult(Collection<JdbcColumnMeta> meta) {
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcQueryMetadataRequest.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcQueryMetadataRequest.java
new file mode 100644
index 0000000..0aec745
--- /dev/null
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/JdbcQueryMetadataRequest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.client.proto.query.event;
+
+import org.apache.ignite.client.proto.query.ClientMessage;
+import org.apache.ignite.internal.client.proto.ClientMessagePacker;
+import org.apache.ignite.internal.client.proto.ClientMessageUnpacker;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * JDBC query metadata request.
+ */
+public class JdbcQueryMetadataRequest implements ClientMessage {
+    /** Cursor ID. */
+    private long cursorId;
+
+    /**
+     * Default constructor.
+     */
+    public JdbcQueryMetadataRequest() {
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param cursorId Cursor ID.
+     */
+    public JdbcQueryMetadataRequest(long cursorId) {
+        this.cursorId = cursorId;
+    }
+
+    /**
+     * Get the cursor id.
+     *
+     * @return Cursor ID.
+     */
+    public long cursorId() {
+        return cursorId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(ClientMessagePacker packer) {
+        packer.packLong(cursorId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(ClientMessageUnpacker unpacker) {
+        cursorId = unpacker.unpackLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryMetadataRequest.class, this);
+    }
+}
diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientDataType.java b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientDataType.java
index 603127e..c9ac4b6 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientDataType.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientDataType.java
@@ -71,4 +71,7 @@
 
     /** Boolean. */
     public static final int BOOLEAN = 17;
+
+    /** Big Integer. */
+    public static final int BIGINTEGER = 18;
 }
diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessagePacker.java b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessagePacker.java
index ea24276..e0406d3 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessagePacker.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessagePacker.java
@@ -701,6 +701,14 @@
                 packInt(ClientDataType.TIMESTAMP);
                 packTimestamp(((java.util.Date)arg).toInstant());
             }
+            else if (cls == BigDecimal.class) {
+                packInt(ClientDataType.DECIMAL);
+                packDecimal(((BigDecimal)arg));
+            }
+            else if (cls == BigInteger.class) {
+                packInt(ClientDataType.BIGINTEGER);
+                packBigInteger(((BigInteger)arg));
+            }
             else
                 throw new UnsupportedOperationException("Custom objects are not supported");
         }
diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessageUnpacker.java b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessageUnpacker.java
index 4474e01..feb848d 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessageUnpacker.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientMessageUnpacker.java
@@ -42,6 +42,7 @@
 import org.msgpack.core.buffer.InputStreamBufferInput;
 import org.msgpack.value.ImmutableValue;
 
+import static org.apache.ignite.internal.client.proto.ClientDataType.BIGINTEGER;
 import static org.apache.ignite.internal.client.proto.ClientDataType.BITMASK;
 import static org.apache.ignite.internal.client.proto.ClientDataType.BOOLEAN;
 import static org.apache.ignite.internal.client.proto.ClientDataType.BYTES;
@@ -616,6 +617,9 @@
             case DECIMAL:
                 return unpackDecimal();
 
+            case BIGINTEGER:
+                return unpackBigInteger();
+
             case BITMASK:
                 return unpackBitSet();
 
diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientOp.java b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientOp.java
index 87b44ad..98ade7f 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientOp.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientOp.java
@@ -131,4 +131,7 @@
 
     /** Get primary key metadata. */
     public static final int SQL_PK_META = 41;
+
+    /** Get query metadata. */
+    public static final int SQL_QUERY_META = 42;
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
index 47173c8..c615e8f 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientInboundMessageHandler.java
@@ -30,6 +30,7 @@
 import org.apache.ignite.client.handler.requests.sql.ClientSqlExecuteRequest;
 import org.apache.ignite.client.handler.requests.sql.ClientSqlFetchRequest;
 import org.apache.ignite.client.handler.requests.sql.ClientSqlPrimaryKeyMetadataRequest;
+import org.apache.ignite.client.handler.requests.sql.ClientSqlQueryMetadataRequest;
 import org.apache.ignite.client.handler.requests.sql.ClientSqlSchemasMetadataRequest;
 import org.apache.ignite.client.handler.requests.sql.ClientSqlTableMetadataRequest;
 import org.apache.ignite.client.handler.requests.sql.JdbcMetadataCatalog;
@@ -368,6 +369,9 @@
             case ClientOp.SQL_PK_META:
                 return ClientSqlPrimaryKeyMetadataRequest.process(in, out, handler);
 
+            case ClientOp.SQL_QUERY_META:
+                return ClientSqlQueryMetadataRequest.process(in, out, handler);
+
             default:
                 throw new IgniteException("Unexpected operation code: " + opCode);
         }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
index c6001ea..5befe1b 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
@@ -24,6 +24,8 @@
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.ignite.client.handler.requests.sql.JdbcMetadataCatalog;
 import org.apache.ignite.client.proto.query.JdbcQueryEventHandler;
 import org.apache.ignite.client.proto.query.event.BatchExecuteRequest;
@@ -38,6 +40,7 @@
 import org.apache.ignite.client.proto.query.event.JdbcMetaTablesRequest;
 import org.apache.ignite.client.proto.query.event.JdbcMetaTablesResult;
 import org.apache.ignite.client.proto.query.event.JdbcPrimaryKeyMeta;
+import org.apache.ignite.client.proto.query.event.JdbcQueryMetadataRequest;
 import org.apache.ignite.client.proto.query.event.JdbcTableMeta;
 import org.apache.ignite.client.proto.query.event.QueryCloseRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseResult;
@@ -49,6 +52,8 @@
 import org.apache.ignite.client.proto.query.event.Response;
 import org.apache.ignite.internal.processors.query.calcite.QueryProcessor;
 import org.apache.ignite.internal.processors.query.calcite.SqlCursor;
+import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadata;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.util.Cursor;
 
 import static org.apache.ignite.client.proto.query.IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
@@ -173,6 +178,65 @@
     }
 
     /** {@inheritDoc} */
+    @Override public JdbcMetaColumnsResult queryMetadata(JdbcQueryMetadataRequest req) {
+        SqlCursor<List<?>> cur = openCursors.get(req.cursorId());
+
+        if (cur == null)
+            return new JdbcMetaColumnsResult(Response.STATUS_FAILED,
+                "Failed to find query cursor with ID: " + req.cursorId());
+
+        if (cur.getColumnMetadata() == null)
+            return new JdbcMetaColumnsResult(Response.STATUS_FAILED,
+                "Failed to get query metadata for cursor with ID : " + req.cursorId());
+
+        FieldsMetadata metadata = cur.getColumnMetadata();
+
+        List<List<String>> origins = metadata.origins();
+        List<RelDataTypeField> list = metadata.rowType().getFieldList();
+
+        List<JdbcColumnMeta> meta = new ArrayList<>(list.size());
+
+        IgniteTypeFactory factory = new IgniteTypeFactory();
+
+        for (int i = 0; i < list.size(); i++) {
+            RelDataTypeField field = list.get(i);
+            List<String> origin = origins == null ? null : origins.get(i);
+
+            meta.add(createColumnMetadata(origin, field, factory));
+        }
+
+        return new JdbcMetaColumnsResult(meta);
+    }
+
+    /**
+     * Create Jdbc representation of column metadata from given origin and RelDataTypeField field.
+     *
+     * @param origin List of column origin. Contains schema name and table name. Might be null.
+     * @param field RelDataTypeField field with info about column.
+     * @param factory IgniteTypeFactory.
+     * @return JdbcColumnMeta object.
+     */
+    private JdbcColumnMeta createColumnMetadata(List<String> origin, RelDataTypeField field, IgniteTypeFactory factory) {
+        RelDataType val = field.getValue();
+
+        String schemaName = origin == null ? null : origin.get(0);
+        String tblName = origin == null ? null : origin.get(1);
+
+        String colName = field.getKey();
+        boolean isNullable = val.isNullable();
+
+        return new JdbcColumnMeta(
+            schemaName,
+            tblName,
+            colName,
+            factory.getJavaClass(val).getTypeName(),
+            val.getPrecision(),
+            val.getScale(),
+            isNullable
+        );
+    }
+
+    /** {@inheritDoc} */
     @Override public JdbcMetaTablesResult tablesMeta(JdbcMetaTablesRequest req) {
         List<JdbcTableMeta> tblsMeta = meta.getTablesMeta(req.schemaName(), req.tableName(), req.tableTypes());
 
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/sql/ClientSqlQueryMetadataRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/sql/ClientSqlQueryMetadataRequest.java
new file mode 100644
index 0000000..ff0c574
--- /dev/null
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/sql/ClientSqlQueryMetadataRequest.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.client.handler.requests.sql;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.client.proto.query.JdbcQueryEventHandler;
+import org.apache.ignite.client.proto.query.event.JdbcMetaColumnsResult;
+import org.apache.ignite.client.proto.query.event.JdbcQueryMetadataRequest;
+import org.apache.ignite.internal.client.proto.ClientMessagePacker;
+import org.apache.ignite.internal.client.proto.ClientMessageUnpacker;
+
+/**
+ * Client sql query metadata request handler.
+ */
+public class ClientSqlQueryMetadataRequest {
+    /**
+     * Processes remote {@code JdbcQueryMetadataRequest}.
+     *
+     * @param in Client message unpacker.
+     * @param out Client message packer.
+     * @param handler Query event handler.
+     * @return null value indicates synchronous operation.
+     */
+    public static CompletableFuture<Void> process(
+        ClientMessageUnpacker in,
+        ClientMessagePacker out,
+        JdbcQueryEventHandler handler
+    ) {
+        var req = new JdbcQueryMetadataRequest();
+
+        req.readBinary(in);
+
+        JdbcMetaColumnsResult res = handler.queryMetadata(req);
+
+        res.writeBinary(out);
+
+        return null;
+    }
+}
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/query/JdbcClientQueryEventHandler.java b/modules/client/src/main/java/org/apache/ignite/internal/client/query/JdbcClientQueryEventHandler.java
index 441776c..e62deeb 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/query/JdbcClientQueryEventHandler.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/query/JdbcClientQueryEventHandler.java
@@ -28,6 +28,7 @@
 import org.apache.ignite.client.proto.query.event.JdbcMetaSchemasResult;
 import org.apache.ignite.client.proto.query.event.JdbcMetaTablesRequest;
 import org.apache.ignite.client.proto.query.event.JdbcMetaTablesResult;
+import org.apache.ignite.client.proto.query.event.JdbcQueryMetadataRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseResult;
 import org.apache.ignite.client.proto.query.event.QueryExecuteRequest;
@@ -122,4 +123,13 @@
 
         return res;
     }
+
+    /** {@inheritDoc} */
+    @Override public JdbcMetaColumnsResult queryMetadata(JdbcQueryMetadataRequest req) {
+        JdbcMetaColumnsResult res = new JdbcMetaColumnsResult();
+
+        client.sendRequest(ClientOp.SQL_QUERY_META, req, res);
+
+        return res;
+    }
 }
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSet.java b/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSet.java
index 8d4a61a..3c4db6f 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSet.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSet.java
@@ -57,6 +57,8 @@
 import org.apache.ignite.client.proto.query.JdbcQueryEventHandler;
 import org.apache.ignite.client.proto.query.SqlStateCode;
 import org.apache.ignite.client.proto.query.event.JdbcColumnMeta;
+import org.apache.ignite.client.proto.query.event.JdbcMetaColumnsResult;
+import org.apache.ignite.client.proto.query.event.JdbcQueryMetadataRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseRequest;
 import org.apache.ignite.client.proto.query.event.QueryCloseResult;
 import org.apache.ignite.client.proto.query.event.QueryFetchRequest;
@@ -137,6 +139,9 @@
     /** Query request handler. */
     private JdbcQueryEventHandler qryHandler;
 
+    /** Jdbc metadata. */
+    private JdbcResultSetMetadata jdbcMeta;
+
     /**
      * Creates new result set.
      *
@@ -721,7 +726,10 @@
     @Override public ResultSetMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("ResultSetMetaData are not supported.");
+        if (jdbcMeta == null)
+            jdbcMeta = new JdbcResultSetMetadata(meta());
+
+        return jdbcMeta;
     }
 
     /** {@inheritDoc} */
@@ -730,9 +738,6 @@
 
         Objects.requireNonNull(colLb);
 
-        if (!metaInit)
-            throw new SQLFeatureNotSupportedException("FindColumn by column label are not supported.");
-
         Integer order = columnOrder().get(colLb.toUpperCase());
 
         if (order == null)
@@ -1971,6 +1976,14 @@
         if (finished && (!isQuery || autoClose))
             throw new SQLException("Server cursor is already closed.", SqlStateCode.INVALID_CURSOR_STATE);
 
-        throw new SQLFeatureNotSupportedException("ResultSetMetaData are not supported.");
+        if (!metaInit) {
+            JdbcMetaColumnsResult res = qryHandler.queryMetadata(new JdbcQueryMetadataRequest(cursorId));
+
+            meta = res.meta();
+
+            metaInit = true;
+        }
+
+        return meta;
     }
 }
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSetMetadata.java b/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSetMetadata.java
new file mode 100644
index 0000000..2feed79
--- /dev/null
+++ b/modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcResultSetMetadata.java
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.ignite.client.proto.query.event.JdbcColumnMeta;
+
+/**
+ * JDBC result set metadata implementation.
+ */
+public class JdbcResultSetMetadata implements ResultSetMetaData {
+    /** Column width. */
+    private static final int COL_WIDTH = 30;
+
+    /** Table names. */
+    private final List<JdbcColumnMeta> meta;
+
+    /**
+     * Constructor.
+     *
+     * @param meta Metadata.
+     */
+    JdbcResultSetMetadata(List<JdbcColumnMeta> meta) {
+        assert meta != null;
+
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getColumnCount() throws SQLException {
+        return meta.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAutoIncrement(int col) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isCaseSensitive(int col) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isSearchable(int col) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isCurrency(int col) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int isNullable(int col) throws SQLException {
+        return columnNullable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isSigned(int col) throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getColumnDisplaySize(int col) throws SQLException {
+        return COL_WIDTH;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getColumnLabel(int col) throws SQLException {
+        return meta.get(col - 1).columnName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getColumnName(int col) throws SQLException {
+        return meta.get(col - 1).columnName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSchemaName(int col) throws SQLException {
+        return meta.get(col - 1).schemaName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getPrecision(int col) throws SQLException {
+        return meta.get(col - 1).precision();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getScale(int col) throws SQLException {
+        return meta.get(col - 1).scale();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTableName(int col) throws SQLException {
+        return meta.get(col - 1).tableName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCatalogName(int col) throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getColumnType(int col) throws SQLException {
+        return meta.get(col - 1).dataType();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getColumnTypeName(int col) throws SQLException {
+        return meta.get(col - 1).dataTypeName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isReadOnly(int col) throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWritable(int col) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDefinitelyWritable(int col) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getColumnClassName(int col) throws SQLException {
+        return meta.get(col - 1).dataTypeClass();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("Result set 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(JdbcResultSetMetadata.class);
+    }
+}
\ No newline at end of file
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCursor.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCursor.java
index cd8f5a2..2e50709 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCursor.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCursor.java
@@ -24,6 +24,7 @@
 import java.util.UUID;
 import org.apache.ignite.internal.processors.query.calcite.SqlCursor;
 import org.apache.ignite.internal.processors.query.calcite.SqlQueryType;
+import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadata;
 
 public class FakeCursor implements SqlCursor<List<?>> {
 
@@ -60,4 +61,8 @@
     @Override public SqlQueryType getQueryType() {
         return SqlQueryType.QUERY;
     }
+
+    @Override public FieldsMetadata getColumnMetadata() {
+        return null;
+    }
 }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java
index d6b13c7..c74bebf 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java
@@ -48,7 +48,7 @@
     }};
 
     /** Cluster nodes. */
-    private static final List<Ignite> clusterNodes = new ArrayList<>();
+    protected static final List<Ignite> clusterNodes = new ArrayList<>();
 
     /**
      * Creates a cluster of three nodes.
@@ -71,9 +71,10 @@
      */
     @AfterAll
     public static void afterAll() throws Exception {
-        for (Ignite clusterNode : clusterNodes) {
+        for (Ignite clusterNode : clusterNodes)
             clusterNode.close();
-        }
+
+        clusterNodes.clear();
     }
 
     /**
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcConnectionPropertiesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcConnectionPropertiesTest.java
similarity index 97%
rename from modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcConnectionPropertiesTest.java
rename to modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcConnectionPropertiesTest.java
index 00f82b7..8b328be 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcConnectionPropertiesTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcConnectionPropertiesTest.java
@@ -27,7 +27,7 @@
 /**
  * {@link ConnectionPropertiesImpl} unit tests.
  */
-public class JdbcConnectionPropertiesTest {
+public class ITJdbcConnectionPropertiesTest {
     /**
      * Test check the {@link ConnectionPropertiesImpl#getDriverPropertyInfo()} return properties with prefix {@link
      * ConnectionPropertiesImpl#PROP_PREFIX}
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcConnectionSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcConnectionSelfTest.java
similarity index 99%
rename from modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcConnectionSelfTest.java
rename to modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcConnectionSelfTest.java
index afe3bf2..003d9ae 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcConnectionSelfTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcConnectionSelfTest.java
@@ -59,7 +59,7 @@
  * Connection test.
  */
 @SuppressWarnings("ThrowableNotThrown")
-public class JdbcConnectionSelfTest extends AbstractJdbcSelfTest {
+public class ITJdbcConnectionSelfTest extends AbstractJdbcSelfTest {
     /**
      * @throws Exception If failed.
      */
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcMetadataSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcMetadataSelfTest.java
similarity index 92%
rename from modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcMetadataSelfTest.java
rename to modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcMetadataSelfTest.java
index 7818ec7..395cf26 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcMetadataSelfTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcMetadataSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.runner.app.jdbc;
 
 import java.math.BigDecimal;
-import java.nio.file.Path;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
@@ -32,26 +31,17 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
 import java.util.Set;
-import org.apache.ignite.app.Ignite;
-import org.apache.ignite.app.IgnitionManager;
 import org.apache.ignite.internal.client.proto.ProtocolVersion;
 import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
-import org.apache.ignite.jdbc.IgniteJdbcDriver;
 import org.apache.ignite.schema.ColumnType;
 import org.apache.ignite.schema.SchemaBuilders;
 import org.apache.ignite.schema.SchemaTable;
 import org.apache.ignite.table.Table;
 import org.apache.ignite.table.Tuple;
-import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
 
 import static java.sql.Types.DATE;
 import static java.sql.Types.DECIMAL;
@@ -66,56 +56,15 @@
 /**
  * Metadata tests.
  */
-public class JdbcMetadataSelfTest {
+public class ITJdbcMetadataSelfTest extends AbstractJdbcSelfTest {
     /** URL. */
     protected static final String URL = "jdbc:ignite:thin://127.0.1.1:10800";
 
-    /** Nodes bootstrap configuration. */
-    private static final Map<String, String> nodesBootstrapCfg = new LinkedHashMap<>() {{
-        put("node2", "{\n" +
-            "  \"node\": {\n" +
-            "    \"metastorageNodes\":[ \"node2\" ]\n" +
-            "  }\n" +
-            "}");
-    }};
-
-    /** Cluster nodes. */
-    protected static final List<Ignite> clusterNodes = new ArrayList<>();
-
-    /**
-     * Creates a cluster of three nodes.
-     *
-     * @param temp Temporal directory.
-     */
+    /** Creates tables. */
     @BeforeAll
-    public static void beforeAll(@TempDir Path temp) {
-        IgniteJdbcDriver.register();
+    public static void createTables() {
+        assert !clusterNodes.isEmpty();
 
-        nodesBootstrapCfg.forEach((nodeName, configStr) ->
-            clusterNodes.add(IgnitionManager.start(nodeName, configStr, temp.resolve(nodeName)))
-        );
-    }
-
-    /**
-     * Close all cluster nodes.
-     *
-     * @throws Exception if failed.
-     */
-    @AfterAll
-    public static void afterAll() throws Exception {
-        for (Ignite clusterNode : clusterNodes) {
-            clusterNode.close();
-        }
-    }
-
-    /**
-     * Create the connection ant statement.
-     *
-     * @throws Exception if failed.
-     */
-    @BeforeEach
-    public void beforeTest() {
-        // Create table on node 0.
         SchemaTable perTbl = SchemaBuilders.tableBuilder("PUBLIC", "PERSON").columns(
             SchemaBuilders.column("NAME", ColumnType.string()).asNullable().build(),
             SchemaBuilders.column("AGE", ColumnType.INT32).asNullable().build(),
@@ -128,9 +77,6 @@
             SchemaBuilders.column("BIGDATA", ColumnType.decimalOf(20, 10)).asNullable().build()
         ).withPrimaryKey("ID").build();
 
-        if (clusterNodes.get(0).tables().table(perTbl.canonicalName()) != null)
-            return;
-
         clusterNodes.get(0).tables().createTable(perTbl.canonicalName(), tblCh ->
             SchemaConfigurationConverter.convert(perTbl, tblCh)
                 .changeReplicas(1)
@@ -154,7 +100,6 @@
      * @throws Exception If failed.
      */
     @Test
-    @Disabled("IGNITE-15187")
     public void testResultSetMetaData() throws Exception {
         Statement stmt = DriverManager.getConnection(URL).createStatement();
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcResultSetSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcResultSetSelfTest.java
similarity index 95%
rename from modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcResultSetSelfTest.java
rename to modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcResultSetSelfTest.java
index 7d2a483..e530f62 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcResultSetSelfTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcResultSetSelfTest.java
@@ -49,11 +49,11 @@
 /**
  * Result set test.
  */
-public class JdbcResultSetSelfTest extends AbstractJdbcSelfTest {
+public class ITJdbcResultSetSelfTest extends AbstractJdbcSelfTest {
     /** SQL query. */
     private static final String SQL =
-        "SELECT 1::INTEGER, true, 1::TINYINT, 1::SMALLINT, 1::INTEGER, 1::BIGINT, 1.0::FLOAT, 1.0::DOUBLE, 1.0::DOUBLE, " +
-            "'1', '1', '1901-02-01'::DATE, '01:01:01'::TIME, 1::TIMESTAMP;";
+        "SELECT 1::INTEGER as id, true as boolVal, 1::TINYINT as byteVal, 1::SMALLINT as shortVal, 1::INTEGER as intVal, 1::BIGINT as longVal, 1.0::FLOAT as floatVal, 1.0::DOUBLE as doubleVal, 1.0::DECIMAL as bigVal, " +
+            "'1' as strVal, '1', '1901-02-01'::DATE as dateVal, '01:01:01'::TIME as timeVal, 1::TIMESTAMP as tsVal;";
 
     /** Statement. */
     private Statement stmt;
@@ -100,8 +100,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getBoolean("boolVal");
+                assertTrue(rs.getBoolean("boolVal"));
                 assertTrue(rs.getBoolean(2));
                 assertEquals(1, rs.getByte(2));
                 assertEquals(1, rs.getInt(2));
@@ -174,9 +173,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getByte("byteVal") == 1;
-
+                assertEquals(1, rs.getByte("byteVal"));
                 assertTrue(rs.getBoolean(3));
                 assertEquals(1, rs.getByte(3));
                 assertEquals(1, rs.getInt(3));
@@ -216,8 +213,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getShort("shortVal") == 1;
+                assertEquals(1, rs.getShort("shortVal"));
 
                 assertTrue(rs.getBoolean(4));
                 assertEquals(1, rs.getByte(4));
@@ -258,8 +254,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getInt("intVal") == 1;
+                assertEquals(1, rs.getInt("intVal"));
 
                 assertTrue(rs.getBoolean(5));
                 assertEquals(1, rs.getByte(5));
@@ -300,8 +295,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getLong("longVal") == 1;
+                assertEquals(1, rs.getLong("longVal"));
 
                 assertTrue(rs.getBoolean(6));
                 assertEquals(1, rs.getByte(6));
@@ -342,8 +336,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getFloat("floatVal") == 1.0;
+                assertEquals(1.0, rs.getFloat("floatVal"));
 
                 assertTrue(rs.getBoolean(7));
                 assertEquals(1, rs.getByte(7));
@@ -384,8 +377,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-//                assert rs.getDouble("doubleVal") == 1.0;
+                assertEquals(1.0, rs.getDouble("doubleVal"));
 
                 assertTrue(rs.getBoolean(8));
                 assertEquals(1, rs.getByte(8));
@@ -419,7 +411,6 @@
      * @throws Exception If failed.
      */
     @Test
-    @Disabled
     public void testBigDecimal() throws Exception {
         ResultSet rs = stmt.executeQuery(SQL);
 
@@ -436,8 +427,8 @@
                 assertEquals(1, rs.getLong(9));
                 assertEquals(1.0, rs.getDouble(9));
                 assertEquals(1.0f, rs.getFloat(9));
-                assertEquals(new BigDecimal(1), rs.getBigDecimal(9));
-                assertEquals(rs.getString(9), "1");
+                assertEquals(new BigDecimal("1.0"), rs.getBigDecimal(9));
+                assertEquals(rs.getString(9), "1.0");
 
                 assertTrue(rs.getObject(9, Boolean.class));
                 assertEquals((byte)1, rs.getObject(9, Byte.class));
@@ -446,8 +437,8 @@
                 assertEquals(1, rs.getObject(9, Long.class));
                 assertEquals(1.f, rs.getObject(9, Float.class));
                 assertEquals(1, rs.getObject(9, Double.class));
-                assertEquals(new BigDecimal(1), rs.getObject(9, BigDecimal.class));
-                assertEquals(rs.getObject(9, String.class), "1");
+                assertEquals(new BigDecimal("1.0"), rs.getObject(9, BigDecimal.class));
+                assertEquals(rs.getObject(9, String.class), "1.0");
             }
 
             cnt++;
@@ -495,8 +486,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-                //TODO IGNITE-15187
-                //assert "1".equals(rs.getString("strVal"));
+                assert "1".equals(rs.getString("strVal"));
 
                 assertTrue(rs.getBoolean(10));
                 assertEquals(1, rs.getByte(10));
@@ -564,7 +554,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-//                assert rs.getDate("dateVal").equals(new Date(1, 1, 1));
+                assert rs.getDate("dateVal").equals(new Date(1, 1, 1));
 
                 assertEquals(new Date(1, 1, 1), rs.getDate(12));
                 assertEquals(new Time(new Date(1, 1, 1).getTime()), rs.getTime(12));
@@ -595,7 +585,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-//                assert rs.getTime("timeVal").equals(new Time(1, 1, 1));
+                assert rs.getTime("timeVal").equals(new Time(1, 1, 1));
 
                 assertEquals(new Date(new Time(1, 1, 1).getTime()), rs.getDate(13));
                 assertEquals(new Time(1, 1, 1), rs.getTime(13));
@@ -625,7 +615,7 @@
 
         while (rs.next()) {
             if (cnt == 0) {
-//                assert rs.getTimestamp("tsVal").getTime() == 1;
+                assertEquals(-10800000, rs.getTimestamp("tsVal").getTime());
                 assertEquals(new Date(new Timestamp(-10800000).getTime()), rs.getDate(14));
                 assertEquals(new Time(new Timestamp(-10800000).getTime()), rs.getTime(14));
                 assertEquals(new Timestamp(-10800000), rs.getTimestamp(14));
@@ -718,12 +708,9 @@
     }
 
     /**
-     * TODO IGNITE-15187
-     *
      * @throws Exception If failed.
      */
     @Test
-    @Disabled
     public void testFindColumn() throws Exception {
         final ResultSet rs = stmt.executeQuery(SQL);
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcStatementSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcStatementSelfTest.java
similarity index 99%
rename from modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcStatementSelfTest.java
rename to modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcStatementSelfTest.java
index 1268001..c24e0d6 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/JdbcStatementSelfTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ITJdbcStatementSelfTest.java
@@ -40,7 +40,7 @@
  * Statement test.
  */
 @SuppressWarnings({"ThrowableNotThrown"})
-public class JdbcStatementSelfTest extends AbstractJdbcSelfTest {
+public class ITJdbcStatementSelfTest extends AbstractJdbcSelfTest {
     /** SQL query. */
     private static final String SQL =
         "select 1::INTEGER, true, 1::TINYINT, 1::SMALLINT, 1::INTEGER, 1::BIGINT, 1.0::FLOAT, 1.0::DOUBLE, 1.0::DOUBLE, '1';";
@@ -89,7 +89,7 @@
      * @throws Exception If failed.
      */
     @Test
-    @Disabled("IGNITE-15187 + IGNITE-15108")
+    @Disabled("IGNITE-15108")
     public void testExecuteQuery0() throws Exception {
         ResultSet rs = stmt.executeQuery(SQL);
 
@@ -146,7 +146,7 @@
      * @throws Exception If failed.
      */
     @Test
-    @Disabled("IGNITE-15187 + IGNITE-15108")
+    @Disabled("IGNITE-15108")
     public void testExecute() throws Exception {
         assertTrue(stmt.execute(SQL));
 
@@ -186,7 +186,7 @@
      * @throws Exception If failed.
      */
     @Test
-    @Disabled("IGNITE-15187 + IGNITE-15108")
+    @Disabled("IGNITE-15108")
     public void testMaxRows() throws Exception {
         stmt.setMaxRows(1);