You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/05/31 07:35:48 UTC

[17/19] ignite git commit: IGNITE-5242: Disallowed interference between SQL and non-SQL caches lifecycle. This closes #1996.

IGNITE-5242: Disallowed interference between SQL and non-SQL caches lifecycle. This closes #1996.


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

Branch: refs/heads/ignite-5075
Commit: 10a0a84a15fde067b522da156d2de2bd17faedfb
Parents: a858611
Author: Alexander Paschenko <al...@gmail.com>
Authored: Wed May 31 10:19:35 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 31 10:19:35 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgniteEx.java    |  9 ++-
 .../apache/ignite/internal/IgniteKernal.java    | 19 +++--
 .../cache/CacheAffinitySharedManager.java       |  1 +
 .../internal/processors/cache/CacheData.java    | 13 ++++
 .../cache/CacheJoinNodeDiscoveryData.java       | 15 +++-
 .../processors/cache/ClusterCachesInfo.java     | 47 +++++++++---
 .../cache/DynamicCacheChangeRequest.java        | 21 +++++-
 .../cache/DynamicCacheDescriptor.java           | 13 ++++
 .../processors/cache/GridCacheAttributes.java   | 14 +++-
 .../processors/cache/GridCacheProcessor.java    | 53 ++++++++++---
 .../processors/cache/IgniteCacheProxy.java      |  2 +-
 .../datastructures/DataStructuresProcessor.java |  1 +
 .../processors/query/GridQueryProcessor.java    |  4 +-
 .../handlers/cache/GridCacheCommandHandler.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +-
 .../processors/igfs/IgfsIgniteMock.java         |  4 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  4 +-
 .../cache/index/H2DynamicTableSelfTest.java     | 79 +++++++++++++++++++-
 18 files changed, 260 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index 8c27027..775f493 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -71,20 +71,23 @@ public interface IgniteEx extends Ignite {
      * of the existing cache.
      *
      * @param cacheCfg Cache configuration to use.
+     * @param sql {@code true} if this call is triggered by SQL command {@code CREATE TABLE}, {@code false} otherwise.
      * @return Tuple [Existing or newly created cache; {@code true} if cache was newly crated, {@code false} otherwise]
      * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(CacheConfiguration<K, V> cacheCfg)
-        throws CacheException;
+    public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(CacheConfiguration<K, V> cacheCfg,
+        boolean sql) throws CacheException;
 
     /**
      * Stops dynamically started cache.
      *
      * @param cacheName Cache name to stop.
+     * @param sql {@code true} if only cache created with SQL command {@code CREATE TABLE} should be affected,
+     *     {@code false} otherwise.
      * @return {@code true} if cache has been stopped as the result of this call, {@code false} otherwise.
      * @throws CacheException If error occurs.
      */
-    public boolean destroyCache0(String cacheName) throws CacheException;
+    public boolean destroyCache0(String cacheName, boolean sql) throws CacheException;
 
     /**
      * Checks if the event type is user-recordable.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 6393e9f..6d05147 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2791,13 +2791,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
-        return getOrCreateCache0(cacheCfg).get1();
+        return getOrCreateCache0(cacheCfg, false).get1();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
-        CacheConfiguration<K, V> cacheCfg) {
+        CacheConfiguration<K, V> cacheCfg, boolean sql) {
         A.notNull(cacheCfg, "cacheCfg");
         CU.validateCacheName(cacheCfg.getName());
 
@@ -2809,7 +2809,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             Boolean res = false;
 
             if (ctx.cache().cache(cacheCfg.getName()) == null) {
-                res = ctx.cache().dynamicStartCache(cacheCfg,
+                res =
+                    sql ? ctx.cache().dynamicStartSqlCache(cacheCfg).get() :
+                    ctx.cache().dynamicStartCache(cacheCfg,
                     cacheCfg.getName(),
                     null,
                     false,
@@ -3017,14 +3019,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
-        destroyCache0(cacheName);
+        destroyCache0(cacheName, false);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+    @Override public boolean destroyCache0(String cacheName, boolean sql) throws CacheException {
         CU.validateCacheName(cacheName);
 
-        IgniteInternalFuture<Boolean> stopFut = destroyCacheAsync(cacheName, true);
+        IgniteInternalFuture<Boolean> stopFut = destroyCacheAsync(cacheName, sql, true);
 
         try {
             return stopFut.get();
@@ -3050,10 +3052,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /**
      * @param cacheName Cache name.
+     * @param sql If the cache needs to be destroyed only if it was created by SQL {@code CREATE TABLE} command.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Ignite future.
      */
-    public IgniteInternalFuture<Boolean> destroyCacheAsync(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> destroyCacheAsync(String cacheName, boolean sql, boolean checkThreadTx) {
         CU.validateCacheName(cacheName);
 
         guard();
@@ -3061,7 +3064,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             checkClusterState();
 
-            return ctx.cache().dynamicDestroyCache(cacheName, checkThreadTx);
+            return ctx.cache().dynamicDestroyCache(cacheName, sql, checkThreadTx);
         }
         finally {
             unguard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index cec42a8..889eeda 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -339,6 +339,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 false,
                 action.descriptor().receivedFrom(),
                 action.descriptor().staticallyConfigured(),
+                action.descriptor().sql(),
                 req.deploymentId(),
                 req.schema());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
index 4768a9a..3e2c259 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheData.java
@@ -53,6 +53,9 @@ public class CacheData implements Serializable {
     private final boolean staticCfg;
 
     /** */
+    private final boolean sql;
+
+    /** */
     private final boolean template;
 
     /** Flags added for future usage. */
@@ -66,6 +69,7 @@ public class CacheData implements Serializable {
      * @param schema Query schema.
      * @param rcvdFrom Node ID cache was started from.
      * @param staticCfg {@code True} if cache was statically configured.
+     * @param sql {@code True} if cache was created by an SQL command such as {@code CREATE TABLE}.
      * @param template {@code True} if this is cache template.
      * @param flags Flags (added for future usage).
      */
@@ -76,6 +80,7 @@ public class CacheData implements Serializable {
         QuerySchema schema,
         UUID rcvdFrom,
         boolean staticCfg,
+        boolean sql,
         boolean template,
         byte flags) {
         assert cacheCfg != null;
@@ -90,6 +95,7 @@ public class CacheData implements Serializable {
         this.schema = schema;
         this.rcvdFrom = rcvdFrom;
         this.staticCfg = staticCfg;
+        this.sql = sql;
         this.template = template;
         this.flags = flags;
     }
@@ -130,6 +136,13 @@ public class CacheData implements Serializable {
     }
 
     /**
+     * @return {@code True} if cache was created by an SQL command such as {@code CREATE TABLE}.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
      * @return Cache configuration.
      */
     public CacheConfiguration cacheConfiguration() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
index ea24140..afc01c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java
@@ -106,17 +106,23 @@ class CacheJoinNodeDiscoveryData implements Serializable {
         @GridToStringInclude
         private final CacheType cacheType;
 
+        /** */
+        @GridToStringInclude
+        private final boolean sql;
+
         /** Flags added for future usage. */
         private final byte flags;
 
         /**
          * @param ccfg Cache configuration.
          * @param cacheType Cache type.
+         * @param sql SQL flag - {@code true} if cache was created with {@code CREATE TABLE}.
          * @param flags Flags (for future usage).
          */
-        CacheInfo(CacheConfiguration ccfg, CacheType cacheType, byte flags) {
+        CacheInfo(CacheConfiguration ccfg, CacheType cacheType, boolean sql, byte flags) {
             this.ccfg = ccfg;
             this.cacheType = cacheType;
+            this.sql = sql;
             this.flags = flags;
         }
 
@@ -134,6 +140,13 @@ class CacheJoinNodeDiscoveryData implements Serializable {
             return cacheType;
         }
 
+        /**
+         * @return SQL flag - {@code true} if cache was created with {@code CREATE TABLE}.
+         */
+        boolean sql() {
+            return sql;
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(CacheInfo.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index c372037..7ff5622 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -52,7 +52,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
 
 /**
- * Logic related to cache discovery date processing.
+ * Logic related to cache discovery data processing.
  */
 class ClusterCachesInfo {
     /** */
@@ -112,7 +112,7 @@ class ClusterCachesInfo {
                 CacheData cacheData = gridData.caches().get(locCfg.getName());
 
                 if (cacheData != null)
-                    checkCache(locCfg, cacheData.cacheConfiguration(), cacheData.receivedFrom());
+                    checkCache(locCacheInfo, cacheData, cacheData.receivedFrom());
             }
         }
 
@@ -122,19 +122,23 @@ class ClusterCachesInfo {
     /**
      * Checks that remote caches has configuration compatible with the local.
      *
-     * @param locCfg Local configuration.
-     * @param rmtCfg Remote configuration.
+     * @param locInfo Local configuration.
+     * @param rmtData Remote configuration.
      * @param rmt Remote node.
      * @throws IgniteCheckedException If check failed.
      */
-    private void checkCache(CacheConfiguration<?, ?> locCfg, CacheConfiguration<?, ?> rmtCfg, UUID rmt)
+    @SuppressWarnings("unchecked")
+    private void checkCache(CacheJoinNodeDiscoveryData.CacheInfo locInfo, CacheData rmtData, UUID rmt)
         throws IgniteCheckedException {
-        GridCacheAttributes rmtAttr = new GridCacheAttributes(rmtCfg);
-        GridCacheAttributes locAttr = new GridCacheAttributes(locCfg);
+        GridCacheAttributes rmtAttr = new GridCacheAttributes(rmtData.cacheConfiguration(), rmtData.sql());
+        GridCacheAttributes locAttr = new GridCacheAttributes(locInfo.config(), locInfo.sql());
 
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheMode", "Cache mode",
             locAttr.cacheMode(), rmtAttr.cacheMode(), true);
 
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "sql", "SQL flag",
+            locAttr.sql(), rmtAttr.sql(), true);
+
         if (rmtAttr.cacheMode() != LOCAL) {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "interceptor", "Cache Interceptor",
                 locAttr.interceptorClassName(), rmtAttr.interceptorClassName(), true);
@@ -147,8 +151,8 @@ class ClusterCachesInfo {
 
             ClusterNode rmtNode = ctx.discovery().node(rmt);
 
-            if (CU.affinityNode(ctx.discovery().localNode(), locCfg.getNodeFilter())
-                && rmtNode != null && CU.affinityNode(rmtNode, rmtCfg.getNodeFilter())) {
+            if (CU.affinityNode(ctx.discovery().localNode(), locInfo.config().getNodeFilter())
+                && rmtNode != null && CU.affinityNode(rmtNode, rmtData.cacheConfiguration().getNodeFilter())) {
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                     locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
             }
@@ -248,6 +252,7 @@ class ClusterCachesInfo {
                         true,
                         req.initiatingNodeId(),
                         false,
+                        false,
                         req.deploymentId(),
                         req.schema());
 
@@ -287,6 +292,7 @@ class ClusterCachesInfo {
                             false,
                             req.initiatingNodeId(),
                             false,
+                            req.sql(),
                             req.deploymentId(),
                             req.schema());
 
@@ -377,6 +383,22 @@ class ClusterCachesInfo {
                 assert req.stop() ^ req.close() : req;
 
                 if (desc != null) {
+                    if (req.sql() && !desc.sql()) {
+                        ctx.cache().completeCacheStartFuture(req, false,
+                            new IgniteCheckedException("Only cache created with CREATE TABLE may be removed with " +
+                                "DROP TABLE [cacheName=" + req.cacheName() + ']'));
+
+                        continue;
+                    }
+
+                    if (!req.sql() && desc.sql()) {
+                        ctx.cache().completeCacheStartFuture(req, false,
+                            new IgniteCheckedException("Only cache created with cache API may be removed with " +
+                                "direct call to destroyCache [cacheName=" + req.cacheName() + ']'));
+
+                        continue;
+                    }
+
                     DynamicCacheDescriptor old = registeredCaches.remove(req.cacheName());
 
                     assert old != null : "Dynamic cache map was concurrently modified [req=" + req + ']';
@@ -587,6 +609,7 @@ class ClusterCachesInfo {
                 desc.schema(),
                 desc.receivedFrom(),
                 desc.staticallyConfigured(),
+                desc.sql(),
                 false,
                 (byte)0);
 
@@ -603,6 +626,7 @@ class ClusterCachesInfo {
                 desc.schema(),
                 desc.receivedFrom(),
                 desc.staticallyConfigured(),
+                false,
                 true,
                 (byte)0);
 
@@ -632,6 +656,7 @@ class ClusterCachesInfo {
                 true,
                 cacheData.receivedFrom(),
                 cacheData.staticallyConfigured(),
+                false,
                 cacheData.deploymentId(),
                 cacheData.schema());
 
@@ -648,6 +673,7 @@ class ClusterCachesInfo {
                 false,
                 cacheData.receivedFrom(),
                 cacheData.staticallyConfigured(),
+                cacheData.sql(),
                 cacheData.deploymentId(),
                 cacheData.schema());
 
@@ -707,6 +733,7 @@ class ClusterCachesInfo {
                             desc.template(),
                             desc.receivedFrom(),
                             desc.staticallyConfigured(),
+                            desc.sql(),
                             desc.deploymentId(),
                             desc.schema());
 
@@ -784,6 +811,7 @@ class ClusterCachesInfo {
                     true,
                     nodeId,
                     true,
+                    false,
                     joinData.cacheDeploymentId(),
                     new QuerySchema(cfg.getQueryEntities()));
 
@@ -803,6 +831,7 @@ class ClusterCachesInfo {
                     false,
                     nodeId,
                     true,
+                    cacheInfo.sql(),
                     joinData.cacheDeploymentId(),
                     new QuerySchema(cfg.getQueryEntities()));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index f8c2c7d..5434061 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -70,6 +70,9 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Close flag. */
     private boolean close;
 
+    /** Whether cache was created through SQL. */
+    private boolean sql;
+
     /** Fail if exists flag. */
     private boolean failIfExists;
 
@@ -165,12 +168,14 @@ public class DynamicCacheChangeRequest implements Serializable {
     /**
      * @param ctx Context.
      * @param cacheName Cache name.
+     * @param sql {@code true} if the cache must be stopped only if it was created by SQL command {@code CREATE TABLE}.
      * @param destroy Destroy flag.
      * @return Cache stop request.
      */
-    static DynamicCacheChangeRequest stopRequest(GridKernalContext ctx, String cacheName, boolean destroy) {
+    static DynamicCacheChangeRequest stopRequest(GridKernalContext ctx, String cacheName, boolean sql, boolean destroy) {
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
+        req.sql(sql);
         req.stop(true);
         req.destroy(destroy);
 
@@ -381,6 +386,20 @@ public class DynamicCacheChangeRequest implements Serializable {
     }
 
     /**
+     * @return SQL flag.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
+     * @param sql New SQL flag.
+     */
+    public void sql(boolean sql) {
+        this.sql = sql;
+    }
+
+    /**
      * @param nodeId ID of node provided cache configuration in discovery data.
      */
     public void receivedFrom(UUID nodeId) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 40d3706..a682f63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -48,6 +48,9 @@ public class DynamicCacheDescriptor {
     /** Statically configured flag. */
     private final boolean staticCfg;
 
+    /** SQL flag - whether the cache is created by an SQL command such as {@code CREATE TABLE}. */
+    private boolean sql;
+
     /** Cache type. */
     private CacheType cacheType;
 
@@ -97,6 +100,7 @@ public class DynamicCacheDescriptor {
      * @param template {@code True} if this is template configuration.
      * @param rcvdFrom ID of node provided cache configuration
      * @param staticCfg {@code True} if cache statically configured.
+     * @param sql SQL flag - whether the cache is created by an SQL command such as {@code CREATE TABLE}.
      * @param deploymentId Deployment ID.
      * @param schema Query schema.
      */
@@ -107,6 +111,7 @@ public class DynamicCacheDescriptor {
         boolean template,
         UUID rcvdFrom,
         boolean staticCfg,
+        boolean sql,
         IgniteUuid deploymentId,
         QuerySchema schema) {
         assert cacheCfg != null;
@@ -123,6 +128,7 @@ public class DynamicCacheDescriptor {
         this.template = template;
         this.rcvdFrom = rcvdFrom;
         this.staticCfg = staticCfg;
+        this.sql = sql;
         this.deploymentId = deploymentId;
 
         pluginMgr = new CachePluginManager(ctx, cacheCfg);
@@ -170,6 +176,13 @@ public class DynamicCacheDescriptor {
     }
 
     /**
+     * @return SQL flag.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
      * @return Cache name.
      */
     public String cacheName() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 1caf60d..4af2518 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -47,11 +47,16 @@ public class GridCacheAttributes implements Serializable {
     /** Cache configuration. */
     private CacheConfiguration ccfg;
 
+    /** SQL flag - whether the cache is created by an SQL command such as {@code CREATE TABLE}. */
+    private boolean sql;
+
     /**
      * @param cfg Cache configuration.
      */
-    public GridCacheAttributes(CacheConfiguration cfg) {
+    public GridCacheAttributes(CacheConfiguration cfg, boolean sql) {
         ccfg = cfg;
+
+        this.sql = sql;
     }
 
     /**
@@ -273,6 +278,13 @@ public class GridCacheAttributes implements Serializable {
     }
 
     /**
+     * @return SQL flag.
+     */
+    public boolean sql() {
+        return sql;
+    }
+
+    /**
      * @param obj Object to get class of.
      * @return Class name or {@code null}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 9a25672..dbcddf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -120,6 +120,7 @@ import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -675,11 +676,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cfg Cache configuration.
+     * @param sql SQL flag - {@code true} if cache was created with {@code CREATE TABLE}.
      * @param caches Caches map.
      * @param templates Templates map.
      * @throws IgniteCheckedException If failed.
      */
-    private void addCacheOnJoin(CacheConfiguration cfg,
+    private void addCacheOnJoin(CacheConfiguration cfg, boolean sql,
         Map<String, CacheJoinNodeDiscoveryData.CacheInfo> caches,
         Map<String, CacheJoinNodeDiscoveryData.CacheInfo> templates) throws IgniteCheckedException {
         CU.validateCacheName(cfg.getName());
@@ -716,10 +718,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             else
                 stopSeq.addFirst(cfg.getName());
 
-            caches.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, cacheType, (byte)0));
+            caches.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, cacheType, sql, (byte)0));
         }
         else
-            templates.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, CacheType.USER, (byte)0));
+            templates.put(cfg.getName(), new CacheJoinNodeDiscoveryData.CacheInfo(cfg, CacheType.USER, false, (byte)0));
     }
 
     /**
@@ -740,7 +742,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             cfgs[i] = cfg; // Replace original configuration value.
 
-            addCacheOnJoin(cfg, caches, templates);
+            addCacheOnJoin(cfg, false, caches, templates);
         }
     }
 
@@ -769,8 +771,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 for (String name : savedCacheNames) {
                     CacheConfiguration cfg = sharedCtx.pageStore().readConfiguration(name);
 
+                    // TODO IGNITE-5306 - set correct SQL flag below.
                     if (cfg != null)
-                        addCacheOnJoin(cfg, caches, templates);
+                        addCacheOnJoin(cfg, false, caches, templates);
                 }
             }
         }
@@ -2218,18 +2221,41 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheName,
             nearCfg,
             CacheType.USER,
+            false,
             failIfExists,
             failIfNotStarted,
             checkThreadTx);
     }
 
     /**
+     * Dynamically starts cache as a result of SQL {@code CREATE TABLE} command.
+     *
+     * @param ccfg Cache configuration.
+     */
+    @SuppressWarnings("IfMayBeConditional")
+    public IgniteInternalFuture<Boolean> dynamicStartSqlCache(
+        CacheConfiguration ccfg
+    ) {
+        A.notNull(ccfg, "ccfg");
+
+        return dynamicStartCache(ccfg,
+            ccfg.getName(),
+            ccfg.getNearConfiguration(),
+            CacheType.USER,
+            true,
+            false,
+            true,
+            true);
+    }
+
+    /**
      * Dynamically starts cache.
      *
      * @param ccfg Cache configuration.
      * @param cacheName Cache name.
      * @param nearCfg Near cache configuration.
      * @param cacheType Cache type.
+     * @param sql If the cache needs to be created as the result of SQL {@code CREATE TABLE} command.
      * @param failIfExists Fail if exists flag.
      * @param failIfNotStarted If {@code true} fails if cache is not started.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
@@ -2241,6 +2267,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
         CacheType cacheType,
+        boolean sql,
         boolean failIfExists,
         boolean failIfNotStarted,
         boolean checkThreadTx
@@ -2256,6 +2283,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cacheName,
                 nearCfg,
                 cacheType,
+                sql,
                 failIfExists,
                 failIfNotStarted);
 
@@ -2309,6 +2337,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     ccfg.getName(),
                     null,
                     cacheType,
+                    false,
                     failIfExists,
                     true
                 );
@@ -2337,16 +2366,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cacheName Cache name to destroy.
+     * @param sql If the cache needs to be destroyed only if it was created as the result
+     *     of SQL {@code CREATE TABLE} command.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Future that will be completed when cache is destroyed.
      */
-    public IgniteInternalFuture<Boolean> dynamicDestroyCache(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> dynamicDestroyCache(String cacheName, boolean sql, boolean checkThreadTx) {
         assert cacheName != null;
 
         if (checkThreadTx)
             checkEmptyTransactions();
 
-        DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true);
+        DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, sql, true);
 
         return F.first(initiateCacheChanges(F.asList(req), false));
     }
@@ -2363,7 +2394,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         List<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheNames.size());
 
         for (String cacheName : cacheNames) {
-            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true);
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false, true);
 
             reqs.add(req);
         }
@@ -2473,7 +2504,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         List<DynamicCacheChangeRequest> reqs = new ArrayList<>();
 
         for (String cacheName : cacheNames()) {
-            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false);
+            DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false, false);
 
             reqs.add(req);
         }
@@ -3366,6 +3397,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param cacheName Cache name
      * @param nearCfg Near cache configuration
      * @param cacheType Cache type
+     * @param sql Whether the cache needs to be created as the result of SQL {@code CREATE TABLE} command.
      * @param failIfExists Fail if exists flag.
      * @param failIfNotStarted If {@code true} fails if cache is not started.
      * @return Request or {@code null} if cache already exists.
@@ -3377,6 +3409,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
         CacheType cacheType,
+        boolean sql,
         boolean failIfExists,
         boolean failIfNotStarted
     ) throws IgniteCheckedException {
@@ -3384,6 +3417,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(UUID.randomUUID(), cacheName, ctx.localNodeId());
 
+        req.sql(sql);
+
         req.failIfExists(failIfExists);
 
         if (ccfg != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index a7d0d19..78216db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -2359,7 +2359,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         IgniteInternalFuture<?> fut;
 
         try {
-            fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), true);
+            fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), false, true);
         }
         finally {
             onLeave(gate);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index eb0981b..df9d269 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -998,6 +998,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
                 null,
                 CacheType.INTERNAL,
                 false,
+                false,
                 true,
                 true).get();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 4187e2b..320c90b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1303,7 +1303,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         // Preserve user specified names as they are.
         newCfg.setSqlEscapeAll(true);
 
-        boolean res = ctx.grid().getOrCreateCache0(newCfg).get2();
+        boolean res = ctx.grid().getOrCreateCache0(newCfg, true).get2();
 
         if (!res && !ifNotExists)
             throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_EXISTS,  entity.getTableName());
@@ -1319,7 +1319,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("unchecked")
     public void dynamicTableDrop(String cacheName, String tblName, boolean ifExists) throws SchemaOperationException {
-        boolean res = ctx.grid().destroyCache0(cacheName);
+        boolean res = ctx.grid().destroyCache0(cacheName, true);
 
         if (!res && !ifExists)
             throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 3b9fa69..da260b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -379,7 +379,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
             switch (cmd) {
                 case DESTROY_CACHE: {
                     // Do not check thread tx here since there can be active system cache txs.
-                    fut = ((IgniteKernal)ctx.grid()).destroyCacheAsync(cacheName, false).chain(
+                    fut = ((IgniteKernal)ctx.grid()).destroyCacheAsync(cacheName, false, false).chain(
                         new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
                             @Override public GridRestResponse applyx(IgniteInternalFuture<?> f)
                                 throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index e7c5ca5..ea3b9e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -198,7 +198,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             @Override public Object call() throws Exception {
-                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true));
+                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true));
 
                 return null;
             }
@@ -266,7 +266,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             @Override public Object call() throws Exception {
                 IgniteEx kernal = grid(ThreadLocalRandom.current().nextInt(nodeCount()));
 
-                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true));
+                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true));
 
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index ed0af3c..b4fcc70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -323,14 +323,14 @@ public class IgfsIgniteMock implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
-        CacheConfiguration<K, V> cacheCfg) {
+        CacheConfiguration<K, V> cacheCfg, boolean sql) {
         throwUnsupported();
 
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+    @Override public boolean destroyCache0(String cacheName, boolean sql) throws CacheException {
         throwUnsupported();
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index f459e57..89ab282 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -536,7 +536,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
-        CacheConfiguration<K, V> cacheCfg) {
+        CacheConfiguration<K, V> cacheCfg, boolean sql) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
@@ -546,7 +546,7 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+    @Override public boolean destroyCache0(String cacheName, boolean sql) throws CacheException {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a0a84a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
index b072fdb..4d74bac 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -23,7 +23,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.Callable;
 
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
@@ -34,6 +36,7 @@ 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.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
@@ -59,8 +62,6 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         for (IgniteConfiguration cfg : configurations())
             Ignition.start(cfg);
 
-        client().getOrCreateCache(cacheConfigurationForIndexing());
-
         client().addCacheConfiguration(cacheConfiguration());
     }
 
@@ -72,8 +73,16 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        client().getOrCreateCache(cacheConfigurationForIndexing());
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        client().destroyCache("Person");
+        if (client().cache("Person") != null)
+            cache().query(new SqlFieldsQuery("DROP TABLE IF EXISTS \"Person\".\"Person\""));
 
         super.afterTest();
     }
@@ -92,6 +101,12 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
 
             assertNotNull(node.cache("Person"));
 
+            DynamicCacheDescriptor cacheDesc = node.context().cache().cacheDescriptor("Person");
+
+            assertNotNull(cacheDesc);
+
+            assertTrue(cacheDesc.sql());
+
             QueryTypeDescriptorImpl desc = typeExisting(node, "Person", "Person");
 
             assertEquals(Object.class, desc.keyClass());
@@ -142,6 +157,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * Test that attempting to {@code CREATE TABLE} that already exists yields an error.
      * @throws Exception if failed.
      */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     public void testCreateExistingTable() throws Exception {
         cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
             " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
@@ -193,6 +209,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * Test that attempting to {@code DROP TABLE} that does not exist yields an error.
      * @throws Exception if failed.
      */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     public void testDropMissingTable() throws Exception {
         GridTestUtils.assertThrows(null, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -204,6 +221,62 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
+     * Check that {@code DROP TABLE} for caches not created with {@code CREATE TABLE} yields an error.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDropNonDynamicTable() throws Exception {
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                cache().query(new SqlFieldsQuery("DROP TABLE \"Integer\""));
+
+                return null;
+            }
+        }, IgniteSQLException.class,
+        "Only cache created with CREATE TABLE may be removed with DROP TABLE [cacheName=cache_idx]");
+    }
+
+    /**
+     * Test that attempting to destroy via cache API a cache created via SQL yields an error.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDestroyDynamicSqlCache() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                client().destroyCache("Person");
+
+                return null;
+            }
+        }, CacheException.class,
+        "Only cache created with cache API may be removed with direct call to destroyCache [cacheName=Person]");
+    }
+
+    /**
+     * Test that attempting to start a node that has a cache with the name already present in the grid and whose
+     * SQL flag does not match that of cache with the same name that is already started, yields an error.
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testSqlFlagCompatibilityCheck() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Ignition.start(clientConfiguration(5).setCacheConfiguration(new CacheConfiguration("Person")));
+
+                return null;
+            }
+        }, IgniteException.class, "SQL flag mismatch (fix sql flag in cache configuration");
+    }
+
+    /**
      * Check that a property in given descriptor is present and has parameters as expected.
      * @param desc Descriptor.
      * @param name Property name.