You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2017/05/23 12:14:14 UTC

ignite git commit: IGNITE-5252: Expose getFieldName method to SqlFieldsQuery result. - Fixes #1982.

Repository: ignite
Updated Branches:
  refs/heads/master 42018e6a9 -> 647fd195b


IGNITE-5252: Expose getFieldName method to SqlFieldsQuery result. - Fixes #1982.

Signed-off-by: Sergi Vladykin <se...@gmail.com>


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

Branch: refs/heads/master
Commit: 647fd195b310df10b230b67c92a8df04b5a064e2
Parents: 42018e6
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Tue May 23 15:13:37 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Tue May 23 15:13:37 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     | 13 +++++++
 .../ignite/cache/query/FieldsQueryCursor.java   | 39 ++++++++++++++++++++
 .../processors/cache/IgniteCacheProxy.java      | 11 +++++-
 .../processors/cache/QueryCursorImpl.java       | 19 +++++++++-
 .../processors/query/GridQueryIndexing.java     |  5 ++-
 .../processors/query/GridQueryProcessor.java    | 15 ++++----
 .../multijvm/IgniteCacheProcessProxy.java       |  8 ++++
 .../processors/query/h2/IgniteH2Indexing.java   |  5 ++-
 .../query/h2/ddl/DdlStatementsProcessor.java    |  4 +-
 .../cache/SqlFieldsQuerySelfTest.java           | 12 ++++--
 10 files changed, 112 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 9c8c090..aeabdb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -21,6 +21,7 @@ import java.io.Serializable;
 import java.sql.Timestamp;
 import java.util.Collection;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -41,6 +42,7 @@ import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.QueryDetailMetrics;
@@ -346,6 +348,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
 
     /**
      * Queries cache. Accepts any subclass of {@link Query} interface.
+     * See also {@link #query(SqlFieldsQuery)}.
      *
      * @param qry Query.
      * @return Cursor.
@@ -354,10 +357,20 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * @see SqlFieldsQuery
      * @see TextQuery
      * @see SpiQuery
+     *
      */
     public <R> QueryCursor<R> query(Query<R> qry);
 
     /**
+     * Queries cache. Accepts {@link SqlFieldsQuery} class.
+     *
+     * @param qry SqlFieldsQuery.
+     * @return Cursor.
+     * @see SqlFieldsQuery
+     */
+    public FieldsQueryCursor<List<?>> query(SqlFieldsQuery qry);
+
+    /**
      * Queries the cache transforming the entries on the server nodes. Can be used, for example,
      * to avoid network overhead in case only one field out of the large is required by client.
      * <p>

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/core/src/main/java/org/apache/ignite/cache/query/FieldsQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/FieldsQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/cache/query/FieldsQueryCursor.java
new file mode 100644
index 0000000..4219bae
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/FieldsQueryCursor.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cache.query;
+
+/**
+ * SQL query result cursor. This extends {@link QueryCursor}
+ * to expose fields metadata to public API for SqlFieldsQueries.
+ */
+public interface FieldsQueryCursor<T> extends QueryCursor<T> {
+    /**
+     * Gets field name.
+     *
+     * @param idx field index.
+     * @return Field name.
+     */
+    String getFieldName(int idx);
+
+    /**
+     * Gets number of columns in a row.
+     *
+     * @return row size.
+     */
+    int getColumnsCount();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index c3a24fa..40aafeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -49,6 +49,7 @@ import org.apache.ignite.cache.CacheManager;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.QueryDetailMetrics;
@@ -561,7 +562,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                             if (needToConvert) {
                                 Map.Entry<K, V> entry = (Map.Entry<K, V>)next;
 
-                                return (R) new CacheEntryImpl<>(entry.getKey(), entry.getValue());
+                                return (R)new CacheEntryImpl<>(entry.getKey(), entry.getValue());
                             }
 
                             return (R)next;
@@ -753,6 +754,12 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
+    @Override public FieldsQueryCursor<List<?>> query(SqlFieldsQuery qry) {
+        return (FieldsQueryCursor<List<?>>)query((Query)qry);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public <R> QueryCursor<R> query(Query<R> qry) {
         A.notNull(qry, "qry");
 
@@ -780,7 +787,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
             }
 
             if (qry instanceof SqlFieldsQuery)
-                return (QueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry);
+                return (FieldsQueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry);
 
             if (qry instanceof ScanQuery)
                 return query((ScanQuery)qry, null, projection(qry.isLocal()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
index 24789fc..4fc1054 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
@@ -38,7 +39,7 @@ import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.
 /**
  * Query cursor implementation.
  */
-public class QueryCursorImpl<T> implements QueryCursorEx<T> {
+public class QueryCursorImpl<T> implements QueryCursorEx<T>, FieldsQueryCursor<T> {
     /** */
     private final static AtomicReferenceFieldUpdater<QueryCursorImpl, State> STATE_UPDATER =
         AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state");
@@ -188,6 +189,22 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
         return fieldsMeta;
     }
 
+    /** {@inheritDoc} */
+    @Override public String getFieldName(int idx) {
+        assert this.fieldsMeta != null;
+
+        GridQueryFieldMetadata metadata = fieldsMeta.get(idx);
+
+        return metadata.fieldName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getColumnsCount() {
+        assert this.fieldsMeta != null;
+
+        return fieldsMeta.size();
+    }
+
     /** Query cursor state */
     protected enum State {
         /** Idle. */IDLE,

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 8180cd0..bfa223e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -24,6 +24,7 @@ import java.util.List;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -82,7 +83,7 @@ public interface GridQueryIndexing {
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public QueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+    public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
         GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
@@ -119,7 +120,7 @@ public interface GridQueryIndexing {
      * @param cancel Query cancel.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> queryLocalSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+    public FieldsQueryCursor<List<?>> queryLocalSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
         IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/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 6a479ab..0df0f52 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
@@ -24,6 +24,7 @@ import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -1653,7 +1654,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Cursor.
      */
     @SuppressWarnings("unchecked")
-    public QueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
+    public FieldsQueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
         checkxEnabled();
 
         if (qry.isReplicatedOnly() && qry.getPartitions() != null)
@@ -1669,14 +1670,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            IgniteOutClosureX<QueryCursor<List<?>>> clo;
+            IgniteOutClosureX<FieldsQueryCursor<List<?>>> clo;
 
             if (loc) {
-                clo = new IgniteOutClosureX<QueryCursor<List<?>>>() {
-                    @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                clo = new IgniteOutClosureX<FieldsQueryCursor<List<?>>>() {
+                    @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
                         GridQueryCancel cancel = new GridQueryCancel();
 
-                        final QueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry,
+                        final FieldsQueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry,
                             idx.backupFilter(requestTopVer.get(), qry.getPartitions()), cancel);
 
                         Iterable<List<?>> iterExec = new Iterable<List<?>>() {
@@ -1699,8 +1700,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 };
             }
             else {
-                clo = new IgniteOutClosureX<QueryCursor<List<?>>>() {
-                    @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                clo = new IgniteOutClosureX<FieldsQueryCursor<List<?>>>() {
+                    @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
                         return idx.queryDistributedSqlFields(cctx, qry, null);
                     }
                 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index d203794..630e441 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testframework.junits.multijvm;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.locks.Lock;
@@ -38,10 +39,12 @@ import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.QueryDetailMetrics;
 import org.apache.ignite.cache.query.QueryMetrics;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
@@ -191,6 +194,11 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public FieldsQueryCursor<List<?>> query(SqlFieldsQuery qry) {
+        throw new UnsupportedOperationException("Method should be supported.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <T, R> QueryCursor<R> query(Query<T> qry, IgniteClosure<T, R> transformer) {
         throw new UnsupportedOperationException("Method should be supported.");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index ce69bb1..c099d77 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -61,6 +61,7 @@ import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -1335,7 +1336,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
+    @Override public FieldsQueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
         final SqlFieldsQuery qry, final IndexingQueryFilter filter, final GridQueryCancel cancel)
         throws IgniteCheckedException {
 
@@ -1568,7 +1569,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+    @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
         GridQueryCancel cancel) {
         final String space = cctx.name();
         final String sqlQry = qry.getSql();

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index fe2bfdb..3991777 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.QueryIndex;
-import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
@@ -76,7 +76,7 @@ public class DdlStatementsProcessor {
      * @param stmt H2 statement to parse and execute.
      */
     @SuppressWarnings("unchecked")
-    public QueryCursor<List<?>> runDdlStatement(String sql, PreparedStatement stmt)
+    public FieldsQueryCursor<List<?>> runDdlStatement(String sql, PreparedStatement stmt)
         throws IgniteCheckedException {
         assert stmt instanceof JdbcPreparedStatement;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/647fd195/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlFieldsQuerySelfTest.java
index a23f254..8860b2b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/SqlFieldsQuerySelfTest.java
@@ -21,7 +21,7 @@ import java.io.Serializable;
 import java.util.List;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -66,11 +66,17 @@ public class SqlFieldsQuerySelfTest extends GridCommonAbstractTest {
     private void executeQuery() {
         IgniteCache<?, ?> cache = grid(1).cache("person");
 
-        SqlFieldsQuery qry = new SqlFieldsQuery("select name, age from person where age > 10");
+        SqlFieldsQuery qry = new SqlFieldsQuery("select name as \"Full Name\", age from person where age > 10");
 
-        QueryCursor<List<?>> qryCursor = cache.query(qry);
+        FieldsQueryCursor<List<?>> qryCursor = cache.query(qry);
 
         assertEquals(2, qryCursor.getAll().size());
+
+        assertEquals(2, qryCursor.getColumnsCount()); // Row contains "name" and "age" fields.
+
+        assertEquals("Full Name", qryCursor.getFieldName(0));
+
+        assertEquals("AGE", qryCursor.getFieldName(1));
     }