You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2019/01/23 09:13:00 UTC

[ignite] branch master updated: IGNITE-10784: SQL: Implemented "TABLES" system view. This closes #5749.

This is an automated email from the ASF dual-hosted git repository.

vozerov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 15e112a  IGNITE-10784: SQL: Implemented "TABLES" system view. This closes #5749.
15e112a is described below

commit 15e112af75c5b5043ba6ab53d8cb71f89bf645d7
Author: Pavel Kuznetsov <pa...@gmail.com>
AuthorDate: Wed Jan 23 12:12:48 2019 +0300

    IGNITE-10784: SQL: Implemented "TABLES" system view. This closes #5749.
---
 .../processors/query/h2/SchemaManager.java         |  38 ++--
 .../processors/query/h2/opt/GridH2Table.java       |   2 +-
 .../query/h2/sys/view/SqlSystemViewTables.java     | 129 +++++++++++
 .../processors/query/SqlSystemViewsSelfTest.java   | 245 ++++++++++++++++++++-
 4 files changed, 393 insertions(+), 21 deletions(-)

diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
index 42d1680..5f692a6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
@@ -17,6 +17,20 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
@@ -42,27 +56,13 @@ import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewCach
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeAttributes;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeMetrics;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodes;
+import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewTables;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.index.Index;
 import org.jetbrains.annotations.Nullable;
 
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
 /**
  * Schema manager. Responsible for all manipulations on schema objects.
  */
@@ -166,6 +166,7 @@ public class SchemaManager {
         views.add(new SqlSystemViewNodeMetrics(ctx));
         views.add(new SqlSystemViewCaches(ctx));
         views.add(new SqlSystemViewCacheGroups(ctx));
+        views.add(new SqlSystemViewTables(ctx, this));
 
         return views;
     }
@@ -696,6 +697,13 @@ public class SchemaManager {
     }
 
     /**
+     * @return all known tables.
+     */
+    public Collection<GridH2Table> dataTables() {
+        return dataTables.values();
+    }
+
+    /**
      * Find table for index.
      *
      * @param schemaName Schema name.
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 fff12e3..4a342a5 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
@@ -242,7 +242,7 @@ public class GridH2Table extends TableBase {
      * for external affinity calculation (e.g. on thin clients).
      * <p>
      * Note that it is different from {@link #isColumnForPartitionPruning(Column)} method in that not every column
-     * which qualifies for partition pruning can be used by thin clients or join partinion prunining logic.
+     * which qualifies for partition pruning can be used by thin clients or join partition pruning logic.
      * <p>
      * Consider the following schema:
      * <pre>
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewTables.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewTables.java
new file mode 100644
index 0000000..8a4d10c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewTables.java
@@ -0,0 +1,129 @@
+/*
+ * 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.h2.sys.view;
+
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.h2.SchemaManager;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.h2.engine.Session;
+import org.h2.result.Row;
+import org.h2.result.SearchRow;
+import org.h2.table.IndexColumn;
+import org.h2.value.Value;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * View that contains information about all the sql tables in the cluster.
+ */
+public class SqlSystemViewTables extends SqlAbstractLocalSystemView {
+    /** Name of the column that contains names of sql tables. */
+    private static final String TABLE_NAME = "TABLE_NAME";
+
+    /** Schema manager. */
+    private final SchemaManager schemaMgr;
+
+    /**
+     * Creates view with columns.
+     *
+     * @param ctx kernal context.
+     */
+    public SqlSystemViewTables(GridKernalContext ctx, SchemaManager schemaMgr) {
+        super("TABLES", "Ignite tables", ctx, TABLE_NAME,
+            newColumn("SCHEMA_NAME"),
+            newColumn(TABLE_NAME),
+            newColumn("CACHE_NAME"),
+            newColumn("CACHE_ID", Value.INT),
+            newColumn("AFFINITY_KEY_COLUMN"),
+            newColumn("KEY_ALIAS"),
+            newColumn("VALUE_ALIAS"),
+            newColumn("KEY_TYPE_NAME"),
+            newColumn("VALUE_TYPE_NAME")
+        );
+
+        this.schemaMgr = schemaMgr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Row> getRows(Session ses, SearchRow first, SearchRow last) {
+        SqlSystemViewColumnCondition nameCond = conditionForColumn(TABLE_NAME, first, last);
+
+        Predicate<GridH2Table> filter;
+
+        if (nameCond.isEquality()) {
+            String tblName = nameCond.valueForEquality().getString();
+
+            filter = tbl -> tblName.equals(tbl.getName());
+        }
+        else
+            filter = tab -> true;
+
+        final AtomicLong keys = new AtomicLong();
+
+        return schemaMgr.dataTables().stream()
+            .filter(filter)
+            .map(tbl -> {
+                    Object[] data = new Object[] {
+                        tbl.getSchema().getName(),
+                        tbl.getName(),
+                        tbl.cacheName(),
+                        tbl.cacheId(),
+                        computeAffinityColumn(tbl),
+                        tbl.rowDescriptor().type().keyFieldAlias(),
+                        tbl.rowDescriptor().type().valueFieldAlias(),
+                        tbl.rowDescriptor().type().keyTypeName(),
+                        tbl.rowDescriptor().type().valueTypeName()
+                    };
+
+                    return createRow(ses, keys.incrementAndGet(), data);
+                }
+            ).iterator();
+    }
+
+    /**
+     * Computes affinity column for the specified table.
+     *
+     * @param tbl Table.
+     * @return "_KEY" for default (all PK), {@code null} if custom mapper specified or name of the desired column
+     * otherwise.
+     */
+    private @Nullable String computeAffinityColumn(GridH2Table tbl) {
+        IndexColumn affCol = tbl.getAffinityKeyColumn();
+
+        if (affCol == null)
+            return null;
+
+        // Only explicit affinity column should be shown. Do not do this for _KEY or it's alias.
+        if (tbl.rowDescriptor().isKeyColumn(affCol.column.getColumnId()))
+            return null;
+
+        return affCol.columnName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canGetRowCount() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCount() {
+        return schemaMgr.dataTables().size();
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
index 2d27762..90114ef 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cache.eviction.EvictableEntry;
 import org.apache.ignite.cache.eviction.EvictionFilter;
 import org.apache.ignite.cache.eviction.EvictionPolicy;
@@ -47,8 +48,10 @@ import org.apache.ignite.configuration.TopologyValidator;
 import org.apache.ignite.internal.ClusterMetricsSnapshot;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewTables;
 import org.apache.ignite.internal.util.lang.GridNodePredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
@@ -60,6 +63,8 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
+import static java.util.Arrays.asList;
+
 /**
  * Tests for ignite SQL system views.
  */
@@ -88,7 +93,7 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
      * @param args Args.
      */
     @SuppressWarnings("unchecked")
-    private List<List<?>> execSql(Ignite ignite, String sql, Object ... args) {
+    private List<List<?>> execSql(Ignite ignite, String sql, Object... args) {
         IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         SqlFieldsQuery qry = new SqlFieldsQuery(sql);
@@ -103,7 +108,7 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
      * @param sql Sql.
      * @param args Args.
      */
-    private List<List<?>> execSql(String sql, Object ... args) {
+    private List<List<?>> execSql(String sql, Object... args) {
         return execSql(grid(), sql, args);
     }
 
@@ -200,7 +205,7 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
      * @param rowData Row data.
      * @param colTypes Column types.
      */
-    private void assertColumnTypes(List<?> rowData, Class<?> ... colTypes) {
+    private void assertColumnTypes(List<?> rowData, Class<?>... colTypes) {
         for (int i = 0; i < colTypes.length; i++) {
             if (rowData.get(i) != null)
                 assertEquals("Column " + i + " type", colTypes[i], rowData.get(i).getClass());
@@ -226,7 +231,7 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
         awaitPartitionMapExchange();
 
         List<List<?>> resAll = execSql("SELECT ID, CONSISTENT_ID, VERSION, IS_CLIENT, IS_DAEMON, " +
-                "NODE_ORDER, ADDRESSES, HOSTNAMES FROM IGNITE.NODES");
+            "NODE_ORDER, ADDRESSES, HOSTNAMES FROM IGNITE.NODES");
 
         assertColumnTypes(resAll.get(0), UUID.class, String.class, String.class, Boolean.class, Boolean.class,
             Integer.class, String.class, String.class);
@@ -371,7 +376,7 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
 
         // Broadcast jobs to server and client nodes to get non zero metric values.
         for (int i = 0; i < 100; i++) {
-            IgniteFuture<Void > fut = igniteSrv.compute(igniteSrv.cluster().forNodeId(nodeId0, nodeId(1)))
+            IgniteFuture<Void> fut = igniteSrv.compute(igniteSrv.cluster().forNodeId(nodeId0, nodeId(1)))
                 .broadcastAsync(
                     new IgniteRunnable() {
                         @Override public void run() {
@@ -526,6 +531,236 @@ public class SqlSystemViewsSelfTest extends AbstractIndexingCommonTest {
     }
 
     /**
+     * Simple test for {@link SqlSystemViewTables}
+     */
+    @Test
+    public void testTablesView() throws Exception {
+        IgniteEx ignite = startGrid(getConfiguration());
+
+        GridCacheProcessor cacheProc = ignite.context().cache();
+
+        execSql("CREATE TABLE CACHE_SQL (ID INT PRIMARY KEY, MY_VAL VARCHAR) WITH " +
+            "\"cache_name=cache_sql,template=partitioned,atomicity=atomic,wrap_value=true,value_type=random_name\"");
+
+        execSql("CREATE TABLE PUBLIC.DFLT_CACHE (ID1 INT, ID2 INT, MY_VAL VARCHAR, PRIMARY KEY (ID1, ID2)) WITH"
+            + "\"affinity_key=ID2,wrap_value=false,key_type=random_name\"");
+
+        int cacheSqlId = cacheProc.cacheDescriptor("cache_sql").cacheId();
+        int ddlTabId = cacheProc.cacheDescriptor("SQL_PUBLIC_DFLT_CACHE").cacheId();
+
+        List<List<?>> cacheSqlInfos = execSql("SELECT * FROM IGNITE.TABLES WHERE TABLE_NAME = 'CACHE_SQL'");
+
+        List<?> expRow = asList(
+            "DEFAULT",           // SCHEMA_NAME
+            "CACHE_SQL",         // TABLE_NAME
+            "cache_sql",         // CACHE_NAME
+            cacheSqlId,          // CACHE_ID
+            null,                // AFFINITY_KEY_COLUMN
+            "ID",                // KEY_ALIAS
+            null,                // VALUE_ALIAS
+            "java.lang.Integer", // KEY_TYPE_NAME
+            "random_name"        // VALUE_TYPE_NAME
+
+        );
+
+        assertEquals("Returned incorrect info. ", expRow, cacheSqlInfos.get(0));
+
+        // no more rows are expected.
+        assertEquals("Expected to return only one row", 1, cacheSqlInfos.size());
+
+        List<List<?>> allInfos = execSql("SELECT * FROM IGNITE.TABLES");
+
+        List<?> allExpRows = asList(
+            expRow,
+            asList(
+                "PUBLIC",                // SCHEMA_NAME
+                "DFLT_CACHE",            // TABLE_NAME
+                "SQL_PUBLIC_DFLT_CACHE", // CACHE_NAME
+                ddlTabId,                // CACHE_ID
+                "ID2",                   // AFFINITY_KEY_COLUMN
+                null,                    // KEY_ALIAS
+                "MY_VAL",                // VALUE_ALIAS
+                "random_name",           // KEY_TYPE_NAME
+                "java.lang.String"       // VALUE_TYPE_NAME
+            )
+        );
+
+        if (!F.eqNotOrdered(allExpRows, allInfos))
+            fail("Returned incorrect rows [expected=" + allExpRows + ", actual=" + allInfos + "].");
+
+        // Filter by cache name:
+        assertEquals(
+            Collections.singletonList(asList("DFLT_CACHE", "SQL_PUBLIC_DFLT_CACHE")),
+            execSql("SELECT TABLE_NAME, CACHE_NAME " +
+                "FROM IGNITE.TABLES " +
+                "WHERE CACHE_NAME LIKE 'SQL\\_PUBLIC\\_%'"));
+
+        assertEquals(
+            Collections.singletonList(asList("CACHE_SQL", "cache_sql")),
+            execSql("SELECT TABLE_NAME, CACHE_NAME " +
+                "FROM IGNITE.TABLES " +
+                "WHERE CACHE_NAME NOT LIKE 'SQL\\_PUBLIC\\_%'"));
+
+        // Join with CACHES view.
+        assertEquals(
+            asList(
+                asList("DFLT_CACHE", "SQL_PUBLIC_DFLT_CACHE", "SQL_PUBLIC_DFLT_CACHE"),
+                asList("CACHE_SQL", "cache_sql", "cache_sql")),
+            execSql("SELECT TABLE_NAME, TAB.CACHE_NAME, C.NAME " +
+                "FROM IGNITE.TABLES AS TAB JOIN IGNITE.CACHES AS C " +
+                "ON TAB.CACHE_ID = C.CACHE_ID " +
+                "ORDER BY C.NAME")
+        );
+    }
+
+    /**
+     * Verify that if we drop or create table, TABLES system view reflects these changes.
+     */
+    @Test
+    public void testTablesDropAndCreate() throws Exception {
+        IgniteEx ignite = startGrid(getConfiguration());
+
+        final String selectTabNameCacheName = "SELECT TABLE_NAME, CACHE_NAME FROM IGNITE.TABLES ORDER BY TABLE_NAME";
+
+        assertTrue("Initially no tables expected", execSql(selectTabNameCacheName).isEmpty());
+
+        execSql("CREATE TABLE PUBLIC.TAB1 (ID INT PRIMARY KEY, VAL VARCHAR)");
+
+        assertEquals(
+            asList(asList("TAB1", "SQL_PUBLIC_TAB1")),
+            execSql(selectTabNameCacheName));
+
+        execSql("CREATE TABLE PUBLIC.TAB2 (ID LONG PRIMARY KEY, VAL_STR VARCHAR) WITH \"cache_name=cache2\"");
+        execSql("CREATE TABLE PUBLIC.TAB3 (ID LONG PRIMARY KEY, VAL_INT INT) WITH \"cache_name=cache3\" ");
+
+        assertEquals(
+            asList(
+                asList("TAB1", "SQL_PUBLIC_TAB1"),
+                asList("TAB2", "cache2"),
+                asList("TAB3", "cache3")
+            ),
+            execSql(selectTabNameCacheName));
+
+        execSql("DROP TABLE PUBLIC.TAB2");
+
+        assertEquals(
+            asList(
+                asList("TAB1", "SQL_PUBLIC_TAB1"),
+                asList("TAB3", "cache3")
+            ),
+            execSql(selectTabNameCacheName));
+
+        execSql("DROP TABLE PUBLIC.TAB3");
+
+        assertEquals(
+            asList(asList("TAB1", "SQL_PUBLIC_TAB1")),
+            execSql(selectTabNameCacheName));
+
+        execSql("DROP TABLE PUBLIC.TAB1");
+
+        assertTrue("All tables should be dropped", execSql(selectTabNameCacheName).isEmpty());
+    }
+
+
+
+    /**
+     * Dummy implementation of the mapper. Required to test "AFFINITY_KEY_COLUMN".
+     */
+    static class ConstantMapper implements AffinityKeyMapper {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 7018626316531791556L;
+
+        /** {@inheritDoc} */
+        @Override public Object affinityKey(Object key) {
+            return 1;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void reset() {
+            //NO-op
+        }
+    }
+
+    /**
+     * Check affinity column if custom affinity mapper is specified.
+     */
+    @Test
+    public void testTablesNullAffinityKey() throws Exception {
+        IgniteEx ignite = startGrid(getConfiguration());
+
+        AffinityKeyMapper fakeMapper = new ConstantMapper();
+
+        ignite.getOrCreateCache(defaultCacheConfiguration().setName("NO_KEY_FIELDS_CACHE").setAffinityMapper(fakeMapper)
+            .setQueryEntities(Collections.singleton(
+                // A cache with  no key fields
+                new QueryEntity(Object.class.getName(), "Object2")
+                    .addQueryField("name", String.class.getName(), null)
+                    .addQueryField("salary", Integer.class.getName(), null)
+                    .setTableName("NO_KEY_TABLE")
+            )));
+
+        List<List<String>> expected = Collections.singletonList(asList("NO_KEY_TABLE", null));
+
+        assertEquals(expected,
+            execSql("SELECT TABLE_NAME, AFFINITY_KEY_COLUMN " +
+                "FROM IGNITE.TABLES " +
+                "WHERE CACHE_NAME = 'NO_KEY_FIELDS_CACHE'"));
+
+        assertEquals(expected,
+            execSql("SELECT TABLE_NAME, AFFINITY_KEY_COLUMN " +
+                "FROM IGNITE.TABLES " +
+                "WHERE AFFINITY_KEY_COLUMN IS NULL"));
+    }
+
+    /**
+     * Special test for key/val name and type. Covers most used cases
+     */
+    @Test
+    public void testTablesViewKeyVal() throws Exception {
+        IgniteEx ignite = startGrid(getConfiguration());
+
+        {
+            ignite.getOrCreateCache(defaultCacheConfiguration().setName("NO_ALIAS_NON_SQL_KEY")
+                .setQueryEntities(Collections.singleton(
+                    // A cache with  no key fields
+                    new QueryEntity(Object.class.getName(), "Object2")
+                        .addQueryField("name", String.class.getName(), null)
+                        .addQueryField("salary", Integer.class.getName(), null)
+                        .setTableName("NO_ALIAS_NON_SQL_KEY")
+                )));
+
+            List<?> keyValAliases = execSql("SELECT KEY_ALIAS, VALUE_ALIAS FROM IGNITE.TABLES " +
+                "WHERE TABLE_NAME = 'NO_ALIAS_NON_SQL_KEY'").get(0);
+
+            assertEquals(asList(null, null), keyValAliases);
+        }
+
+        {
+            execSql("CREATE TABLE PUBLIC.SIMPLE_KEY_SIMPLE_VAL (ID INT PRIMARY KEY, NAME VARCHAR) WITH \"wrap_value=false\"");
+
+            List<?> keyValAliases = execSql("SELECT KEY_ALIAS, VALUE_ALIAS FROM IGNITE.TABLES " +
+                "WHERE TABLE_NAME = 'SIMPLE_KEY_SIMPLE_VAL'").get(0);
+
+            assertEquals(asList("ID", "NAME"), keyValAliases);
+
+        }
+
+        {
+            execSql("CREATE TABLE PUBLIC.COMPLEX_KEY_COMPLEX_VAL " +
+                "(ID1 INT, " +
+                "ID2 INT, " +
+                "VAL1 VARCHAR, " +
+                "VAL2 VARCHAR, " +
+                "PRIMARY KEY(ID1, ID2))");
+
+            List<?> keyValAliases = execSql("SELECT KEY_ALIAS, VALUE_ALIAS FROM IGNITE.TABLES " +
+                "WHERE TABLE_NAME = 'COMPLEX_KEY_COMPLEX_VAL'").get(0);
+
+            assertEquals(asList(null, null), keyValAliases);
+        }
+    }
+
+    /**
      * Test caches system views.
      */
     @SuppressWarnings("ConstantConditions")