Repository: ignite
Updated Branches:
  refs/heads/master 9afea90b9 -> 8b3812790


IGNITE-9500: SQL: implemented system view for list of caches (IGNITE.CACHES). 
This closes #4716.


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

Branch: refs/heads/master
Commit: 8b381279017080b74e3c6fee4d8a177c50411189
Parents: 9afea90
Author: Aleksey Plekhanov <plehanov.a...@gmail.com>
Authored: Wed Oct 10 11:57:11 2018 +0300
Committer: devozerov <voze...@gridgain.com>
Committed: Wed Oct 10 11:57:11 2018 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../h2/sys/view/SqlAbstractLocalSystemView.java |  13 +-
 .../query/h2/sys/view/SqlSystemViewCaches.java  | 197 ++++++++++++++++
 .../h2/sys/view/SqlSystemViewNodeMetrics.java   |   2 +-
 .../query/h2/sys/view/SqlSystemViewNodes.java   |   2 +-
 .../query/SqlSystemViewsSelfTest.java           | 225 +++++++++++++++++++
 6 files changed, 438 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8b381279/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 fe563b7..7c5f274 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.SqlSystemViewCaches;
 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;
@@ -3268,6 +3269,7 @@ public class IgniteH2Indexing implements 
GridQueryIndexing {
         views.add(new SqlSystemViewNodeAttributes(ctx));
         views.add(new SqlSystemViewBaselineNodes(ctx));
         views.add(new SqlSystemViewNodeMetrics(ctx));
+        views.add(new SqlSystemViewCaches(ctx));
 
         return views;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b381279/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java
index d692dba..d028406 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlAbstractLocalSystemView.java
@@ -37,7 +37,7 @@ public abstract class SqlAbstractLocalSystemView extends 
SqlAbstractSystemView {
      * @param tblName Table name.
      * @param desc Description.
      * @param ctx Context.
-     * @param indexes Indexed columns.
+     * @param indexes Indexes.
      * @param cols Columns.
      */
     public SqlAbstractLocalSystemView(String tblName, String desc, 
GridKernalContext ctx, String[] indexes,
@@ -53,6 +53,17 @@ public abstract class SqlAbstractLocalSystemView extends 
SqlAbstractSystemView {
      * @param tblName Table name.
      * @param desc Description.
      * @param ctx Context.
+     * @param indexedCols Indexed columns.
+     * @param cols Columns.
+     */
+    public SqlAbstractLocalSystemView(String tblName, String desc, 
GridKernalContext ctx, String indexedCols, Column... cols) {
+        this(tblName, desc, ctx, new String[] {indexedCols}, cols);
+    }
+
+    /**
+     * @param tblName Table name.
+     * @param desc Description.
+     * @param ctx Context.
      * @param cols Columns.
      */
     public SqlAbstractLocalSystemView(String tblName, String desc, 
GridKernalContext ctx, Column ... cols) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b381279/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
new file mode 100644
index 0000000..ff9ef32
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCaches.java
@@ -0,0 +1,197 @@
+/*
+ * 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.DynamicCacheDescriptor;
+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: caches.
+ */
+public class SqlSystemViewCaches extends SqlAbstractLocalSystemView {
+    /**
+     * @param ctx Grid context.
+     */
+    public SqlSystemViewCaches(GridKernalContext ctx) {
+        super("CACHES", "Ignite caches", ctx, "NAME",
+            newColumn("NAME"),
+            newColumn("CACHE_ID", Value.INT),
+            newColumn("CACHE_TYPE"),
+            newColumn("GROUP_ID", Value.INT),
+            newColumn("GROUP_NAME"),
+            newColumn("CACHE_MODE"),
+            newColumn("ATOMICITY_MODE"),
+            newColumn("IS_ONHEAP_CACHE_ENABLED", Value.BOOLEAN),
+            newColumn("IS_COPY_ON_READ", Value.BOOLEAN),
+            newColumn("IS_LOAD_PREVIOUS_VALUE", Value.BOOLEAN),
+            newColumn("IS_READ_FROM_BACKUP", Value.BOOLEAN),
+            newColumn("PARTITION_LOSS_POLICY"),
+            newColumn("NODE_FILTER"),
+            newColumn("TOPOLOGY_VALIDATOR"),
+            newColumn("IS_EAGER_TTL", Value.BOOLEAN),
+            newColumn("WRITE_SYNCHRONIZATION_MODE"),
+            newColumn("IS_INVALIDATE", Value.BOOLEAN),
+            newColumn("IS_EVENTS_DISABLED", Value.BOOLEAN),
+            newColumn("IS_STATISTICS_ENABLED", Value.BOOLEAN),
+            newColumn("IS_MANAGEMENT_ENABLED", Value.BOOLEAN),
+            newColumn("BACKUPS", Value.INT),
+            newColumn("AFFINITY"),
+            newColumn("AFFINITY_MAPPER"),
+            newColumn("REBALANCE_MODE"),
+            newColumn("REBALANCE_BATCH_SIZE", Value.INT),
+            newColumn("REBALANCE_TIMEOUT", Value.LONG),
+            newColumn("REBALANCE_DELAY", Value.LONG),
+            newColumn("REBALANCE_THROTTLE", Value.LONG),
+            newColumn("REBALANCE_BATCHES_PREFETCH_COUNT", Value.LONG),
+            newColumn("REBALANCE_ORDER", Value.INT),
+            newColumn("EVICTION_FILTER"),
+            newColumn("EVICTION_POLICY_FACTORY"),
+            newColumn("IS_NEAR_CACHE_ENABLED", Value.BOOLEAN),
+            newColumn("NEAR_CACHE_EVICTION_POLICY_FACTORY"),
+            newColumn("NEAR_CACHE_START_SIZE", Value.INT),
+            newColumn("DEFAULT_LOCK_TIMEOUT", Value.LONG),
+            newColumn("CACHE_INTERCEPTOR"),
+            newColumn("CACHE_STORE_FACTORY"),
+            newColumn("IS_STORE_KEEP_BINARY", Value.BOOLEAN),
+            newColumn("IS_READ_THROUGH", Value.BOOLEAN),
+            newColumn("IS_WRITE_THROUGH", Value.BOOLEAN),
+            newColumn("IS_WRITE_BEHIND_ENABLED", Value.BOOLEAN),
+            newColumn("WRITE_BEHIND_COALESCING", Value.BOOLEAN),
+            newColumn("WRITE_BEHIND_FLUSH_SIZE", Value.INT),
+            newColumn("WRITE_BEHIND_FLUSH_FREQUENCY", Value.LONG),
+            newColumn("WRITE_BEHIND_FLUSH_THREAD_COUNT", Value.INT),
+            newColumn("WRITE_BEHIND_FLUSH_BATCH_SIZE", Value.INT),
+            newColumn("MAX_CONCURRENT_ASYNC_OPERATIONS", Value.INT),
+            newColumn("CACHE_LOADER_FACTORY"),
+            newColumn("CACHE_WRITER_FACTORY"),
+            newColumn("EXPIRY_POLICY_FACTORY"),
+            newColumn("IS_SQL_ESCAPE_ALL", Value.BOOLEAN),
+            newColumn("SQL_SCHEMA"),
+            newColumn("SQL_INDEX_MAX_INLINE_SIZE", Value.INT),
+            newColumn("IS_SQL_ONHEAP_CACHE_ENABLED", Value.BOOLEAN),
+            newColumn("SQL_ONHEAP_CACHE_MAX_SIZE", Value.INT),
+            newColumn("QUERY_DETAILS_METRICS_SIZE", Value.INT),
+            newColumn("QUERY_PARALLELISM", Value.INT),
+            newColumn("MAX_QUERY_ITERATORS_COUNT", Value.INT),
+            newColumn("DATA_REGION_NAME")
+        );
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Iterator<Row> getRows(Session ses, SearchRow first, 
SearchRow last) {
+        SqlSystemViewColumnCondition nameCond = conditionForColumn("NAME", 
first, last);
+
+        Collection<DynamicCacheDescriptor> caches;
+
+        if (nameCond.isEquality()) {
+            DynamicCacheDescriptor cache = 
ctx.cache().cacheDescriptor(nameCond.valueForEquality().getString());
+
+            caches = cache == null ? Collections.emptySet() : 
Collections.singleton(cache);
+        }
+        else
+            caches = ctx.cache().cacheDescriptors().values();
+
+        AtomicLong rowKey = new AtomicLong();
+
+        return F.iterator(caches,
+            cache -> createRow(ses, rowKey.incrementAndGet(),
+                cache.cacheName(),
+                cache.cacheId(),
+                cache.cacheType(),
+                cache.groupId(),
+                cache.groupDescriptor().groupName(),
+                cache.cacheConfiguration().getCacheMode(),
+                cache.cacheConfiguration().getAtomicityMode(),
+                cache.cacheConfiguration().isOnheapCacheEnabled(),
+                cache.cacheConfiguration().isCopyOnRead(),
+                cache.cacheConfiguration().isLoadPreviousValue(),
+                cache.cacheConfiguration().isReadFromBackup(),
+                cache.cacheConfiguration().getPartitionLossPolicy(),
+                cache.cacheConfiguration().getNodeFilter(),
+                cache.cacheConfiguration().getTopologyValidator(),
+                cache.cacheConfiguration().isEagerTtl(),
+                cache.cacheConfiguration().getWriteSynchronizationMode(),
+                cache.cacheConfiguration().isInvalidate(),
+                cache.cacheConfiguration().isEventsDisabled(),
+                cache.cacheConfiguration().isStatisticsEnabled(),
+                cache.cacheConfiguration().isManagementEnabled(),
+                cache.cacheConfiguration().getBackups(),
+                cache.cacheConfiguration().getAffinity(),
+                cache.cacheConfiguration().getAffinityMapper(),
+                cache.cacheConfiguration().getRebalanceMode(),
+                cache.cacheConfiguration().getRebalanceBatchSize(),
+                cache.cacheConfiguration().getRebalanceTimeout(),
+                cache.cacheConfiguration().getRebalanceDelay(),
+                cache.cacheConfiguration().getRebalanceThrottle(),
+                cache.cacheConfiguration().getRebalanceBatchesPrefetchCount(),
+                cache.cacheConfiguration().getRebalanceOrder(),
+                cache.cacheConfiguration().getEvictionFilter(),
+                cache.cacheConfiguration().getEvictionPolicyFactory(),
+                cache.cacheConfiguration().getNearConfiguration() != null,
+                cache.cacheConfiguration().getNearConfiguration() != null ?
+                    
cache.cacheConfiguration().getNearConfiguration().getNearEvictionPolicyFactory()
 : null,
+                cache.cacheConfiguration().getNearConfiguration() != null ?
+                    
cache.cacheConfiguration().getNearConfiguration().getNearStartSize() : null,
+                cache.cacheConfiguration().getDefaultLockTimeout(),
+                cache.cacheConfiguration().getInterceptor(),
+                cache.cacheConfiguration().getCacheStoreFactory(),
+                cache.cacheConfiguration().isStoreKeepBinary(),
+                cache.cacheConfiguration().isReadThrough(),
+                cache.cacheConfiguration().isWriteThrough(),
+                cache.cacheConfiguration().isWriteBehindEnabled(),
+                cache.cacheConfiguration().getWriteBehindCoalescing(),
+                cache.cacheConfiguration().getWriteBehindFlushSize(),
+                cache.cacheConfiguration().getWriteBehindFlushFrequency(),
+                cache.cacheConfiguration().getWriteBehindFlushThreadCount(),
+                cache.cacheConfiguration().getWriteBehindBatchSize(),
+                cache.cacheConfiguration().getMaxConcurrentAsyncOperations(),
+                cache.cacheConfiguration().getCacheLoaderFactory(),
+                cache.cacheConfiguration().getCacheWriterFactory(),
+                cache.cacheConfiguration().getExpiryPolicyFactory(),
+                cache.cacheConfiguration().isSqlEscapeAll(),
+                cache.cacheConfiguration().getSqlSchema(),
+                cache.cacheConfiguration().getSqlIndexMaxInlineSize(),
+                cache.cacheConfiguration().isSqlOnheapCacheEnabled(),
+                cache.cacheConfiguration().getSqlOnheapCacheMaxSize(),
+                cache.cacheConfiguration().getQueryDetailMetricsSize(),
+                cache.cacheConfiguration().getQueryParallelism(),
+                cache.cacheConfiguration().getMaxQueryIteratorsCount(),
+                cache.cacheConfiguration().getDataRegionName()
+            ), true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canGetRowCount() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCount() {
+        return ctx.cache().cacheDescriptors().size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b381279/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
index 01b4e97..d3921aa 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
@@ -40,7 +40,7 @@ public class SqlSystemViewNodeMetrics extends 
SqlAbstractLocalSystemView {
      * @param ctx Grid context.
      */
     public SqlSystemViewNodeMetrics(GridKernalContext ctx) {
-        super("NODE_METRICS", "Node metrics", ctx, new String[] {"NODE_ID"},
+        super("NODE_METRICS", "Node metrics", ctx, "NODE_ID",
             newColumn("NODE_ID", Value.UUID),
             newColumn("LAST_UPDATE_TIME", Value.TIMESTAMP),
             newColumn("MAX_ACTIVE_JOBS", Value.INT),

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b381279/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodes.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodes.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodes.java
index 514f92e..d872031 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodes.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodes.java
@@ -39,7 +39,7 @@ public class SqlSystemViewNodes extends 
SqlAbstractLocalSystemView {
      * @param ctx Grid context.
      */
     public SqlSystemViewNodes(GridKernalContext ctx) {
-        super("NODES", "Topology nodes", ctx, new String[] {"ID"},
+        super("NODES", "Topology nodes", ctx, "ID",
             newColumn("ID", Value.UUID),
             newColumn("CONSISTENT_ID"),
             newColumn("VERSION"),

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b381279/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 ccd0796..feee9a0 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
@@ -19,24 +19,35 @@ package org.apache.ignite.internal.processors.query;
 
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 import java.util.TimeZone;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
 import org.apache.ignite.Ignite;
 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.eviction.EvictableEntry;
+import org.apache.ignite.cache.eviction.EvictionFilter;
+import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 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.IgniteNodeAttributes;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.util.lang.GridNodePredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
@@ -505,6 +516,154 @@ public class SqlSystemViewsSelfTest extends 
GridCommonAbstractTest {
     }
 
     /**
+     * Test caches system views.
+     */
+    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));
+
+        Ignite ignite1 = 
startGrid(getConfiguration().setDataStorageConfiguration(dsCfg).setIgniteInstanceName("node1"));
+
+        ignite0.cluster().active(true);
+
+        Ignite ignite2 = 
startGrid(getConfiguration().setDataStorageConfiguration(dsCfg).setIgniteInstanceName("node2"));
+
+        Ignite ignite3 = 
startGrid(getConfiguration().setDataStorageConfiguration(dsCfg).setIgniteInstanceName("node3")
+            .setClientMode(true));
+
+        ignite0.getOrCreateCache(new CacheConfiguration<>()
+            .setName("cache_atomic_part")
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setGroupName("part_grp")
+            .setNodeFilter(new TestNodeFilter(ignite0.cluster().localNode()))
+        );
+
+        ignite0.getOrCreateCache(new CacheConfiguration<>()
+            .setName("cache_atomic_repl")
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setCacheMode(CacheMode.REPLICATED)
+            .setDataRegionName("dr1")
+            .setTopologyValidator(new TestTopologyValidator())
+        );
+
+        ignite0.getOrCreateCache(new CacheConfiguration<>()
+            .setName("cache_tx_part")
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setGroupName("part_grp")
+            .setNodeFilter(new TestNodeFilter(ignite0.cluster().localNode()))
+        );
+
+        ignite0.getOrCreateCache(new CacheConfiguration<>()
+            .setName("cache_tx_repl")
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+            .setCacheMode(CacheMode.REPLICATED)
+            .setDataRegionName("dr2")
+            .setEvictionFilter(new TestEvictionFilter())
+            .setEvictionPolicyFactory(new TestEvictionPolicyFactory())
+            .setOnheapCacheEnabled(true)
+        );
+
+        execSql("CREATE TABLE cache_sql (ID INT PRIMARY KEY, VAL VARCHAR) WITH 
" +
+            "\"cache_name=cache_sql,template=partitioned,atomicity=atomic\"");
+
+        awaitPartitionMapExchange();
+
+        List<List<?>> resAll = execSql("SELECT NAME, CACHE_ID, CACHE_TYPE, 
GROUP_ID, GROUP_NAME, " +
+                "CACHE_MODE, ATOMICITY_MODE, IS_ONHEAP_CACHE_ENABLED, 
IS_COPY_ON_READ, IS_LOAD_PREVIOUS_VALUE, " +
+                "IS_READ_FROM_BACKUP, PARTITION_LOSS_POLICY, NODE_FILTER, 
TOPOLOGY_VALIDATOR, IS_EAGER_TTL, " +
+                "WRITE_SYNCHRONIZATION_MODE, IS_INVALIDATE, 
IS_EVENTS_DISABLED, IS_STATISTICS_ENABLED, " +
+                "IS_MANAGEMENT_ENABLED, BACKUPS, AFFINITY, AFFINITY_MAPPER, " +
+                "REBALANCE_MODE, REBALANCE_BATCH_SIZE, REBALANCE_TIMEOUT, 
REBALANCE_DELAY, REBALANCE_THROTTLE, " +
+                "REBALANCE_BATCHES_PREFETCH_COUNT, REBALANCE_ORDER, " +
+                "EVICTION_FILTER, EVICTION_POLICY_FACTORY, " +
+                "IS_NEAR_CACHE_ENABLED, NEAR_CACHE_EVICTION_POLICY_FACTORY, 
NEAR_CACHE_START_SIZE, " +
+                "DEFAULT_LOCK_TIMEOUT, CACHE_INTERCEPTOR, CACHE_STORE_FACTORY, 
" +
+                "IS_STORE_KEEP_BINARY, IS_READ_THROUGH, IS_WRITE_THROUGH, " +
+                "IS_WRITE_BEHIND_ENABLED, WRITE_BEHIND_COALESCING, 
WRITE_BEHIND_FLUSH_SIZE, " +
+                "WRITE_BEHIND_FLUSH_FREQUENCY, 
WRITE_BEHIND_FLUSH_THREAD_COUNT, WRITE_BEHIND_FLUSH_BATCH_SIZE, " +
+                "MAX_CONCURRENT_ASYNC_OPERATIONS, CACHE_LOADER_FACTORY, 
CACHE_WRITER_FACTORY, EXPIRY_POLICY_FACTORY, " +
+                "IS_SQL_ESCAPE_ALL, SQL_SCHEMA, SQL_INDEX_MAX_INLINE_SIZE, 
IS_SQL_ONHEAP_CACHE_ENABLED, " +
+                "SQL_ONHEAP_CACHE_MAX_SIZE, QUERY_DETAILS_METRICS_SIZE, 
QUERY_PARALLELISM, MAX_QUERY_ITERATORS_COUNT, " +
+                "DATA_REGION_NAME FROM IGNITE.CACHES");
+
+        assertColumnTypes(resAll.get(0),
+            String.class, Integer.class, String.class, Integer.class, 
String.class,
+            String.class, String.class, Boolean.class, Boolean.class, 
Boolean.class,
+            Boolean.class, String.class, String.class, String.class, 
Boolean.class,
+            String.class, Boolean.class, Boolean.class, Boolean.class,
+            Boolean.class, Integer.class, String.class, String.class,
+            String.class, Integer.class, Long.class, Long.class, Long.class, 
// Rebalance.
+            Long.class, Integer.class,
+            String.class, String.class, // Eviction.
+            Boolean.class, String.class, Integer.class, // Near cache.
+            Long.class, String.class, String.class,
+            Boolean.class, Boolean.class, Boolean.class,
+            Boolean.class, Boolean.class, Integer.class, // Write-behind.
+            Long.class, Integer.class, Integer.class,
+            Integer.class, String.class, String.class, String.class,
+            Boolean.class, String.class, Integer.class, Boolean.class, // SQL.
+            Integer.class, Integer.class, Integer.class, Integer.class,
+            String.class);
+
+        assertEquals("cache_tx_part", execSql("SELECT NAME FROM IGNITE.CACHES 
WHERE " +
+            "CACHE_MODE = 'PARTITIONED' AND ATOMICITY_MODE = 'TRANSACTIONAL' 
AND NAME like 'cache%'").get(0).get(0));
+
+        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'")
+            .get(0).get(0));
+
+        assertEquals("cache_atomic_repl", execSql("SELECT NAME FROM 
IGNITE.CACHES " +
+            "WHERE DATA_REGION_NAME = 'dr1'").get(0).get(0));
+
+        assertEquals("cache_tx_repl", execSql("SELECT NAME FROM IGNITE.CACHES 
" +
+            "WHERE DATA_REGION_NAME = 'dr2'").get(0).get(0));
+
+        assertEquals("PARTITIONED", execSql("SELECT CACHE_MODE FROM 
IGNITE.CACHES " +
+            "WHERE NAME = 'cache_atomic_part'").get(0).get(0));
+
+        assertEquals("USER", execSql("SELECT CACHE_TYPE FROM IGNITE.CACHES 
WHERE NAME = 'cache_sql'")
+            .get(0).get(0));
+
+        assertEquals(0L, execSql("SELECT COUNT(*) FROM IGNITE.CACHES WHERE 
NAME = 'no_such_cache'").get(0)
+            .get(0));
+
+        assertEquals(0L, execSql("SELECT COUNT(*) FROM IGNITE.CACHES WHERE 
NAME = 1").get(0).get(0));
+
+        assertEquals("TestNodeFilter", execSql("SELECT NODE_FILTER FROM 
IGNITE.CACHES WHERE NAME = " +
+            "'cache_atomic_part'").get(0).get(0));
+
+        assertEquals("TestEvictionFilter", execSql("SELECT EVICTION_FILTER 
FROM IGNITE.CACHES " +
+            "WHERE NAME = 'cache_tx_repl'").get(0).get(0));
+
+        assertEquals("TestEvictionPolicyFactory", execSql("SELECT 
EVICTION_POLICY_FACTORY " +
+            "FROM IGNITE.CACHES WHERE NAME = 'cache_tx_repl'").get(0).get(0));
+
+        assertEquals("TestTopologyValidator", execSql("SELECT 
TOPOLOGY_VALIDATOR FROM IGNITE.CACHES " +
+            "WHERE NAME = 'cache_atomic_repl'").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));
+
+        assertEquals(5L, execSql(ignite1, "SELECT COUNT(*) FROM IGNITE.CACHES 
WHERE NAME like 'cache%'")
+            .get(0).get(0));
+
+        assertEquals(5L, execSql(ignite2, "SELECT COUNT(*) FROM IGNITE.CACHES 
WHERE NAME like 'cache%'")
+            .get(0).get(0));
+
+        assertEquals(5L, execSql(ignite3, "SELECT COUNT(*) FROM IGNITE.CACHES 
WHERE NAME like 'cache%'")
+            .get(0).get(0));
+    }
+
+    /**
      * Gets ignite configuration with persistence enabled.
      */
     private IgniteConfiguration getPdsConfiguration(String consistentId) 
throws Exception {
@@ -533,4 +692,70 @@ public class SqlSystemViewsSelfTest extends 
GridCommonAbstractTest {
 
         return time0.getTime() + 
TimeZone.getDefault().getOffset(time0.getTime());
     }
+
+    /**
+     *
+     */
+    private static class TestNodeFilter extends GridNodePredicate {
+        /**
+         * @param node Node.
+         */
+        public TestNodeFilter(ClusterNode node) {
+            super(node);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "TestNodeFilter";
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestEvictionFilter implements EvictionFilter<Object, 
Object> {
+        /** {@inheritDoc} */
+        @Override public boolean evictAllowed(Cache.Entry<Object, Object> 
entry) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "TestEvictionFilter";
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestEvictionPolicyFactory implements 
Factory<EvictionPolicy<Object, Object>> {
+        /** {@inheritDoc} */
+        @Override public EvictionPolicy<Object, Object> create() {
+            return new EvictionPolicy<Object, Object>() {
+                @Override public void onEntryAccessed(boolean rmv, 
EvictableEntry<Object, Object> entry) {
+                    // No-op.
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "TestEvictionPolicyFactory";
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestTopologyValidator implements TopologyValidator {
+        /** {@inheritDoc} */
+        @Override public boolean validate(Collection<ClusterNode> nodes) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "TestTopologyValidator";
+        }
+    }
 }

Reply via email to