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 2018/10/25 13:51:10 UTC

ignite git commit: IGNITE-9780: SQL: Add IGNITE.CACHE_GROUPS view. This closes #4985.

Repository: ignite
Updated Branches:
  refs/heads/master 610b96ddd -> b94efd511


IGNITE-9780: SQL: Add IGNITE.CACHE_GROUPS view. This closes #4985.


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

Branch: refs/heads/master
Commit: b94efd5114467075dcff057a2c896491976c617b
Parents: 610b96d
Author: Aleksey Plekhanov <pl...@gmail.com>
Authored: Thu Oct 25 16:50:56 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 25 16:50:56 2018 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../h2/sys/view/SqlSystemViewCacheGroups.java   | 111 +++++++++++++++++++
 .../query/h2/sys/view/SqlSystemViewCaches.java  |   2 +-
 .../query/SqlSystemViewsSelfTest.java           |  79 ++++++++++++-
 4 files changed, 189 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b94efd51/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index cbe2d0b..06868c4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -131,6 +131,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
 import org.apache.ignite.internal.processors.query.h2.sys.SqlSystemTableEngine;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemView;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewBaselineNodes;
+import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewCacheGroups;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewCaches;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeAttributes;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeMetrics;
@@ -3273,6 +3274,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         views.add(new SqlSystemViewBaselineNodes(ctx));
         views.add(new SqlSystemViewNodeMetrics(ctx));
         views.add(new SqlSystemViewCaches(ctx));
+        views.add(new SqlSystemViewCacheGroups(ctx));
 
         return views;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b94efd51/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroups.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroups.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroups.java
new file mode 100644
index 0000000..d4363b2
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroups.java
@@ -0,0 +1,111 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.h2.engine.Session;
+import org.h2.result.Row;
+import org.h2.result.SearchRow;
+import org.h2.value.Value;
+
+/**
+ * System view: cache groups.
+ */
+public class SqlSystemViewCacheGroups extends SqlAbstractLocalSystemView {
+    /**
+     * @param ctx Grid context.
+     */
+    public SqlSystemViewCacheGroups(GridKernalContext ctx) {
+        super("CACHE_GROUPS", "Cache groups", ctx, "ID",
+            newColumn("ID", Value.INT),
+            newColumn("GROUP_NAME"),
+            newColumn("IS_SHARED", Value.BOOLEAN),
+            newColumn("CACHE_COUNT", Value.INT),
+            newColumn("CACHE_MODE"),
+            newColumn("ATOMICITY_MODE"),
+            newColumn("AFFINITY"),
+            newColumn("PARTITIONS_COUNT", Value.INT),
+            newColumn("NODE_FILTER"),
+            newColumn("DATA_REGION_NAME"),
+            newColumn("TOPOLOGY_VALIDATOR"),
+            newColumn("PARTITION_LOSS_POLICY"),
+            newColumn("REBALANCE_MODE"),
+            newColumn("REBALANCE_DELAY", Value.LONG),
+            newColumn("REBALANCE_ORDER", Value.INT),
+            newColumn("BACKUPS", Value.INT)
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Row> getRows(Session ses, SearchRow first, SearchRow last) {
+        SqlSystemViewColumnCondition idCond = conditionForColumn("ID", first, last);
+
+        Collection<CacheGroupDescriptor> cacheGroups;
+
+        if (idCond.isEquality()) {
+            try {
+                CacheGroupDescriptor cacheGrp = ctx.cache().cacheGroupDescriptors().get(idCond.valueForEquality().getInt());
+
+                cacheGroups = cacheGrp == null ? Collections.emptySet() : Collections.singleton(cacheGrp);
+            }
+            catch (Exception ignore) {
+                cacheGroups = Collections.emptySet();
+            }
+        }
+        else
+            cacheGroups = ctx.cache().cacheGroupDescriptors().values();
+
+        AtomicLong rowKey = new AtomicLong();
+
+        return F.iterator(cacheGroups,
+            grp -> createRow(ses, rowKey.incrementAndGet(),
+                grp.groupId(),
+                grp.cacheOrGroupName(),
+                grp.sharedGroup(),
+                grp.caches() == null ? 0 : grp.caches().size(),
+                grp.config().getCacheMode(),
+                grp.config().getAtomicityMode(),
+                grp.config().getAffinity(),
+                grp.config().getAffinity() != null ? grp.config().getAffinity().partitions() : null,
+                grp.config().getNodeFilter(),
+                grp.config().getDataRegionName(),
+                grp.config().getTopologyValidator(),
+                grp.config().getPartitionLossPolicy(),
+                grp.config().getRebalanceMode(),
+                grp.config().getRebalanceDelay(),
+                grp.config().getRebalanceOrder(),
+                grp.config().getBackups()
+            ), true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canGetRowCount() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCount() {
+        return ctx.cache().cacheGroupDescriptors().size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b94efd51/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCaches.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCaches.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCaches.java
index ff9ef32..5e63bc0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCaches.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCaches.java
@@ -124,7 +124,7 @@ public class SqlSystemViewCaches extends SqlAbstractLocalSystemView {
                 cache.cacheId(),
                 cache.cacheType(),
                 cache.groupId(),
-                cache.groupDescriptor().groupName(),
+                cache.groupDescriptor().cacheOrGroupName(),
                 cache.cacheConfiguration().getCacheMode(),
                 cache.cacheConfiguration().getAtomicityMode(),
                 cache.cacheConfiguration().isOnheapCacheEnabled(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/b94efd51/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSystemViewsSelfTest.java
----------------------------------------------------------------------
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 feee9a0..243dcd1 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
@@ -45,6 +45,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 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.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.util.lang.GridNodePredicate;
@@ -518,13 +519,14 @@ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest {
     /**
      * Test caches system views.
      */
+    @SuppressWarnings("ConstantConditions")
     public void testCachesViews() throws Exception {
         DataStorageConfiguration dsCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setName("def").setPersistenceEnabled(true))
             .setDataRegionConfigurations(new DataRegionConfiguration().setName("dr1"),
                 new DataRegionConfiguration().setName("dr2"));
 
-        Ignite ignite0 = startGrid(getConfiguration().setDataStorageConfiguration(dsCfg));
+        IgniteEx ignite0 = startGrid(getConfiguration().setDataStorageConfiguration(dsCfg));
 
         Ignite ignite1 = startGrid(getConfiguration().setDataStorageConfiguration(dsCfg).setIgniteInstanceName("node1"));
 
@@ -539,7 +541,7 @@ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest {
             .setName("cache_atomic_part")
             .setAtomicityMode(CacheAtomicityMode.ATOMIC)
             .setCacheMode(CacheMode.PARTITIONED)
-            .setGroupName("part_grp")
+            .setGroupName("cache_grp")
             .setNodeFilter(new TestNodeFilter(ignite0.cluster().localNode()))
         );
 
@@ -555,7 +557,7 @@ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest {
             .setName("cache_tx_part")
             .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
             .setCacheMode(CacheMode.PARTITIONED)
-            .setGroupName("part_grp")
+            .setGroupName("cache_grp")
             .setNodeFilter(new TestNodeFilter(ignite0.cluster().localNode()))
         );
 
@@ -617,7 +619,7 @@ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest {
         assertEquals("cache_atomic_repl", execSql("SELECT NAME FROM IGNITE.CACHES WHERE " +
             "CACHE_MODE = 'REPLICATED' AND ATOMICITY_MODE = 'ATOMIC' AND NAME like 'cache%'").get(0).get(0));
 
-        assertEquals(2L, execSql("SELECT COUNT(*) FROM IGNITE.CACHES WHERE GROUP_NAME = 'part_grp'")
+        assertEquals(2L, execSql("SELECT COUNT(*) FROM IGNITE.CACHES WHERE GROUP_NAME = 'cache_grp'")
             .get(0).get(0));
 
         assertEquals("cache_atomic_repl", execSql("SELECT NAME FROM IGNITE.CACHES " +
@@ -649,6 +651,10 @@ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest {
         assertEquals("TestTopologyValidator", execSql("SELECT TOPOLOGY_VALIDATOR FROM IGNITE.CACHES " +
             "WHERE NAME = 'cache_atomic_repl'").get(0).get(0));
 
+        // Check quick count.
+        assertEquals(execSql("SELECT COUNT(*) FROM IGNITE.CACHES").get(0).get(0),
+            execSql("SELECT COUNT(*) FROM IGNITE.CACHES WHERE CACHE_ID <> CACHE_ID + 1").get(0).get(0));
+
         // Check that caches are the same on BLT, BLT filtered by node filter, non BLT and client nodes.
         assertEquals(5L, execSql("SELECT COUNT(*) FROM IGNITE.CACHES WHERE NAME like 'cache%'").get(0)
             .get(0));
@@ -661,6 +667,71 @@ public class SqlSystemViewsSelfTest extends GridCommonAbstractTest {
 
         assertEquals(5L, execSql(ignite3, "SELECT COUNT(*) FROM IGNITE.CACHES WHERE NAME like 'cache%'")
             .get(0).get(0));
+
+        // Check cache groups.
+        resAll = execSql("SELECT ID, GROUP_NAME, IS_SHARED, CACHE_COUNT, " +
+            "CACHE_MODE, ATOMICITY_MODE, AFFINITY, PARTITIONS_COUNT, " +
+            "NODE_FILTER, DATA_REGION_NAME, TOPOLOGY_VALIDATOR, PARTITION_LOSS_POLICY, " +
+            "REBALANCE_MODE, REBALANCE_DELAY, REBALANCE_ORDER, BACKUPS " +
+            "FROM IGNITE.CACHE_GROUPS");
+
+        assertColumnTypes(resAll.get(0),
+            Integer.class, String.class, Boolean.class, Integer.class,
+            String.class, String.class, String.class, Integer.class,
+            String.class, String.class, String.class, String.class,
+            String.class, Long.class, Integer.class, Integer.class);
+
+        assertEquals(2, execSql("SELECT CACHE_COUNT FROM IGNITE.CACHE_GROUPS " +
+            "WHERE GROUP_NAME = 'cache_grp'").get(0).get(0));
+
+        assertEquals("cache_grp", execSql("SELECT GROUP_NAME FROM IGNITE.CACHE_GROUPS " +
+            "WHERE IS_SHARED = true AND GROUP_NAME like 'cache%'").get(0).get(0));
+
+        // Check index on ID column.
+        assertEquals("cache_tx_repl", execSql("SELECT GROUP_NAME FROM IGNITE.CACHE_GROUPS " +
+            "WHERE ID = ?", ignite0.cachex("cache_tx_repl").context().groupId()).get(0).get(0));
+
+        assertEquals(0, execSql("SELECT ID FROM IGNITE.CACHE_GROUPS WHERE ID = 0").size());
+
+        // Check join by indexed column.
+        assertEquals("cache_tx_repl", execSql("SELECT CG.GROUP_NAME FROM IGNITE.CACHES C JOIN " +
+            "IGNITE.CACHE_GROUPS CG ON C.GROUP_ID = CG.ID WHERE C.NAME = 'cache_tx_repl'").get(0).get(0));
+
+        // Check join by non-indexed column.
+        assertEquals("cache_grp", execSql("SELECT CG.GROUP_NAME FROM IGNITE.CACHES C JOIN " +
+            "IGNITE.CACHE_GROUPS CG ON C.GROUP_NAME = CG.GROUP_NAME WHERE C.NAME = 'cache_tx_part'").get(0).get(0));
+
+        // Check configuration equality for cache and cache group views.
+        assertEquals(3L, execSql("SELECT COUNT(*) FROM IGNITE.CACHES C JOIN IGNITE.CACHE_GROUPS CG " +
+            "ON C.NAME = CG.GROUP_NAME WHERE C.NAME like 'cache%' " +
+            "AND C.CACHE_MODE = CG.CACHE_MODE " +
+            "AND C.ATOMICITY_MODE = CG.ATOMICITY_MODE " +
+            "AND COALESCE(C.AFFINITY, '-') = COALESCE(CG.AFFINITY, '-') " +
+            "AND COALESCE(C.NODE_FILTER, '-') = COALESCE(CG.NODE_FILTER, '-') " +
+            "AND COALESCE(C.DATA_REGION_NAME, '-') = COALESCE(CG.DATA_REGION_NAME, '-') " +
+            "AND COALESCE(C.TOPOLOGY_VALIDATOR, '-') = COALESCE(CG.TOPOLOGY_VALIDATOR, '-') " +
+            "AND C.PARTITION_LOSS_POLICY = CG.PARTITION_LOSS_POLICY " +
+            "AND C.REBALANCE_MODE = CG.REBALANCE_MODE " +
+            "AND C.REBALANCE_DELAY = CG.REBALANCE_DELAY " +
+            "AND C.REBALANCE_ORDER = CG.REBALANCE_ORDER " +
+            "AND C.BACKUPS = CG.BACKUPS").get(0).get(0));
+
+        // Check quick count.
+        assertEquals(execSql("SELECT COUNT(*) FROM IGNITE.CACHE_GROUPS").get(0).get(0),
+            execSql("SELECT COUNT(*) FROM IGNITE.CACHE_GROUPS WHERE ID <> ID + 1").get(0).get(0));
+
+        // Check that cache groups are the same on different nodes.
+        assertEquals(4L, execSql("SELECT COUNT(*) FROM IGNITE.CACHE_GROUPS " +
+            "WHERE GROUP_NAME like 'cache%'").get(0).get(0));
+
+        assertEquals(4L, execSql(ignite1, "SELECT COUNT(*) FROM IGNITE.CACHE_GROUPS " +
+            "WHERE GROUP_NAME like 'cache%'").get(0).get(0));
+
+        assertEquals(4L, execSql(ignite2, "SELECT COUNT(*) FROM IGNITE.CACHE_GROUPS " +
+            "WHERE GROUP_NAME like 'cache%'").get(0).get(0));
+
+        assertEquals(4L, execSql(ignite3, "SELECT COUNT(*) FROM IGNITE.CACHE_GROUPS " +
+            "WHERE GROUP_NAME like 'cache%'").get(0).get(0));
     }
 
     /**