You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/27 07:17:04 UTC

[02/37] ignite git commit: IGNITE-3487: hidden _key and _val columns - Fixes #1865.

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index b9394ec..a00ea90 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.opt;
 
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -41,6 +42,7 @@ import org.h2.command.ddl.CreateTableData;
 import org.h2.engine.Session;
 import org.h2.index.Index;
 import org.h2.index.IndexType;
+import org.h2.index.SpatialIndex;
 import org.h2.message.DbException;
 import org.h2.result.Row;
 import org.h2.result.SearchRow;
@@ -152,6 +154,14 @@ public class GridH2Table extends TableBase {
 
         assert idxs != null;
 
+        List<Index> clones = new ArrayList<>(idxs.size());
+        for (Index index: idxs) {
+            Index clone = createDuplicateIndexIfNeeded(index);
+            if (clone != null)
+               clones.add(clone);
+        }
+        idxs.addAll(clones);
+
         // Add scan index at 0 which is required by H2.
         if (idxs.size() >= 2 && index(0).getIndexType().isHash())
             idxs.add(0, new GridH2PrimaryScanIndex(this, index(1), index(0)));
@@ -365,6 +375,9 @@ public class GridH2Table extends TableBase {
 
             boolean reuseExisting = s != null;
 
+            if (!(idxs.get(i) instanceof GridH2IndexBase))
+                continue;
+
             s = index(i).takeSnapshot(s, qctx);
 
             if (reuseExisting && s == null) { // Existing snapshot was invalidated before we were able to reserve it.
@@ -373,7 +386,8 @@ public class GridH2Table extends TableBase {
                     qctx.clearSnapshots();
 
                 for (int j = 2; j < i; j++)
-                    index(j).releaseSnapshot();
+                    if ((idxs.get(j) instanceof GridH2IndexBase))
+                        index(j).releaseSnapshot();
 
                 // Drop invalidated snapshot.
                 actualSnapshot.compareAndSet(snapshots, null);
@@ -406,7 +420,8 @@ public class GridH2Table extends TableBase {
             destroyed = true;
 
             for (int i = 1, len = idxs.size(); i < len; i++)
-                index(i).destroy();
+                if (idxs.get(i) instanceof GridH2IndexBase)
+                    index(i).destroy();
         }
         finally {
             unlock(true);
@@ -561,6 +576,8 @@ public class GridH2Table extends TableBase {
                 // Put row if absent to all indexes sequentially.
                 // Start from 3 because 0 - Scan (don't need to update), 1 - PK hash (already updated), 2 - PK (already updated).
                 while (++i < len) {
+                    if (!(idxs.get(i) instanceof GridH2IndexBase))
+                        continue;
                     GridH2IndexBase idx = index(i);
 
                     addToIndex(idx, pk, row, old, false);
@@ -577,6 +594,8 @@ public class GridH2Table extends TableBase {
                     // Remove row from all indexes.
                     // Start from 3 because 0 - Scan (don't need to update), 1 - PK hash (already updated), 2 - PK (already updated).
                     for (int i = 3, len = idxs.size(); i < len; i++) {
+                        if (!(idxs.get(i) instanceof GridH2IndexBase))
+                            continue;
                         Row res = index(i).remove(old);
 
                         assert eq(pk, res, old) : "\n" + old + "\n" + res + "\n" + i + " -> " + index(i).getName();
@@ -650,7 +669,8 @@ public class GridH2Table extends TableBase {
         ArrayList<GridH2IndexBase> res = new ArrayList<>(idxs.size() - 2);
 
         for (int i = 2, len = idxs.size(); i < len; i++)
-            res.add(index(i));
+            if (idxs.get(i) instanceof GridH2IndexBase)
+                res.add(index(i));
 
         return res;
     }
@@ -721,16 +741,24 @@ public class GridH2Table extends TableBase {
 
             assert idx != null;
 
-            ArrayList<Index> newIdxs = new ArrayList<>(idxs.size() + 1);
+            Index cloneIdx = createDuplicateIndexIfNeeded(idx);
+
+            ArrayList<Index> newIdxs = new ArrayList<>(
+                    idxs.size() + ((cloneIdx == null) ? 1 : 2));
 
             newIdxs.addAll(idxs);
 
             newIdxs.add(idx);
+            if (cloneIdx != null)
+                newIdxs.add(cloneIdx);
 
             idxs = newIdxs;
 
             database.addSchemaObject(ses, idx);
 
+            if (cloneIdx != null)
+                database.addSchemaObject(ses, cloneIdx);
+
             setModified();
 
             return idx;
@@ -762,23 +790,42 @@ public class GridH2Table extends TableBase {
 
     /** {@inheritDoc} */
     @Override public void removeIndex(Index h2Idx) {
+        throw DbException.getUnsupportedException("must use removeIndex(session, idx)");
+    }
+
+    /**
+     * Remove the given index from the list.
+     *
+     * @param h2Idx the index to remove
+     */
+    public void removeIndex(Session session, Index h2Idx) {
         lock(true);
 
         try {
             ArrayList<Index> idxs = new ArrayList<>(this.idxs);
 
-            for (int i = 2; i < idxs.size(); i++) {
-                GridH2IndexBase idx = (GridH2IndexBase)idxs.get(i);
+            Index targetIdx = (h2Idx instanceof GridH2ProxyIndex)?
+                    ((GridH2ProxyIndex)h2Idx).underlyingIndex(): h2Idx;
 
-                if (idx != h2Idx)
-                    continue;
+            for (int i = 2; i < idxs.size(); ) {
+                Index idx = idxs.get(i);
 
-                idxs.remove(i);
+                if (idx == targetIdx || (idx instanceof GridH2ProxyIndex &&
+                   ((GridH2ProxyIndex)idx).underlyingIndex() == targetIdx)) {
 
-                this.idxs = idxs;
+                    idxs.remove(i);
 
-                return;
+                    if (idx instanceof GridH2ProxyIndex &&
+                        idx.getSchema().findIndex(session, idx.getName()) != null)
+                        database.removeSchemaObject(session, idx);
+
+                    continue;
+                }
+
+                i++;
             }
+
+            this.idxs = idxs;
         }
         finally {
             unlock(true);
@@ -912,4 +959,47 @@ public class GridH2Table extends TableBase {
         return rowFactory;
     }
 
+    /**
+     * Creates proxy index for given target index.
+     * Proxy index refers to alternative key and val columns.
+     *
+     * @param target Index to clone.
+     * @return Proxy index.
+     */
+    public Index createDuplicateIndexIfNeeded(Index target) {
+        if (!(target instanceof H2TreeIndex) &&
+            !(target instanceof SpatialIndex))
+            return null;
+
+        IndexColumn[] cols = target.getIndexColumns();
+        List<IndexColumn> proxyCols = new ArrayList<>(cols.length);
+        boolean modified = false;
+        for (int i = 0; i < cols.length; i++) {
+            IndexColumn col = cols[i];
+            IndexColumn proxyCol = new IndexColumn();
+            proxyCol.columnName = col.columnName;
+            proxyCol.column = col.column;
+            proxyCol.sortType = col.sortType;
+
+            int altColId = desc.getAlternativeColumnId(proxyCol.column.getColumnId());
+            if (altColId != proxyCol.column.getColumnId()) {
+                proxyCol.column = getColumn(altColId);
+                proxyCol.columnName = proxyCol.column.getName();
+                modified = true;
+            }
+
+            proxyCols.add(proxyCol);
+        }
+
+        if (modified) {
+            String proxyName = target.getName() + "_proxy";
+
+            if (target.getIndexType().isSpatial())
+                return new GridH2ProxySpatialIndex(this, proxyName, proxyCols, target);
+
+            return new GridH2ProxyIndex(this, proxyName, proxyCols, target);
+        }
+
+        return null;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
index 4395024..6671dc6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
@@ -54,8 +54,8 @@ import org.apache.lucene.util.Version;
 import org.h2.util.JdbcUtils;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.KEY_FIELD_NAME;
-import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.VAL_FIELD_NAME;
+import static org.apache.ignite.internal.processors.query.QueryUtils.KEY_FIELD_NAME;
+import static org.apache.ignite.internal.processors.query.QueryUtils.VAL_FIELD_NAME;
 
 /**
  * Lucene fulltext index.

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
index 91694ae..5a1d412 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -24,7 +24,6 @@ import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow;
@@ -175,8 +174,15 @@ public final class DmlAstUtils {
         if (filter == null)
             return null;
 
-        if (update.cols().size() != 1 ||
-            !IgniteH2Indexing.VAL_FIELD_NAME.equalsIgnoreCase(update.cols().get(0).columnName()))
+        if (update.cols().size() != 1)
+            return null;
+
+        Table tbl = update.cols().get(0).column().getTable();
+        if (!(tbl instanceof GridH2Table))
+            return null;
+
+        GridH2RowDescriptor desc = ((GridH2Table)tbl).rowDescriptor();
+        if (!desc.isValueColumn(update.cols().get(0).column().getColumnId()))
             return null;
 
         GridSqlElement set = update.set().get(update.cols().get(0).columnName());
@@ -268,19 +274,29 @@ public final class DmlAstUtils {
 
     /**
      * @param op Operation.
-     * @param colName Column name to check.
+     * @param key true - check for key equality condition,
+     *            otherwise check for value equality condition
      * @return Whether this condition is of form {@code colName} = ?
      */
-    private static boolean isEqualityCondition(GridSqlOperation op, String colName) {
+    private static boolean isEqualityCondition(GridSqlOperation op, boolean key) {
         if (op.operationType() != GridSqlOperationType.EQUAL)
             return false;
 
         GridSqlElement left = op.child(0);
         GridSqlElement right = op.child(1);
 
-        return left instanceof GridSqlColumn &&
-            colName.equals(((GridSqlColumn) left).columnName()) &&
-            (right instanceof GridSqlConst || right instanceof GridSqlParameter);
+        if (!(left instanceof GridSqlColumn))
+            return false;
+
+        GridSqlColumn column = (GridSqlColumn)left;
+        if (!(column.column().getTable() instanceof GridH2Table))
+            return false;
+
+        GridH2RowDescriptor desc =((GridH2Table) column.column().getTable()).rowDescriptor();
+
+        return  (key ? desc.isKeyColumn(column.column().getColumnId()) :
+                       desc.isValueColumn(column.column().getColumnId())) &&
+                (right instanceof GridSqlConst || right instanceof GridSqlParameter);
     }
 
     /**
@@ -288,7 +304,7 @@ public final class DmlAstUtils {
      * @return Whether this condition is of form _key = ?
      */
     private static boolean isKeyEqualityCondition(GridSqlOperation op) {
-        return isEqualityCondition(op, IgniteH2Indexing.KEY_FIELD_NAME);
+        return isEqualityCondition(op, true);
     }
 
     /**
@@ -296,7 +312,7 @@ public final class DmlAstUtils {
      * @return Whether this condition is of form _val = ?
      */
     private static boolean isValueEqualityCondition(GridSqlOperation op) {
-        return isEqualityCondition(op, IgniteH2Indexing.VAL_FIELD_NAME);
+        return isEqualityCondition(op, false);
     }
 
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
index 9d18b62..c845496 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java
@@ -292,7 +292,7 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
 
         final SqlQuery<T, Person> qry = new SqlQuery<>(Person.class, "where _key=?");
 
-        final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select * from Person where _key=?");
+        final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select _key, _val, * from Person where _key=?");
 
         qry.setLocal(isLocal());
         fieldsQry.setLocal(isLocal());
@@ -335,7 +335,7 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest
 
         final SqlQuery<Person, T> qry = new SqlQuery<>(valType, "where _val=?");
 
-        final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select * from " + valType.getSimpleName() + " where _val=?");
+        final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select _key, _val, * from " + valType.getSimpleName() + " where _val=?");
 
         qry.setLocal(isLocal());
         fieldsQry.setLocal(isLocal());

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index fedeef6..fed8980 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -249,17 +249,13 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
                     Map<String, String> fields = meta.fields("Person");
 
                     assert fields != null;
-                    assert fields.size() == 5;
+                    assert fields.size() == 3;
 
                     if (binaryMarshaller) {
-                        assert Object.class.getName().equals(fields.get("_KEY"));
-                        assert Object.class.getName().equals(fields.get("_VAL"));
                         assert Integer.class.getName().equals(fields.get("AGE"));
                         assert Integer.class.getName().equals(fields.get("ORGID"));
                     }
                     else {
-                        assert AffinityKey.class.getName().equals(fields.get("_KEY"));
-                        assert Person.class.getName().equals(fields.get("_VAL"));
                         assert int.class.getName().equals(fields.get("AGE"));
                         assert int.class.getName().equals(fields.get("ORGID"));
                     }
@@ -310,18 +306,15 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
                     Map<String, String> fields = meta.fields("Organization");
 
                     assert fields != null;
-                    assertEquals("Fields: " + fields, 4, fields.size());
+                    assertEquals("Fields: " + fields, 2, fields.size());
 
                     if (binaryMarshaller) {
-                        assert Object.class.getName().equals(fields.get("_VAL"));
                         assert Integer.class.getName().equals(fields.get("ID"));
                     }
                     else {
-                        assert Organization.class.getName().equals(fields.get("_VAL"));
                         assert int.class.getName().equals(fields.get("ID"));
                     }
 
-                    assert String.class.getName().equals(fields.get("_KEY"));
                     assert String.class.getName().equals(fields.get("NAME"));
                 }
                 else if (intCache.getName().equals(meta.cacheName())) {
@@ -587,7 +580,7 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
     public void testSelectAllJoined() throws Exception {
         QueryCursor<List<?>> qry =
             personCache.query(sqlFieldsQuery(
-                String.format("select * from \"%s\".Person p, \"%s\".Organization o where p.orgId = o.id",
+                String.format("select p._key, p._val, p.*, o._key, o._val, o.* from \"%s\".Person p, \"%s\".Organization o where p.orgId = o.id",
                     personCache.getName(), orgCache.getName())));
 
         List<List<?>> res = new ArrayList<>(qry.getAll());
@@ -768,7 +761,7 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
 
         cache.put(key, val);
 
-        Collection<List<?>> res = cache.query(sqlFieldsQuery("select * from Person")).getAll();
+        Collection<List<?>> res = cache.query(sqlFieldsQuery("select _key, _val, * from Person")).getAll();
 
         assertEquals(1, res.size());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
index 12662db..92c40b8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
@@ -39,7 +39,7 @@ public class IgniteCacheDeleteSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by id"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by id"));
 
         List<List<?>> leftovers = c.getAll();
 
@@ -63,7 +63,7 @@ public class IgniteCacheDeleteSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by id, _key"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by id, _key"));
 
         List<List<?>> leftovers = c.getAll();
 
@@ -91,7 +91,7 @@ public class IgniteCacheDeleteSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by id"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by id"));
 
         List<List<?>> leftovers = c.getAll();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
index 3626341..20cbe3a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -73,7 +73,7 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by _key, id"));
 
         List<List<?>> leftovers = c.getAll();
 
@@ -103,7 +103,7 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by id, _key"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by id, _key"));
 
         List<List<?>> leftovers = c.getAll();
 
@@ -133,7 +133,7 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by _key, id"));
 
         List<List<?>> leftovers = c.getAll();
 
@@ -163,7 +163,7 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
 
         c.iterator();
 
-        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by _key, id"));
 
         List<List<?>> leftovers = c.getAll();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
index 19d6f54..1f50244 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
@@ -82,6 +82,9 @@ public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
     /** Field 3. */
     protected static final String FIELD_NAME_3 = "field3";
 
+    /** Key alias */
+    protected static final String FIELD_KEY_ALIAS = "key";
+
     /**
      * Get type on the given node for the given cache and table name. Type must exist.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
index 1ed7426..3db3050 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractSelfTest.java
@@ -157,6 +157,9 @@ public abstract class DynamicIndexAbstractSelfTest extends AbstractSchemaSelfTes
         entity.setKeyType(KeyClass.class.getName());
         entity.setValueType(ValueClass.class.getName());
 
+        entity.setKeyFieldName(FIELD_KEY_ALIAS);
+        entity.addQueryField(FIELD_KEY_ALIAS, entity.getKeyType(), null);
+
         entity.addQueryField(FIELD_KEY, Long.class.getName(), null);
         entity.addQueryField(FIELD_NAME_1, Long.class.getName(), null);
         entity.addQueryField(FIELD_NAME_2, Long.class.getName(), null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlKeyValueFieldsTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlKeyValueFieldsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlKeyValueFieldsTest.java
new file mode 100644
index 0000000..d63be7c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlKeyValueFieldsTest.java
@@ -0,0 +1,392 @@
+/*
+ * 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.query;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Test hidden _key, _val, _ver columns
+ */
+public class IgniteSqlKeyValueFieldsTest  extends GridCommonAbstractTest {
+
+    /** IP finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static String NODE_BAD_CONF_MISS_KEY_FIELD = "badConf1";
+    /** */
+    private static String NODE_BAD_CONF_MISS_VAL_FIELD = "badConf2";
+    /** */
+    private static String NODE_CLIENT = "client";
+
+    /** */
+    private static String CACHE_PERSON_NO_KV = "PersonNoKV";
+    /** */
+    private static String CACHE_INT_NO_KV_TYPE = "IntNoKVType";
+    /** */
+    private static String CACHE_PERSON = "Person";
+    /** */
+    private static String CACHE_JOB = "Job";
+
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        c.setDiscoverySpi(disco);
+
+        c.setMarshaller(new BinaryMarshaller());
+
+        List<CacheConfiguration> ccfgs = new ArrayList<>();
+        CacheConfiguration ccfg = buildCacheConfiguration(gridName);
+        if (ccfg != null)
+            ccfgs.add(ccfg);
+
+        ccfgs.add(buildCacheConfiguration(CACHE_PERSON_NO_KV));
+        ccfgs.add(buildCacheConfiguration(CACHE_INT_NO_KV_TYPE));
+        ccfgs.add(buildCacheConfiguration(CACHE_PERSON));
+        ccfgs.add(buildCacheConfiguration(CACHE_JOB));
+
+        c.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
+        if (gridName.equals(NODE_CLIENT))
+            c.setClientMode(true);
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrid(0);
+        startGrid(NODE_CLIENT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    private CacheConfiguration buildCacheConfiguration(String name) {
+        if (name.equals(NODE_BAD_CONF_MISS_KEY_FIELD)) {
+            CacheConfiguration ccfg = new CacheConfiguration(NODE_BAD_CONF_MISS_KEY_FIELD);
+            QueryEntity qe = new QueryEntity(Object.class.getName(), Object.class.getName());
+            qe.setKeyFieldName("k");
+            qe.addQueryField("a", Integer.class.getName(), null);
+            ccfg.setQueryEntities(F.asList(qe));
+            return ccfg;
+        }
+        else if (name.equals(NODE_BAD_CONF_MISS_VAL_FIELD)) {
+            CacheConfiguration ccfg = new CacheConfiguration(NODE_BAD_CONF_MISS_VAL_FIELD);
+            QueryEntity qe = new QueryEntity(Object.class.getName(), Object.class.getName());
+            qe.setValueFieldName("v");
+            qe.addQueryField("a", Integer.class.getName(), null);
+            ccfg.setQueryEntities(F.asList(qe));
+            return ccfg;
+        }
+        else if (name.equals(CACHE_PERSON_NO_KV)) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_PERSON_NO_KV);
+
+            QueryEntity entity = new QueryEntity();
+
+            entity.setKeyType(Integer.class.getName());
+            entity.setValueType(Person.class.getName());
+
+            LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+            fields.put("name", String.class.getName());
+            fields.put("age", Integer.class.getName());
+
+            entity.setFields(fields);
+
+            ccfg.setQueryEntities(Arrays.asList(entity));
+            return ccfg;
+        }
+        else if (name.equals(CACHE_INT_NO_KV_TYPE)) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_INT_NO_KV_TYPE);
+            QueryEntity entity = new QueryEntity();
+
+            entity.setKeyType(null);
+            entity.setValueType(null);
+
+            entity.setKeyFieldName("id");
+            entity.setValueFieldName("v");
+
+            LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+            fields.put("id", Integer.class.getName());
+            fields.put("v", Integer.class.getName());
+
+            entity.setFields(fields);
+
+            ccfg.setQueryEntities(Arrays.asList(entity));
+            return ccfg;
+        }
+        else if (name.equals(CACHE_PERSON)) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_PERSON);
+
+            QueryEntity entity = new QueryEntity();
+
+            entity.setKeyType(Integer.class.getName());
+            entity.setValueType(Person.class.getName());
+
+            entity.setKeyFieldName("id");
+            entity.setValueFieldName("v");
+
+            LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+            fields.put("name", String.class.getName());
+            fields.put("age", Integer.class.getName());
+
+            fields.put(entity.getKeyFieldName(), entity.getKeyType());
+            fields.put(entity.getValueFieldName(), entity.getValueType());
+
+            entity.setFields(fields);
+
+            ccfg.setQueryEntities(Arrays.asList(entity));
+            return ccfg;
+        }
+        else if (name.equals(CACHE_JOB)) {
+            CacheConfiguration ccfg = new CacheConfiguration(CACHE_JOB);
+            ccfg.setIndexedTypes(Integer.class, Integer.class);
+            return ccfg;
+        }
+        return null;
+    }
+
+    /** Test for setIndexedTypes() primitive types */
+    public void testSetIndexTypesPrimitive() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(NODE_CLIENT).cache(CACHE_JOB);
+
+        checkInsert(cache, "insert into Integer (_key, _val) values (?,?)", 1, 100);
+
+        checkSelect(cache, "select * from Integer", 1, 100);
+        checkSelect(cache, "select _key, _val from Integer", 1, 100);
+    }
+
+    /** Test configuration error : keyFieldName is missing from fields */
+    public void testErrorKeyFieldMissingFromFields() throws Exception {
+        checkCacheStartupError(NODE_BAD_CONF_MISS_KEY_FIELD);
+    }
+
+    /** Test configuration error : valueFieldName is missing from fields */
+    public void testErrorValueFieldMissingFromFields() throws Exception {
+        checkCacheStartupError(NODE_BAD_CONF_MISS_VAL_FIELD);
+    }
+
+    /** */
+    private void checkCacheStartupError(final String name) {
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(name);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+    }
+
+    /**
+     * Check that it is allowed to leave QE.keyType and QE.valueType unset
+     * in case keyFieldName and valueFieldName are set and present in fields
+     */
+    public void testQueryEntityAutoKeyValTypes() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(NODE_CLIENT).cache(CACHE_INT_NO_KV_TYPE);
+
+        checkInsert(cache, "insert into Integer (_key, _val) values (?,?)", 1, 100);
+
+        checkSelect(cache, "select * from Integer where id = 1", 1, 100);
+
+        checkSelect(cache, "select * from Integer", 1, 100);
+        checkSelect(cache, "select _key, _val from Integer", 1, 100);
+        checkSelect(cache, "select id, v from Integer", 1, 100);
+    }
+
+    /** Check that it is possible to not have keyFieldName and valueFieldName */
+    public void testNoKeyValueAliases() throws Exception {
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON_NO_KV);
+
+        Person alice = new Person("Alice", 1);
+        checkInsert(cache, "insert into Person (_key, _val) values (?,?)", 1, alice);
+
+        checkSelect(cache, "select * from Person", alice.name, alice.age);
+        checkSelect(cache, "select _key, _val from Person", 1, alice);
+    }
+
+    /** Check keyFieldName and valueFieldName columns access */
+    public void testKeyValueAlias() throws Exception {
+        //_key, _val, _ver | name, age, id, v
+        Person alice = new Person("Alice", 1);
+        Person bob = new Person("Bob", 2);
+
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON);
+
+        checkInsert(cache, "insert into Person (_key, _val) values (?,?)", 1, alice);
+        checkInsert(cache, "insert into Person (id, v) values (?,?)", 2, bob);
+
+        checkSelect(cache, "select * from Person where _key=1", alice.name, alice.age, 1, alice);
+        checkSelect(cache, "select _key, _val from Person where id=1", 1, alice);
+
+        checkSelect(cache, "select * from Person where _key=2", bob.name, bob.age, 2, bob);
+        checkSelect(cache, "select _key, _val from Person where id=2", 2, bob);
+
+        checkInsert(cache, "update Person set age = ? where id = ?", 3, 1);
+        checkSelect(cache, "select _key, age from Person where id=1", 1, 3);
+
+        checkInsert(cache, "update Person set v = ? where id = ?", alice, 1);
+        checkSelect(cache, "select _key, _val from Person where id=1", 1, alice);
+    }
+
+    /** Check _ver version field is accessible */
+    public void testVersionField() throws Exception {
+        Person alice = new Person("Alice", 1);
+        Person bob = new Person("Bob", 2);
+
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON);
+
+        checkInsert(cache, "insert into Person (id, v) values (?,?)", 1, alice);
+        assertNotNull(getVersion(cache, 1));
+
+        checkInsert(cache, "insert into Person (id, v) values (?,?)", 2, bob);
+        assertNotNull(getVersion(cache, 2));
+
+        GridCacheVersion v1 = getVersion(cache, 1);
+
+        checkInsert(cache, "update Person set age = ? where id = ?", 3, 1);
+
+        GridCacheVersion v2 = getVersion(cache, 1);
+
+        assertFalse( v1.equals(v2) );
+    }
+
+    /** Check that joins are working on keyFieldName, valueFieldName columns */
+    public void testJoinKeyValFields() throws Exception {
+        IgniteEx client = grid(NODE_CLIENT);
+        IgniteCache<Integer, Person> cache = client.cache(CACHE_PERSON);
+        IgniteCache<Integer, Integer> cache2 = client.cache(CACHE_JOB);
+
+        checkInsert(cache, "insert into Person (id, v) values (?, ?)", 1, new Person("Bob", 30));
+        checkInsert(cache, "insert into Person (id, v) values (?, ?)", 2, new Person("David", 35));
+        checkInsert(cache2, "insert into Integer (_key, _val) values (?, ?)", 100, 1);
+        checkInsert(cache2, "insert into Integer (_key, _val) values (?, ?)", 200, 2);
+
+        QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("select p.id, j._key from Person p, \""+ CACHE_JOB +"\".Integer j where p.id = j._val"));
+        List<List<?>> results = cursor.getAll();
+        assertEquals(2, results.size());
+        assertEquals(1, results.get(0).get(0));
+        assertEquals(100, results.get(0).get(1));
+        assertEquals(2, results.get(1).get(0));
+        assertEquals(200, results.get(1).get(1));
+    }
+
+    /** Check automatic addition of index for keyFieldName column */
+    public void testAutoKeyFieldIndex() throws Exception {
+        IgniteEx client = grid(NODE_CLIENT);
+        IgniteCache<Integer, Person> cache = client.cache(CACHE_PERSON);
+
+        QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("explain select * from Person where id = 1"));
+        List<List<?>> results = cursor.getAll();
+        assertEquals(2, results.size());
+        assertTrue(((String)results.get(0).get(0)).contains("\"_key_PK_proxy\""));
+
+        cursor = cache.query(new SqlFieldsQuery("explain select * from Person where _key = 1"));
+        results = cursor.getAll();
+        assertEquals(2, results.size());
+        assertTrue(((String)results.get(0).get(0)).contains("\"_key_PK\""));
+    }
+
+    /** */
+    private GridCacheVersion getVersion(IgniteCache<?, ?> cache, int key) {
+        QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery("select _ver from Person where id = ?").setArgs(key));
+        List<List<?>> results = cursor.getAll();
+        assertEquals(1, results.size());
+        return ((GridCacheVersion) results.get(0).get(0));
+    }
+
+    /** */
+    private void checkInsert(IgniteCache<?, ?> cache, String qry, Object ... args) throws Exception {
+        QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery(qry).setArgs(args));
+        assertEquals(1, ((Number) cursor.getAll().get(0).get(0)).intValue());
+    }
+
+    /** */
+    private void checkSelect(IgniteCache<?, ?> cache, String selectQry, Object ... expected) {
+        QueryCursor<List<?>> cursor = cache.query(new SqlFieldsQuery(selectQry));
+
+        List<List<?>> results = cursor.getAll();
+
+        assertEquals(1, results.size());
+
+        List<?> row0 = results.get(0);
+        for(int col = 0; col < expected.length; ++col)
+            assertEquals(expected[col], row0.get(col));
+    }
+
+    /** */
+    private static class Person {
+        /** */
+        private String name;
+
+        /** */
+        private int age;
+
+        /** */
+        public Person(String name, int age) {
+            this.name = name;
+            this.age = age;
+        }
+
+        /** */
+        @Override public int hashCode() {
+            return name.hashCode() ^ age;
+        }
+
+        /** */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (!(o instanceof Person))
+                return false;
+            Person other = (Person)o;
+            return name.equals(other.name) && age == other.age;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index fa4eded..6110faa 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -799,14 +799,14 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
                 false,
                 0,
                 "select p._key k1, o._key k2 " +
-                    "from \"persPart\".Person2 p, (select * from \"orgRepl\".Organization) o " +
+                    "from \"persPart\".Person2 p, (select _key, _val, * from \"orgRepl\".Organization) o " +
                     "where p.orgId = o._key");
 
             checkQueryPlan(persPart,
                 false,
                 0,
                 "select p._key k1, o._key k2 " +
-                    "from (select * from \"orgRepl\".Organization) o, \"persPart\".Person2 p " +
+                    "from (select _key, _val, * from \"orgRepl\".Organization) o, \"persPart\".Person2 p " +
                     "where p.orgId = o._key");
 
             checkQueryPlan(persPart,
@@ -1019,7 +1019,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
                     sql);
 
                 sql = "select p1._key k1, p2._key k2, o._key k3 " +
-                    "from (select * from \"orgRepl\".Organization) o, \"persPartAff\".Person2 p1, \"persPart\".Person2 p2 " +
+                    "from (select _key, _val, * from \"orgRepl\".Organization) o, \"persPartAff\".Person2 p1, \"persPart\".Person2 p2 " +
                     "where p1._key=p2.name and p2.orgId = o._key";
 
                 checkQueryPlan(persPart,
@@ -1029,7 +1029,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
                     "batched:unicast");
 
                 sql = "select p1._key k1, p2._key k2, o._key k3 " +
-                    "from (select * from \"orgRepl\".Organization) o, \"persPartAff\".Person2 p1, \"persPart\".Person2 p2 " +
+                    "from (select _key, _val, * from \"orgRepl\".Organization) o, \"persPartAff\".Person2 p1, \"persPart\".Person2 p2 " +
                     "where p1._key=p2._key and p2.orgId = o._key";
 
                 checkQueryPlan(persPart,

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/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 e62199a..7e2026b 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
@@ -682,6 +682,16 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         @Override public int typeId() {
             return 0;
         }
+
+        /** {@inheritDoc} */
+        @Override public String keyFieldName() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String valueFieldName() {
+            return null;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index 9782f28..7423c9b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -506,7 +506,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
 
         // Schemas for index and table must match
         assertParseThrows("create index if not exists sch2.idx on sch1.Person (name)",
-            DbException.class, "Schema name must match [90080-195]");
+            DbException.class, "Schema name must match");
 
         assertParseThrows("create hash index if not exists idx on Person (name)",
             IgniteSQLException.class, "Only SPATIAL modifier is supported for CREATE INDEX");

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 032e544..6370cd0 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -108,6 +108,7 @@ import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTes
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteQueryDedicatedPoolTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlEntryCacheModeAgnosticTest;
+import org.apache.ignite.internal.processors.query.IgniteSqlKeyValueFieldsTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexSelfTest;
@@ -283,6 +284,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheDistributedPartitionQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheDistributedPartitionQueryNodeRestartsSelfTest.class);
         suite.addTestSuite(IgniteCacheDistributedPartitionQueryConfigurationSelfTest.class);
+        suite.addTestSuite(IgniteSqlKeyValueFieldsTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/cpp/core-test/config/cache-query-default.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query-default.xml b/modules/platforms/cpp/core-test/config/cache-query-default.xml
index 366435c..1c1e5f3 100644
--- a/modules/platforms/cpp/core-test/config/cache-query-default.xml
+++ b/modules/platforms/cpp/core-test/config/cache-query-default.xml
@@ -50,8 +50,14 @@
                         <list>
                             <bean class="org.apache.ignite.cache.QueryEntity">
                                 <property name="valueType" value="QueryPerson"/>
+
+                                <property name="keyFieldName" value="k"/>
+                                <property name="valueFieldName" value="v"/>
+
                                 <property name="fields">
                                     <map>
+                                        <entry key="k" value="java.lang.Integer"/>
+                                        <entry key="v" value="QueryPerson"/>
                                         <entry key="name" value="java.lang.String"/>
                                         <entry key="age" value="java.lang.Integer"/>
                                         <entry key="birthday" value="java.util.Date"/>

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/cpp/core-test/src/cache_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
index 259a9a1..e9c4610 100644
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@ -163,6 +163,29 @@ public:
         return recordCreated;
     }
 
+    /**
+     * @return true if objects are equal.
+     */
+    friend bool operator==(QueryPerson const& lhs, QueryPerson const& rhs)
+    {
+        return lhs.GetName() == rhs.GetName() && lhs.GetAge() == rhs.GetAge() &&
+            lhs.GetBirthday() == rhs.GetBirthday() && lhs.GetCreationTime() == rhs.GetCreationTime();
+    }
+
+    /**
+     * Outputs the object to stream.
+     *
+     * @return Stream.
+     */
+    friend std::ostream& operator<<(std::ostream& str, QueryPerson const& obj)
+    {
+        str << "QueryPerson::name: " << obj.GetName()
+            << "QueryPerson::age: " << obj.GetAge()
+            << "QueryPerson::birthday: " << obj.GetBirthday().GetMilliseconds()
+            << "QueryPerson::recordCreated: " << obj.GetCreationTime().GetSeconds() << "." << obj.GetCreationTime().GetSecondFraction();
+        return str;
+    }
+
 private:
     /** Name. */
     char* name;
@@ -1811,4 +1834,63 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryPageZero)
     BOOST_CHECK_THROW(CheckFieldsQueryPages(0, 100, 0), IgniteError);
 }
 
+/**
+ * Test query for key and value fields.
+ */
+BOOST_AUTO_TEST_CASE(TestKeyValFields)
+{
+    Cache<int, QueryPerson> cache = GetPersonCache();
+
+    QueryPerson person("John", 30, MakeDateGmt(1987), MakeTimestampGmt(2017, 1, 1, 1, 1));
+
+    cache.Put(1, person);
+
+    for (int i = 0; i < 2; i++)
+    {
+        SqlFieldsQuery qry(i == 0 ?
+            "select _key, _val, k, v, name, age, birthday, recordCreated from QueryPerson" :
+            "select _key, _val, * from QueryPerson");
+
+        QueryFieldsCursor cursor = cache.Query(qry);
+
+        BOOST_REQUIRE(cursor.HasNext());
+
+        QueryFieldsRow row = cursor.GetNext();
+
+        BOOST_REQUIRE(row.HasNext());
+        int id = row.GetNext<int>();
+        BOOST_CHECK_EQUAL(1, id);
+
+        BOOST_REQUIRE(row.HasNext());
+        QueryPerson p = row.GetNext<QueryPerson>();
+        BOOST_CHECK_EQUAL(p, person);
+
+        BOOST_REQUIRE(row.HasNext());
+        id = row.GetNext<int>();
+        BOOST_CHECK_EQUAL(1, id);
+
+        BOOST_REQUIRE(row.HasNext());
+        p = row.GetNext<QueryPerson>();
+        BOOST_CHECK_EQUAL(p, person);
+
+        BOOST_REQUIRE(row.HasNext());
+        std::string name = row.GetNext<std::string>();
+        BOOST_CHECK_EQUAL(name, person.GetName());
+
+        BOOST_REQUIRE(row.HasNext());
+        int age = row.GetNext<int>();
+        BOOST_CHECK_EQUAL(age, person.GetAge());
+
+        BOOST_REQUIRE(row.HasNext());
+        Date birthday = row.GetNext<Date>();
+        BOOST_CHECK(birthday == person.GetBirthday());
+
+        BOOST_REQUIRE(row.HasNext());
+        Timestamp recordCreated = row.GetNext<Timestamp>();
+        BOOST_CHECK(recordCreated == person.GetCreationTime());
+
+        BOOST_CHECK(!row.HasNext());
+    }
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/cpp/odbc-test/config/queries-default.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-default.xml b/modules/platforms/cpp/odbc-test/config/queries-default.xml
index c1eaea8..56040f1 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-default.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-default.xml
@@ -95,8 +95,13 @@
                                 <property name="keyType" value="java.lang.Long"/>
                                 <property name="valueType" value="ComplexType"/>
 
+                                <property name="keyFieldName" value="k"/>
+                                <property name="valueFieldName" value="v"/>
+
                                 <property name="fields">
                                     <map>
+                                        <entry key="k" value="java.lang.Long"/>
+                                        <entry key="v" value="ComplexType"/>
                                         <entry key="i32Field" value="java.lang.Integer"/>
                                         <entry key="objField" value="TestObject"/>
                                         <entry key="strField" value="java.lang.String"/>

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/cpp/odbc-test/include/complex_type.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/complex_type.h b/modules/platforms/cpp/odbc-test/include/complex_type.h
index d844cd5..8a1bd59 100644
--- a/modules/platforms/cpp/odbc-test/include/complex_type.h
+++ b/modules/platforms/cpp/odbc-test/include/complex_type.h
@@ -50,6 +50,31 @@ namespace ignite
         TestObject objField;
         std::string strField;
     };
+
+    bool operator==(TestObject const& lhs, TestObject const& rhs)
+    {
+        return lhs.f1 == rhs.f1 && lhs.f2 == rhs.f2;
+    }
+
+    bool operator==(ComplexType const& lhs, ComplexType const& rhs)
+    {
+        return lhs.i32Field == rhs.i32Field && lhs.objField == rhs.objField && lhs.strField == rhs.strField;
+    }
+
+    std::ostream& operator<<(std::ostream& str, TestObject const& obj)
+    {
+        str << "TestObject::f1: " << obj.f1
+            << "TestObject::f2: " << obj.f2;
+        return str;
+    }
+
+    std::ostream& operator<<(std::ostream& str, ComplexType const& obj)
+    {
+        str << "ComplexType::i32Field: " << obj.i32Field
+            << "ComplexType::objField: " << obj.objField
+            << "ComplexType::strField: " << obj.strField;
+        return str;
+    }
 }
 
 namespace ignite

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index eb0904f..fdf3503 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -35,6 +35,7 @@
 #include "ignite/ignite.h"
 #include "ignite/ignition.h"
 #include "ignite/impl/binary/binary_utils.h"
+#include "ignite/binary/binary_object.h"
 
 #include "test_type.h"
 #include "complex_type.h"
@@ -45,6 +46,9 @@ using namespace ignite::cache;
 using namespace ignite::cache::query;
 using namespace ignite::common;
 using namespace ignite_test;
+using namespace ignite::binary;
+using namespace ignite::impl::binary;
+using namespace ignite::impl::interop;
 
 using namespace boost::unit_test;
 
@@ -1414,4 +1418,148 @@ BOOST_AUTO_TEST_CASE(TestTablesMeta)
     BOOST_CHECK(ret == SQL_NO_DATA);
 }
 
+template<typename T>
+void CheckObjectData(int8_t* data, int32_t len, T const& value)
+{
+    InteropUnpooledMemory mem(len);
+    mem.Length(len);
+    memcpy(mem.Data(), data, len);
+
+    BinaryObject obj(BinaryObjectImpl::FromMemory(mem, 0, 0));
+
+    T actual = obj.Deserialize<T>();
+    
+    BOOST_CHECK_EQUAL(value, actual);
+}
+
+BOOST_AUTO_TEST_CASE(TestKeyVal)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache2");
+
+    SQLRETURN ret;
+
+    ComplexType obj;
+
+    obj.i32Field = 123;
+    obj.strField = "Some string";
+
+    obj.objField.f1 = 54321;
+    obj.objField.f2 = "Hello Ignite";
+
+    cache2.Put(1, obj);
+
+    //_key
+    int64_t column1 = 0;
+    //_val
+    int8_t column2[ODBC_BUFFER_SIZE] = { 0 };
+    //k
+    int64_t column3 = 0;
+    //v
+    int8_t column4[ODBC_BUFFER_SIZE] = { 0 };
+    //i32Field
+    int64_t column5 = 0;
+    //objField
+    int8_t column6[ODBC_BUFFER_SIZE] = { 0 };
+    //strField
+    char column7[ODBC_BUFFER_SIZE] = { 0 };
+    
+    SQLLEN column1Len = sizeof(column1);
+    SQLLEN column2Len = sizeof(column2);
+    SQLLEN column3Len = sizeof(column3);
+    SQLLEN column4Len = sizeof(column4);
+    SQLLEN column5Len = sizeof(column5);
+    SQLLEN column6Len = sizeof(column6);
+    SQLLEN column7Len = sizeof(column7);
+
+    // Binding columns.
+    ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &column1, column1Len, &column1Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLBindCol(stmt, 2, SQL_C_BINARY, &column2, column2Len, &column2Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLBindCol(stmt, 3, SQL_C_SLONG, &column3, column3Len, &column3Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLBindCol(stmt, 4, SQL_C_BINARY, &column4, column4Len, &column4Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLBindCol(stmt, 5, SQL_C_SLONG, &column5, column5Len, &column5Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLBindCol(stmt, 6, SQL_C_BINARY, &column6, column6Len, &column6Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLBindCol(stmt, 7, SQL_C_CHAR, &column7, column7Len, &column7Len);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    SQLCHAR request[] = "SELECT _key, _val, k, v, i32Field, objField, strField FROM ComplexType";
+
+    ret = SQLExecDirect(stmt, request, SQL_NTS);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    BOOST_CHECK_EQUAL(column1, 1);
+
+    CheckObjectData(column2, static_cast<int32_t>(column2Len), obj);
+
+    BOOST_CHECK_EQUAL(column3, 1);
+
+    CheckObjectData(column4, static_cast<int32_t>(column4Len), obj);
+
+    BOOST_CHECK_EQUAL(column5, obj.i32Field);
+    
+    CheckObjectData(column6, static_cast<int32_t>(column6Len), obj.objField);
+
+    BOOST_CHECK_EQUAL(column7, obj.strField);
+
+    ret = SQLFetch(stmt);
+    BOOST_CHECK(ret == SQL_NO_DATA);
+
+    SQLCHAR requestStar[] = "SELECT _key, _val, * FROM ComplexType";
+
+    ret = SQLExecDirect(stmt, requestStar, SQL_NTS);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    BOOST_CHECK_EQUAL(column1, 1);
+
+    CheckObjectData(column2, static_cast<int32_t>(column2Len), obj);
+
+    BOOST_CHECK_EQUAL(column3, 1);
+
+    CheckObjectData(column4, static_cast<int32_t>(column4Len), obj);
+
+    BOOST_CHECK_EQUAL(column5, obj.i32Field);
+
+    CheckObjectData(column6, static_cast<int32_t>(column6Len), obj.objField);
+
+    BOOST_CHECK_EQUAL(column7, obj.strField);
+
+    ret = SQLFetch(stmt);
+    BOOST_CHECK(ret == SQL_NO_DATA);
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index b2213d8..c638449 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -1423,6 +1423,13 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                         {
                             new QueryAlias("AliasTest", "Person_AliasTest"),
                             new QueryAlias("Address.AliasTest", "Addr_AliasTest")
+                        },
+                        KeyFieldName = "MyKey",
+                        ValueFieldName = "MyValue",
+                        Fields =
+                        {
+                            new QueryField("MyKey", typeof(int)),
+                            new QueryField("MyValue", typeof(T)),
                         }
                     },
                     new QueryEntity(typeof (int), typeof (Organization))) {CacheMode = CacheMode.Replicated});

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
index 0b13bf9..1fa993b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -622,6 +622,58 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests custom key and value field names.
+        /// </summary>
+        [Test]
+        public void TestCustomKeyValueFieldNames()
+        {
+            // Check select * with default config - does not include _key, _val.
+            var cache = Cache();
+
+            cache[1] = new QueryPerson("Joe", 48);
+
+            var row = cache.QueryFields(new SqlFieldsQuery("select * from QueryPerson")).GetAll()[0];
+            Assert.AreEqual(2, row.Count);
+            Assert.AreEqual(48, row[0]);
+            Assert.AreEqual("Joe", row[1]);
+
+            // Check select * with custom names - fields are included.
+            cache = GetIgnite().GetOrCreateCache<int, QueryPerson>(
+                new CacheConfiguration("customKeyVal")
+                {
+                    QueryEntities = new[]
+                    {
+                        new QueryEntity(typeof(int), typeof(QueryPerson))
+                        {
+                            Fields = new[]
+                            {
+                                new QueryField("age", "int"),
+                                new QueryField("FullKey", "int"),
+                                new QueryField("FullVal", "QueryPerson")
+                            },
+                            KeyFieldName = "FullKey",
+                            ValueFieldName = "FullVal"
+                        }
+                    }
+                });
+
+            cache[1] = new QueryPerson("John", 33);
+
+            row = cache.QueryFields(new SqlFieldsQuery("select * from QueryPerson")).GetAll()[0];
+            
+            Assert.AreEqual(3, row.Count);
+            Assert.AreEqual(33, row[0]);
+            Assert.AreEqual(1, row[1]);
+
+            var person = (QueryPerson) row[2];
+            Assert.AreEqual("John", person.Name);
+
+            // Check explicit select.
+            row = cache.QueryFields(new SqlFieldsQuery("select FullKey from QueryPerson")).GetAll()[0];
+            Assert.AreEqual(1, row[0]);
+        }
+
+        /// <summary>
         /// Validates the query results.
         /// </summary>
         /// <param name="cache">Cache.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 0524c05..2600028 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -661,7 +661,9 @@ namespace Apache.Ignite.Core.Tests
                                 },
                                 KeyType = typeof (string),
                                 ValueType = typeof (long),
-                                TableName = "table-1"
+                                TableName = "table-1",
+                                KeyFieldName = "k",
+                                ValueFieldName = "v"
                             },
                         },
                         ReadFromBackup = false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index 8aadf0e..b6163ee 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -148,6 +148,20 @@ namespace Apache.Ignite.Core.Cache.Configuration
         }
 
         /// <summary>
+        /// Gets or sets the name of the field that is used to denote the entire key.
+        /// <para />
+        /// By default, entite key can be accessed with a special "_key" field name.
+        /// </summary>
+        public string KeyFieldName { get; set; }
+
+        /// <summary>
+        /// Gets or sets the name of the field that is used to denote the entire value.
+        /// <para />
+        /// By default, entite value can be accessed with a special "_val" field name.
+        /// </summary>
+        public string ValueFieldName { get; set; }
+
+        /// <summary>
         /// Gets or sets the name of the SQL table.
         /// When not set, value type name is used.
         /// </summary>
@@ -197,6 +211,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
             count = reader.ReadInt();
             Indexes = count == 0 ? null : Enumerable.Range(0, count).Select(x => new QueryIndex(reader)).ToList();
+
+            KeyFieldName = reader.ReadString();
+            ValueFieldName = reader.ReadString();
         }
 
         /// <summary>
@@ -250,6 +267,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
             }
             else
                 writer.WriteInt(0);
+
+            writer.WriteString(KeyFieldName);
+            writer.WriteString(ValueFieldName);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 728e62b..76c2068 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -417,6 +417,20 @@
                                                                     </xs:documentation>
                                                                 </xs:annotation>
                                                             </xs:attribute>
+                                                            <xs:attribute name="keyFieldName" type="xs:string">
+                                                                <xs:annotation>
+                                                                    <xs:documentation>
+                                                                        Name of the field that denotes the key.
+                                                                    </xs:documentation>
+                                                                </xs:annotation>
+                                                            </xs:attribute>
+                                                            <xs:attribute name="valueFieldName" type="xs:string">
+                                                                <xs:annotation>
+                                                                    <xs:documentation>
+                                                                        Name of the field that denotes the value.
+                                                                    </xs:documentation>
+                                                                </xs:annotation>
+                                                            </xs:attribute>
                                                         </xs:complexType>
                                                     </xs:element>
                                                 </xs:sequence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
index 3659158..2d14ec4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -51,18 +51,24 @@ namespace Apache.Ignite.Linq.Impl
         private static readonly CopyOnWriteConcurrentDictionary<MemberInfo, string> FieldNameMap =
             new CopyOnWriteConcurrentDictionary<MemberInfo, string>();
 
+        /** */
+        private readonly bool _includeAllFields;
+
         /// <summary>
         /// Initializes a new instance of the <see cref="CacheQueryExpressionVisitor" /> class.
         /// </summary>
         /// <param name="modelVisitor">The _model visitor.</param>
         /// <param name="useStar">Flag indicating that star '*' qualifier should be used
         /// for the whole-table select instead of _key, _val.</param>
-        public CacheQueryExpressionVisitor(CacheQueryModelVisitor modelVisitor, bool useStar)
+        /// <param name="includeAllFields">Flag indicating that star '*' qualifier should be used
+        /// for the whole-table select as well as _key, _val.</param>
+        public CacheQueryExpressionVisitor(CacheQueryModelVisitor modelVisitor, bool useStar, bool includeAllFields)
         {
             Debug.Assert(modelVisitor != null);
 
             _modelVisitor = modelVisitor;
             _useStar = useStar;
+            _includeAllFields = includeAllFields;
         }
 
         /// <summary>
@@ -271,7 +277,11 @@ namespace Apache.Ignite.Linq.Impl
         {
             // Count, sum, max, min expect a single field or *
             // In other cases we need both parts of cache entry
-            var format = _useStar ? "{0}.*" : "{0}._key, {0}._val";
+            var format = _includeAllFields
+                ? "{0}.*, {0}._key, {0}._val"
+                : _useStar
+                    ? "{0}.*"
+                    : "{0}._key, {0}._val";
 
             var tableName = Aliases.GetTableAlias(expression);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/efb7abce/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
index 2cf4420..f74ccc7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs
@@ -107,7 +107,7 @@ namespace Apache.Ignite.Linq.Impl
         /// <summary>
         /// Visits the query model.
         /// </summary>
-        private void VisitQueryModel(QueryModel queryModel, bool forceStar)
+        private void VisitQueryModel(QueryModel queryModel, bool includeAllFields)
         {
             _aliases.Push();
 
@@ -115,7 +115,7 @@ namespace Apache.Ignite.Linq.Impl
             _builder.Append("select ");
 
             // TOP 1 FLD1, FLD2
-            VisitSelectors(queryModel, forceStar);
+            VisitSelectors(queryModel, includeAllFields);
 
             // FROM ... WHERE ... JOIN ...
             base.VisitQueryModel(queryModel);
@@ -129,14 +129,14 @@ namespace Apache.Ignite.Linq.Impl
         /// <summary>
         /// Visits the selectors.
         /// </summary>
-        public void VisitSelectors(QueryModel queryModel, bool forceStar)
+        public void VisitSelectors(QueryModel queryModel, bool includeAllFields)
         {
             var parenCount = ProcessResultOperatorsBegin(queryModel);
 
             if (parenCount >= 0)
             {
                 // FIELD1, FIELD2
-                BuildSqlExpression(queryModel.SelectClause.Selector, forceStar || parenCount > 0);
+                BuildSqlExpression(queryModel.SelectClause.Selector, parenCount > 0, includeAllFields);
                 _builder.Append(')', parenCount).Append(" ");
             }
         }
@@ -512,9 +512,9 @@ namespace Apache.Ignite.Linq.Impl
         /// <summary>
         /// Builds the SQL expression.
         /// </summary>
-        private void BuildSqlExpression(Expression expression, bool useStar = false)
+        private void BuildSqlExpression(Expression expression, bool useStar = false, bool includeAllFields = false)
         {
-            new CacheQueryExpressionVisitor(this, useStar).Visit(expression);
+            new CacheQueryExpressionVisitor(this, useStar, includeAllFields).Visit(expression);
         }
     }
 }