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/26 09:47:00 UTC

[21/28] ignite git commit: IGNITE-5052: DDL: Implemented CREATE TABLE and DROP TABLE commands. This closes #1958.

IGNITE-5052: DDL: Implemented CREATE TABLE and DROP TABLE commands. This closes #1958.


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

Branch: refs/heads/ignite-5075-cc
Commit: e8f5af333a6f02f103a257058b2740f6a510a367
Parents: af026d1
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu May 25 13:48:40 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu May 25 13:48:40 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   5 +-
 .../org/apache/ignite/internal/IgniteEx.java    |  27 ++
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../processors/cache/ClusterCachesInfo.java     |  10 +-
 .../processors/cache/ExchangeActions.java       |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  43 ++-
 .../processors/query/GridQueryProcessor.java    |  52 +++
 .../handlers/cache/GridCacheCommandHandler.java |   8 +-
 .../processors/igfs/IgfsIgniteMock.java         |  17 +
 .../junits/multijvm/IgniteProcessProxy.java     |  13 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |  55 +++-
 .../query/h2/sql/GridSqlCreateTable.java        | 153 +++++++++
 .../query/h2/sql/GridSqlDropTable.java          |  79 +++++
 .../query/h2/sql/GridSqlQueryParser.java        | 260 +++++++++++++++
 .../cache/index/H2DynamicTableSelfTest.java     | 320 +++++++++++++++++++
 .../query/h2/sql/GridQueryParsingTest.java      | 284 +++++++++++++++-
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 17 files changed, 1326 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 11fc43a..1136c71 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -1794,9 +1794,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @return {@code this} for chaining.
      */
     public CacheConfiguration<K, V> setQueryEntities(Collection<QueryEntity> qryEntities) {
-        if (this.qryEntities == null)
+        if (this.qryEntities == null) {
             this.qryEntities = new ArrayList<>(qryEntities);
 
+            return this;
+        }
+
         for (QueryEntity entity : qryEntities) {
             boolean found = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 421d6f9..8c27027 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
@@ -18,13 +18,17 @@
 package org.apache.ignite.internal;
 
 import java.util.Collection;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
@@ -60,6 +64,29 @@ public interface IgniteEx extends Ignite {
     public Collection<IgniteInternalCache<?, ?>> cachesx(@Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p);
 
     /**
+     * Gets existing cache with the given name or creates new one with the given configuration.
+     * <p>
+     * If a cache with the same name already exists, this method will not check that the given
+     * configuration matches the configuration of existing cache and will return an instance
+     * of the existing cache.
+     *
+     * @param cacheCfg Cache configuration to use.
+     * @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;
+
+    /**
+     * Stops dynamically started cache.
+     *
+     * @param cacheName Cache name to stop.
+     * @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;
+
+    /**
      * Checks if the event type is user-recordable.
      *
      * @param type Event type to check.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 c36fd7e..0f43169 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
@@ -46,6 +46,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.CacheException;
 import javax.management.JMException;
 import javax.management.ObjectName;
 import org.apache.ignite.IgniteAtomicLong;
@@ -163,6 +164,7 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
@@ -2789,6 +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();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
+        CacheConfiguration<K, V> cacheCfg) {
         A.notNull(cacheCfg, "cacheCfg");
         CU.validateCacheName(cacheCfg.getName());
 
@@ -2797,8 +2806,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             checkClusterState();
 
+            Boolean res = false;
+
             if (ctx.cache().cache(cacheCfg.getName()) == null) {
-                ctx.cache().dynamicStartCache(cacheCfg,
+                res = ctx.cache().dynamicStartCache(cacheCfg,
                     cacheCfg.getName(),
                     null,
                     false,
@@ -2806,7 +2817,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     true).get();
             }
 
-            return ctx.cache().publicJCache(cacheCfg.getName());
+            return new IgniteBiTuple<>((IgniteCache<K, V>)ctx.cache().publicJCache(cacheCfg.getName()), res);
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -3006,12 +3017,17 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
+        destroyCache0(cacheName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean destroyCache0(String cacheName) throws CacheException {
         CU.validateCacheName(cacheName);
 
-        IgniteInternalFuture stopFut = destroyCacheAsync(cacheName, true);
+        IgniteInternalFuture<Boolean> stopFut = destroyCacheAsync(cacheName, true);
 
         try {
-            stopFut.get();
+            return stopFut.get();
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -3037,7 +3053,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
      * @return Ignite future.
      */
-    public IgniteInternalFuture<?> destroyCacheAsync(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> destroyCacheAsync(String cacheName, boolean checkThreadTx) {
         CU.validateCacheName(cacheName);
 
         guard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 7150cf4..ab5cf37 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
@@ -272,7 +272,7 @@ class ClusterCachesInfo {
             if (req.start()) {
                 if (desc == null) {
                     if (req.clientStartOnly()) {
-                        ctx.cache().completeCacheStartFuture(req, new IgniteCheckedException("Failed to start " +
+                        ctx.cache().completeCacheStartFuture(req, false, new IgniteCheckedException("Failed to start " +
                             "client cache (a cache with the given name is not started): " + req.cacheName()));
                     }
                     else {
@@ -327,7 +327,7 @@ class ClusterCachesInfo {
                     }
                     else {
                         if (req.failIfExists()) {
-                            ctx.cache().completeCacheStartFuture(req,
+                            ctx.cache().completeCacheStartFuture(req, false,
                                 new CacheExistsException("Failed to start cache " +
                                     "(a cache with the same name is already started): " + req.cacheName()));
                         }
@@ -420,7 +420,7 @@ class ClusterCachesInfo {
         if (!F.isEmpty(reqsToComplete)) {
             ctx.closure().callLocalSafe(new Callable<Void>() {
                 @Override public Void call() throws Exception {
-                    for (T2<DynamicCacheChangeRequest, AffinityTopologyVersion> t :reqsToComplete) {
+                    for (T2<DynamicCacheChangeRequest, AffinityTopologyVersion> t : reqsToComplete) {
                         final DynamicCacheChangeRequest req = t.get1();
                         AffinityTopologyVersion waitTopVer = t.get2();
 
@@ -428,11 +428,11 @@ class ClusterCachesInfo {
                             ctx.cache().context().exchange().affinityReadyFuture(waitTopVer) : null;
 
                         if (fut == null || fut.isDone())
-                            ctx.cache().completeCacheStartFuture(req, null);
+                            ctx.cache().completeCacheStartFuture(req, false, null);
                         else {
                             fut.listen(new IgniteInClosure<IgniteInternalFuture<?>>() {
                                 @Override public void apply(IgniteInternalFuture<?> fut) {
-                                    ctx.cache().completeCacheStartFuture(req, null);
+                                    ctx.cache().completeCacheStartFuture(req, false, null);
                                 }
                             });
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index eac1120..5ac51ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -132,7 +132,7 @@ public class ExchangeActions {
     private void completeRequestFutures(Map<String, ActionData> map, GridCacheSharedContext ctx) {
         if (map != null) {
             for (ActionData req : map.values())
-                ctx.cache().completeCacheStartFuture(req.req, null);
+                ctx.cache().completeCacheStartFuture(req.req, true, null);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 a716ff4..d94c41f 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
@@ -1967,12 +1967,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param req Request to complete future for.
      * @param err Error if any.
      */
-    void completeCacheStartFuture(DynamicCacheChangeRequest req, @Nullable Exception err) {
+    void completeCacheStartFuture(DynamicCacheChangeRequest req, boolean success, @Nullable Exception err) {
         if (req.initiatingNodeId().equals(ctx.localNodeId())) {
             DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(req.requestId());
 
             if (fut != null)
-                fut.onDone(null, err);
+                fut.onDone(success, err);
         }
     }
 
@@ -2065,7 +2065,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     public IgniteInternalFuture<?> createFromTemplate(String cacheName) {
         try {
-            CacheConfiguration cfg = createConfigFromTemplate(cacheName);
+            CacheConfiguration cfg = getOrCreateConfigFromTemplate(cacheName);
 
             return dynamicStartCache(cfg, cacheName, null, true, true, true);
         }
@@ -2088,7 +2088,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (publicJCache(cacheName, false, checkThreadTx) != null) // Cache with given name already started.
                 return new GridFinishedFuture<>();
 
-            CacheConfiguration cfg = createConfigFromTemplate(cacheName);
+            CacheConfiguration cfg = getOrCreateConfigFromTemplate(cacheName);
 
             return dynamicStartCache(cfg, cacheName, null, false, true, checkThreadTx);
         }
@@ -2099,10 +2099,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param cacheName Cache name.
-     * @return Cache configuration.
+     * @return Cache configuration, or {@code null} if no template with matching name found.
      * @throws IgniteCheckedException If failed.
      */
-    private CacheConfiguration createConfigFromTemplate(String cacheName) throws IgniteCheckedException {
+    public CacheConfiguration getConfigFromTemplate(String cacheName) throws IgniteCheckedException {
         CacheConfiguration cfgTemplate = null;
 
         CacheConfiguration dfltCacheCfg = null;
@@ -2160,7 +2160,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cfgTemplate == null)
             cfgTemplate = dfltCacheCfg;
 
-        cfgTemplate = cfgTemplate == null ? new CacheConfiguration() : cloneCheckSerializable(cfgTemplate);
+        if (cfgTemplate == null)
+            return null;
+
+        cfgTemplate = cloneCheckSerializable(cfgTemplate);
 
         CacheConfiguration cfg = new CacheConfiguration(cfgTemplate);
 
@@ -2170,6 +2173,20 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     * @throws IgniteCheckedException If failed.
+     */
+    private CacheConfiguration getOrCreateConfigFromTemplate(String cacheName) throws IgniteCheckedException {
+        CacheConfiguration cfg = getConfigFromTemplate(cacheName);
+
+        if (cfg != null)
+            return cfg;
+        else
+            return new CacheConfiguration(cacheName);
+    }
+
+    /**
      * Dynamically starts cache.
      *
      * @param ccfg Cache configuration.
@@ -2181,7 +2198,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when cache is deployed.
      */
     @SuppressWarnings("IfMayBeConditional")
-    public IgniteInternalFuture<?> dynamicStartCache(
+    public IgniteInternalFuture<Boolean> dynamicStartCache(
         @Nullable CacheConfiguration ccfg,
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
@@ -2211,7 +2228,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when cache is deployed.
      */
     @SuppressWarnings("IfMayBeConditional")
-    public IgniteInternalFuture<?> dynamicStartCache(
+    public IgniteInternalFuture<Boolean> dynamicStartCache(
         @Nullable CacheConfiguration ccfg,
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
@@ -2315,7 +2332,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @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<?> dynamicDestroyCache(String cacheName, boolean checkThreadTx) {
+    public IgniteInternalFuture<Boolean> dynamicDestroyCache(String cacheName, boolean checkThreadTx) {
         assert cacheName != null;
 
         if (checkThreadTx)
@@ -2517,7 +2534,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     if (desc == null)
                         // No-op.
-                        fut.onDone();
+                        fut.onDone(false);
                     else {
                         assert desc.cacheConfiguration() != null : desc;
 
@@ -3452,7 +3469,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      *
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    private class DynamicCacheStartFuture extends GridFutureAdapter<Object> {
+    private class DynamicCacheStartFuture extends GridFutureAdapter<Boolean> {
         /** Cache name. */
         private String cacheName;
 
@@ -3477,7 +3494,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
+        @Override public boolean onDone(@Nullable Boolean res, @Nullable Throwable err) {
             // Make sure to remove future before completion.
             pendingFuts.remove(req.requestId(), this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 a40c9e9..09dbe42 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
@@ -1266,6 +1266,58 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Create cache and table from given query entity.
+     *
+     * @param schemaName Schema name to create table in.
+     * @param entity Entity to create table from.
+     * @param templateCacheName Cache name to take settings from.
+     * @param ifNotExists Quietly ignore this command if table already exists.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void dynamicTableCreate(String schemaName, QueryEntity entity, String templateCacheName, boolean ifNotExists)
+        throws IgniteCheckedException {
+        CacheConfiguration<?, ?> templateCfg = ctx.cache().getConfigFromTemplate(templateCacheName);
+
+        if (templateCfg == null)
+            throw new SchemaOperationException(SchemaOperationException.CODE_CACHE_NOT_FOUND, templateCacheName);
+
+        if (!F.isEmpty(templateCfg.getQueryEntities()))
+            throw new SchemaOperationException("Template cache already contains query entities which it should not " +
+                "[cacheName=" + templateCacheName + ']');
+
+        CacheConfiguration<?, ?> newCfg = new CacheConfiguration<>(templateCfg);
+
+        newCfg.setName(entity.getTableName());
+
+        newCfg.setQueryEntities(Collections.singleton(entity));
+
+        // We want to preserve user specified names as they are
+        newCfg.setSqlEscapeAll(true);
+
+        boolean res = ctx.grid().getOrCreateCache0(newCfg).get2();
+
+        if (!res && !ifNotExists)
+            throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_EXISTS,  entity.getTableName());
+    }
+
+    /**
+     * Drop table by destroying its cache if it's an 1:1 per cache table.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param ifExists Quietly ignore this command if table does not exist.
+     * @throws SchemaOperationException if {@code ifExists} is {@code false} and cache was not found.
+     */
+    @SuppressWarnings("unchecked")
+    public void dynamicTableDrop(String schemaName, String tblName, boolean ifExists) throws SchemaOperationException {
+        boolean res = ctx.grid().destroyCache0(tblName);
+
+        if (!res && !ifExists)
+            throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName);
+    }
+
+    /**
      * Register cache in indexing SPI.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 9edf37e..3b9fa69 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
@@ -383,7 +383,9 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                         new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
                             @Override public GridRestResponse applyx(IgniteInternalFuture<?> f)
                                 throws IgniteCheckedException {
-                                return new GridRestResponse(f.get());
+                                f.get();
+
+                                return new GridRestResponse(null);
                             }
                         });
 
@@ -396,7 +398,9 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                         new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
                             @Override public GridRestResponse applyx(IgniteInternalFuture<?> f)
                                 throws IgniteCheckedException {
-                                return new GridRestResponse(f.get());
+                                f.get();
+
+                                return new GridRestResponse(null);
                             }
                         });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 ecba1bf..cc058b1 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
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.plugin.IgnitePlugin;
@@ -62,6 +63,7 @@ import org.jetbrains.annotations.Nullable;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.concurrent.ExecutorService;
+import javax.cache.CacheException;
 
 /**
  * Mocked Ignite implementation for IGFS tests.
@@ -320,6 +322,21 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
+        CacheConfiguration<K, V> cacheCfg) {
+        throwUnsupported();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+        throwUnsupported();
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
         throwUnsupported();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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 26c86dc..34ca22f 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
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteAtomicReference;
@@ -73,6 +74,7 @@ import org.apache.ignite.internal.util.typedef.G;
 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.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -533,11 +535,22 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public <K, V> IgniteBiTuple<IgniteCache<K, V>, Boolean> getOrCreateCache0(
+        CacheConfiguration<K, V> cacheCfg) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
+    @Override public boolean destroyCache0(String cacheName) throws CacheException {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void destroyCaches(Collection<String> cacheNames) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index 3991777..7157d1f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -23,6 +23,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.internal.GridKernalContext;
@@ -34,16 +35,23 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.h2.command.Prepared;
 import org.h2.command.ddl.CreateIndex;
+import org.h2.command.ddl.CreateTable;
 import org.h2.command.ddl.DropIndex;
+import org.h2.command.ddl.DropTable;
 import org.h2.jdbc.JdbcPreparedStatement;
+import org.h2.table.Column;
+import org.h2.value.DataType;
 
 import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
 
@@ -137,11 +145,27 @@ public class DdlStatementsProcessor {
                             dropIdx.indexName());
                 }
             }
+            else if (gridStmt instanceof GridSqlCreateTable) {
+                GridSqlCreateTable createTbl = (GridSqlCreateTable)gridStmt;
+
+                ctx.query().dynamicTableCreate(createTbl.schemaName(), toQueryEntity(createTbl),
+                    createTbl.templateCacheName(), createTbl.ifNotExists());
+
+                fut = null;
+            }
+            else if (gridStmt instanceof GridSqlDropTable) {
+                GridSqlDropTable dropTbl = (GridSqlDropTable)gridStmt;
+
+                ctx.query().dynamicTableDrop(dropTbl.schemaName(), dropTbl.tableName(), dropTbl.ifExists());
+
+                fut = null;
+            }
             else
                 throw new IgniteSQLException("Unsupported DDL operation: " + sql,
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
-            fut.get();
+            if (fut != null)
+                fut.get();
 
             QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
                 (Collections.singletonList(0L)), null, false);
@@ -211,10 +235,37 @@ public class DdlStatementsProcessor {
     }
 
     /**
+     * Convert this statement to query entity and do Ignite specific sanity checks on the way.
+     * @return Query entity mimicking this SQL statement.
+     */
+    private static QueryEntity toQueryEntity(GridSqlCreateTable createTbl) {
+        QueryEntity res = new QueryEntity();
+
+        res.setTableName(createTbl.tableName());
+
+        for (Map.Entry<String, GridSqlColumn> e : createTbl.columns().entrySet()) {
+            GridSqlColumn gridCol = e.getValue();
+
+            Column col = gridCol.column();
+
+            res.addQueryField(e.getKey(), DataType.getTypeClassName(col.getType()), null);
+        }
+
+        res.setKeyType(createTbl.tableName() + "Key");
+
+        res.setValueType(createTbl.tableName());
+
+        res.setKeyFields(createTbl.primaryKeyColumns());
+
+        return res;
+    }
+
+    /**
      * @param cmd Statement.
      * @return Whether {@code cmd} is a DDL statement we're able to handle.
      */
     public static boolean isDdlStatement(Prepared cmd) {
-        return cmd instanceof CreateIndex || cmd instanceof DropIndex;
+        return cmd instanceof CreateIndex || cmd instanceof DropIndex || cmd instanceof CreateTable ||
+            cmd instanceof DropTable;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
new file mode 100644
index 0000000..52c9cc9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
@@ -0,0 +1,153 @@
+/*
+ * 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.sql;
+
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+
+/**
+ * CREATE TABLE statement.
+ */
+public class GridSqlCreateTable extends GridSqlStatement {
+    /**
+     * Schema name upon which this statement has been issued - <b>not</b> the name of the schema where this new table
+     * will be created. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Cache name upon which new cache configuration for this table must be based. */
+    private String tplCacheName;
+
+    /** Quietly ignore this command if table already exists. */
+    private boolean ifNotExists;
+
+    /** Columns. */
+    private LinkedHashMap<String, GridSqlColumn> cols;
+
+    /** Primary key columns. */
+    private LinkedHashSet<String> pkCols;
+
+    /** Extra WITH-params. */
+    private List<String> params;
+
+    /**
+     * @return Cache name upon which new cache configuration for this table must be based.
+     */
+    public String templateCacheName() {
+        return tplCacheName;
+    }
+
+    /**
+     * @param tplCacheName Cache name upon which new cache configuration for this table must be based.
+     */
+    public void templateCacheName(String tplCacheName) {
+        this.tplCacheName = tplCacheName;
+    }
+
+    /**
+     * @return Columns.
+     */
+    public LinkedHashMap<String, GridSqlColumn> columns() {
+        return cols;
+    }
+
+    /**
+     * @param cols Columns.
+     */
+    public void columns(LinkedHashMap<String, GridSqlColumn> cols) {
+        this.cols = cols;
+    }
+
+    /**
+     * @return Primary key columns.
+     */
+    public LinkedHashSet<String> primaryKeyColumns() {
+        return pkCols;
+    }
+
+    /**
+     * @param pkCols Primary key columns.
+     */
+    public void primaryKeyColumns(LinkedHashSet<String> pkCols) {
+        this.pkCols = pkCols;
+    }
+
+    /**
+     * @return Schema name upon which this statement has been issued.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name upon which this statement has been issued.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Quietly ignore this command if table already exists.
+     */
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+    /**
+     * @param ifNotExists Quietly ignore this command if table already exists.
+     */
+    public void ifNotExists(boolean ifNotExists) {
+        this.ifNotExists = ifNotExists;
+    }
+
+    /**
+     * @return Extra WITH-params.
+     */
+    public List<String> params() {
+        return params;
+    }
+
+    /**
+     * @param params Extra WITH-params.
+     */
+    public void params(List<String> params) {
+        this.params = params;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java
new file mode 100644
index 0000000..34cb6fe
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropTable.java
@@ -0,0 +1,79 @@
+/*
+ * 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.sql;
+
+/**
+ * DROP TABLE statement.
+ */
+public class GridSqlDropTable extends GridSqlStatement {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Quietly ignore this command if table does not exist. */
+    private boolean ifExists;
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Quietly ignore this command if table does not exist.
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @param ifExists Quietly ignore this command if table does not exist.
+     */
+    public void ifExists(boolean ifExists) {
+        this.ifExists = ifExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 71c77ac..f310e0f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import javax.cache.CacheException;
@@ -32,11 +33,18 @@ import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.F;
 import org.h2.command.Command;
 import org.h2.command.CommandContainer;
 import org.h2.command.Prepared;
+import org.h2.command.ddl.AlterTableAddConstraint;
 import org.h2.command.ddl.CreateIndex;
+import org.h2.command.ddl.CreateTable;
+import org.h2.command.ddl.CreateTableData;
+import org.h2.command.ddl.DefineCommand;
 import org.h2.command.ddl.DropIndex;
+import org.h2.command.ddl.DropTable;
 import org.h2.command.ddl.SchemaCommand;
 import org.h2.command.dml.Delete;
 import org.h2.command.dml.Explain;
@@ -46,6 +54,7 @@ import org.h2.command.dml.Query;
 import org.h2.command.dml.Select;
 import org.h2.command.dml.SelectUnion;
 import org.h2.command.dml.Update;
+import org.h2.engine.Constants;
 import org.h2.engine.FunctionAlias;
 import org.h2.expression.Aggregate;
 import org.h2.expression.Alias;
@@ -353,6 +362,47 @@ public class GridSqlQueryParser {
     private static final Getter<SchemaCommand, Schema> SCHEMA_COMMAND_SCHEMA = getter(SchemaCommand.class, "schema");
 
     /** */
+    private static final Getter<CreateTable, CreateTableData> CREATE_TABLE_DATA = getter(CreateTable.class, "data");
+
+    /** */
+    private static final Getter<CreateTable, ArrayList<DefineCommand>> CREATE_TABLE_CONSTRAINTS =
+        getter(CreateTable.class, "constraintCommands");
+
+    /** */
+    private static final Getter<CreateTable, IndexColumn[]> CREATE_TABLE_PK = getter(CreateTable.class,
+        "pkColumns");
+
+    /** */
+    private static final Getter<CreateTable, Boolean> CREATE_TABLE_IF_NOT_EXISTS = getter(CreateTable.class,
+        "ifNotExists");
+
+    /** */
+    private static final Getter<CreateTable, Query> CREATE_TABLE_QUERY = getter(CreateTable.class, "asQuery");
+
+    /** */
+    private static final Getter<DropTable, Boolean> DROP_TABLE_IF_EXISTS = getter(DropTable.class, "ifExists");
+
+    /** */
+    private static final Getter<DropTable, String> DROP_TABLE_NAME = getter(DropTable.class, "tableName");
+
+    /** */
+    private static final Getter<Column, Boolean> COLUMN_IS_COMPUTED = getter(Column.class, "isComputed");
+
+    /** */
+    private static final Getter<Column, Expression> COLUMN_CHECK_CONSTRAINT = getter(Column.class, "checkConstraint");
+
+    /** */
+    private static final String PARAM_NAME_VALUE_SEPARATOR = "=";
+
+    /** */
+    private static final String PARAM_CACHE_TEMPLATE = "cacheTemplate";
+
+    /** Names of the params that need to be present in WITH clause of CREATE TABLE. */
+    private static final String[] MANDATORY_CREATE_TABLE_PARAMS = {
+        PARAM_CACHE_TEMPLATE
+    };
+
+    /** */
     private final IdentityHashMap<Object, Object> h2ObjToGridObj = new IdentityHashMap<>();
 
     /** */
@@ -781,6 +831,210 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * Parse {@code CREATE TABLE} statement.
+     *
+     * @param createTbl {@code CREATE TABLE} statement.
+     * @see <a href="http://h2database.com/html/grammar.html#create_table">H2 {@code CREATE TABLE} spec.</a>
+     */
+    private GridSqlCreateTable parseCreateTable(CreateTable createTbl) {
+        GridSqlCreateTable res = new GridSqlCreateTable();
+
+        Query qry = CREATE_TABLE_QUERY.get(createTbl);
+
+        if (qry != null)
+            throw new IgniteSQLException("CREATE TABLE ... AS ... syntax is not supported",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        List<DefineCommand> constraints = CREATE_TABLE_CONSTRAINTS.get(createTbl);
+
+        if (constraints.size() == 0)
+            throw new IgniteSQLException("No PRIMARY KEY defined for CREATE TABLE",
+                IgniteQueryErrorCode.PARSING);
+
+        if (constraints.size() > 1)
+            throw new IgniteSQLException("Too many constraints - only PRIMARY KEY is supported for CREATE TABLE",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        DefineCommand constraint = constraints.get(0);
+
+        if (!(constraint instanceof AlterTableAddConstraint))
+            throw new IgniteSQLException("Unsupported type of constraint for CREATE TABLE - only PRIMARY KEY " +
+                "is supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        AlterTableAddConstraint alterTbl = (AlterTableAddConstraint)constraint;
+
+        if (alterTbl.getType() != Command.ALTER_TABLE_ADD_CONSTRAINT_PRIMARY_KEY)
+            throw new IgniteSQLException("Unsupported type of constraint for CREATE TABLE - only PRIMARY KEY " +
+                "is supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(createTbl);
+
+        res.schemaName(schema.getName());
+
+        CreateTableData data = CREATE_TABLE_DATA.get(createTbl);
+
+        LinkedHashMap<String, GridSqlColumn> cols = new LinkedHashMap<>(data.columns.size());
+
+        for (Column col : data.columns) {
+            if (col.isAutoIncrement())
+                throw new IgniteSQLException("AUTO_INCREMENT columns are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (!col.isNullable())
+                throw new IgniteSQLException("Non nullable columns are forbidden [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.PARSING);
+
+            if (COLUMN_IS_COMPUTED.get(col))
+                throw new IgniteSQLException("Computed columns are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (col.getDefaultExpression() != null)
+                throw new IgniteSQLException("DEFAULT expressions are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (col.getSequence() != null)
+                throw new IgniteSQLException("SEQUENCE columns are not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (col.getSelectivity() != Constants.SELECTIVITY_DEFAULT)
+                throw new IgniteSQLException("SELECTIVITY column attr is not supported [colName=" + col.getName() + ']',
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            if (COLUMN_CHECK_CONSTRAINT.get(col) != null)
+                throw new IgniteSQLException("Column CHECK constraints are not supported [colName=" + col.getName() +
+                    ']', IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            GridSqlColumn gridCol = new GridSqlColumn(col, null, col.getName());
+
+            gridCol.resultType(GridSqlType.fromColumn(col));
+
+            cols.put(col.getName(), gridCol);
+        }
+
+        if (cols.containsKey(QueryUtils.KEY_FIELD_NAME.toUpperCase()) ||
+            cols.containsKey(QueryUtils.VAL_FIELD_NAME.toUpperCase()))
+            throw new IgniteSQLException("Direct specification of _KEY and _VAL columns is forbidden",
+                IgniteQueryErrorCode.PARSING);
+
+        IndexColumn[] pkIdxCols = CREATE_TABLE_PK.get(createTbl);
+
+        if (F.isEmpty(pkIdxCols))
+            throw new AssertionError("No PRIMARY KEY columns specified");
+
+        LinkedHashSet<String> pkCols = new LinkedHashSet<>();
+
+        for (IndexColumn pkIdxCol : pkIdxCols) {
+            GridSqlColumn gridCol = cols.get(pkIdxCol.columnName);
+
+            assert gridCol != null;
+
+            pkCols.add(gridCol.columnName());
+        }
+
+        int valColsNum = cols.size() - pkCols.size();
+
+        if (valColsNum == 0)
+            throw new IgniteSQLException("No cache value related columns found");
+
+        res.columns(cols);
+
+        res.primaryKeyColumns(pkCols);
+
+        res.tableName(data.tableName);
+
+        res.ifNotExists(CREATE_TABLE_IF_NOT_EXISTS.get(createTbl));
+
+        List<String> extraParams = data.tableEngineParams;
+
+        res.params(extraParams);
+
+        Map<String, String> params = new HashMap<>();
+
+        if (!F.isEmpty(extraParams)) {
+            for (String p : extraParams) {
+                String[] parts = p.split(PARAM_NAME_VALUE_SEPARATOR);
+
+                if (parts.length > 2)
+                    throw new IgniteSQLException("Invalid param syntax: key[=value] expected [paramStr=" + p + ']',
+                        IgniteQueryErrorCode.PARSING);
+
+                String name = parts[0];
+
+                String val = parts.length > 1 ? parts[1] : null;
+
+                if (F.isEmpty(name))
+                    throw new IgniteSQLException("Invalid param syntax: no name given [paramStr=" + p + ']',
+                        IgniteQueryErrorCode.PARSING);
+
+                params.put(name, val);
+            }
+        }
+
+        for (String mandParamName : MANDATORY_CREATE_TABLE_PARAMS) {
+            if (!params.containsKey(mandParamName))
+                throw new IgniteSQLException("Mandatory param is missing [paramName=" + mandParamName + ']');
+        }
+
+        for (Map.Entry<String, String> e : params.entrySet())
+            processExtraParam(e.getKey(), e.getValue(), res);
+
+        return res;
+    }
+
+    /**
+     * Parse {@code DROP TABLE} statement.
+     *
+     * @param dropTbl {@code DROP TABLE} statement.
+     * @see <a href="http://h2database.com/html/grammar.html#drop_table">H2 {@code DROP TABLE} spec.</a>
+     */
+    private GridSqlDropTable parseDropTable(DropTable dropTbl) {
+        GridSqlDropTable res = new GridSqlDropTable();
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(dropTbl);
+
+        res.schemaName(schema.getName());
+
+        res.ifExists(DROP_TABLE_IF_EXISTS.get(dropTbl));
+
+        res.tableName(DROP_TABLE_NAME.get(dropTbl));
+
+        return res;
+    }
+
+    /**
+     * @param name Param name.
+     * @param val Param value.
+     * @param res Table params to update.
+     */
+    private static void processExtraParam(String name, String val, GridSqlCreateTable res) {
+        assert !F.isEmpty(name);
+
+        switch (name) {
+            case PARAM_CACHE_TEMPLATE:
+                ensureParamValueNotEmpty(PARAM_CACHE_TEMPLATE, val);
+
+                res.templateCacheName(val);
+
+                break;
+
+            default:
+                throw new IgniteSQLException("Unknown CREATE TABLE param [paramName=" + name + ']',
+                    IgniteQueryErrorCode.PARSING);
+        }
+    }
+
+    /**
+     * Check that param with mandatory value has it specified.
+     * @param name Param name.
+     * @param val Param value to check.
+     */
+    private static void ensureParamValueNotEmpty(String name, String val) {
+        if (F.isEmpty(val))
+            throw new IgniteSQLException("No value has been given for a CREATE TABLE param [paramName=" + name + ']',
+                IgniteQueryErrorCode.PARSING);
+    }
+
+    /**
      * @param sortOrder Sort order.
      * @param qry Query.
      */
@@ -849,6 +1103,12 @@ public class GridSqlQueryParser {
         if (stmt instanceof DropIndex)
             return parseDropIndex((DropIndex)stmt);
 
+        if (stmt instanceof CreateTable)
+            return parseCreateTable((CreateTable)stmt);
+
+        if (stmt instanceof DropTable)
+            return parseDropTable((DropTable) stmt);
+
         throw new CacheException("Unsupported SQL statement: " + stmt);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/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
new file mode 100644
index 0000000..b072fdb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -0,0 +1,320 @@
+/*
+ * 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.cache.index;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Tests for CREATE/DROP TABLE.
+ */
+public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
+    /** Client node index. */
+    private final static int CLIENT = 2;
+
+    /** */
+    private final static String INDEXED_CACHE_NAME = CACHE_NAME + "_idx";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        for (IgniteConfiguration cfg : configurations())
+            Ignition.start(cfg);
+
+        client().getOrCreateCache(cacheConfigurationForIndexing());
+
+        client().addCacheConfiguration(cacheConfiguration());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        client().destroyCache("Person");
+
+        super.afterTest();
+    }
+
+    /**
+     * Test that {@code CREATE TABLE} actually creates new cache, H2 table and type descriptor on all nodes.
+     * @throws Exception if failed.
+     */
+    public void testCreateTable() 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\""));
+
+        for (int i = 0; i < 4; i++) {
+            IgniteEx node = grid(i);
+
+            assertNotNull(node.cache("Person"));
+
+            QueryTypeDescriptorImpl desc = typeExisting(node, "Person", "Person");
+
+            assertEquals(Object.class, desc.keyClass());
+
+            assertEquals("PersonKey", desc.keyTypeName());
+
+            assertEquals(Object.class, desc.valueClass());
+
+            assertEquals("Person", desc.valueTypeName());
+
+            assertEquals(
+                F.asList("id", "city", "name", "surname", "age"),
+                new ArrayList<>(desc.fields().keySet())
+            );
+
+            assertProperty(desc, "id", Integer.class, true);
+
+            assertProperty(desc, "city", String.class, true);
+
+            assertProperty(desc, "name", String.class, false);
+
+            assertProperty(desc, "surname", String.class, false);
+
+            assertProperty(desc, "age", Integer.class, false);
+
+            GridH2Table tbl = ((IgniteH2Indexing)node.context().query().getIndexing()).dataTable("Person", "Person");
+
+            assertNotNull(tbl);
+        }
+    }
+
+    /**
+     * Test that attempting to {@code CREATE TABLE} that already exists does not yield an error if the statement
+     *     contains {@code IF NOT EXISTS} clause.
+     * @throws Exception if failed.
+     */
+    public void testCreateTableIfNotExists() 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\""));
+
+        cache().query(new SqlFieldsQuery("CREATE TABLE IF NOT EXISTS \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+    }
+
+    /**
+     * Test that attempting to {@code CREATE TABLE} that already exists yields an error.
+     * @throws Exception if failed.
+     */
+    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 " +
+            "\"cacheTemplate=cache\""));
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() 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\""));
+
+                return null;
+            }
+        }, IgniteSQLException.class, "Table already exists: Person");
+    }
+
+    /**
+     * Test that {@code DROP TABLE} actually removes specified cache and type descriptor on all nodes.
+     * @throws Exception if failed.
+     */
+    public void testDropTable() throws Exception {
+        cache().query(new SqlFieldsQuery("CREATE TABLE IF NOT EXISTS \"Person\" (\"id\" int, \"city\" varchar," +
+            " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
+            "\"cacheTemplate=cache\""));
+
+        cache().query(new SqlFieldsQuery("DROP TABLE \"Person\".\"Person\""));
+
+        for (int i = 0; i < 4; i++) {
+            IgniteEx node = grid(i);
+
+            assertNull(node.cache("Person"));
+
+            QueryTypeDescriptorImpl desc = type(node, "Person", "Person");
+
+            assertNull(desc);
+        }
+    }
+
+    /**
+     * Test that attempting to {@code DROP TABLE} that does not exist does not yield an error if the statement contains
+     *     {@code IF EXISTS} clause.
+     * @throws Exception if failed.
+     */
+    public void testDropMissingTableIfExists() throws Exception {
+        cache().query(new SqlFieldsQuery("DROP TABLE IF EXISTS \"cache_idx\".\"City\""));
+    }
+
+    /**
+     * Test that attempting to {@code DROP TABLE} that does not exist yields an error.
+     * @throws Exception if failed.
+     */
+    public void testDropMissingTable() throws Exception {
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                cache().query(new SqlFieldsQuery("DROP TABLE \"cache_idx\".\"City\""));
+
+                return null;
+            }
+        }, IgniteSQLException.class, "Table doesn't exist: City");
+    }
+
+    /**
+     * Check that a property in given descriptor is present and has parameters as expected.
+     * @param desc Descriptor.
+     * @param name Property name.
+     * @param type Expected property type.
+     * @param isKey {@code true} if the property is expected to belong to key, {@code false} is it's expected to belong
+     *     to value.
+     */
+    private void assertProperty(QueryTypeDescriptorImpl desc, String name, Class<?> type, boolean isKey) {
+        GridQueryProperty p = desc.property(name);
+
+        assertNotNull(name, p);
+
+        assertEquals(type, p.type());
+
+        assertEquals(isKey, p.key());
+    }
+
+    /**
+     * Get configurations to be used in test.
+     *
+     * @return Configurations.
+     * @throws Exception If failed.
+     */
+    private List<IgniteConfiguration> configurations() throws Exception {
+        return Arrays.asList(
+            serverConfiguration(0),
+            serverConfiguration(1),
+            clientConfiguration(2),
+            serverConfiguration(3)
+        );
+    }
+
+    /**
+     * Create server configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration serverConfiguration(int idx) throws Exception {
+        return commonConfiguration(idx);
+    }
+
+    /**
+     * Create client configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration clientConfiguration(int idx) throws Exception {
+        return commonConfiguration(idx).setClientMode(true);
+    }
+
+    /**
+     * Create common node configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration commonConfiguration(int idx) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(getTestIgniteInstanceName(idx));
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return optimize(cfg);
+    }
+
+    /**
+     * @return Client node.
+     */
+    private IgniteEx client() {
+        return grid(CLIENT);
+    }
+
+    /**
+     * @return Cache to issue queries upon.
+     */
+    private IgniteCache<?, ?> cache() {
+        return client().cache(INDEXED_CACHE_NAME);
+    }
+
+    /**
+     * @return Default cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration<?, ?> ccfg = new CacheConfiguration<>(CACHE_NAME);
+
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg.setSqlEscapeAll(true);
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+
+        return ccfg;
+    }
+
+    /**
+     * @return Cache configuration with query entities - unfortunately, we need this to enable indexing at all.
+     */
+    private CacheConfiguration cacheConfigurationForIndexing() {
+        CacheConfiguration<?, ?> ccfg = cacheConfiguration();
+
+        ccfg.setName(INDEXED_CACHE_NAME);
+
+        ccfg.setQueryEntities(Collections.singletonList(
+            new QueryEntity()
+                .setKeyType(Integer.class.getName())
+                .setValueType(Integer.class.getName())
+        ));
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index 9838084..5939b59 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -22,8 +22,11 @@ import java.sql.Connection;
 import java.sql.Date;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
@@ -40,17 +43,23 @@ import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.h2.command.Parser;
 import org.h2.command.Prepared;
+import org.h2.command.ddl.CreateTable;
 import org.h2.engine.Session;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.message.DbException;
+import org.h2.table.Column;
+import org.h2.value.Value;
 import org.jetbrains.annotations.NotNull;
 
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
@@ -532,7 +541,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
      *
      */
     public void testParseDropIndex() throws Exception {
-        // Schema that is not set defaults to default schema of connection which is empty string
+        // Schema that is not set defaults to default schema of connection which is sch1
         assertDropIndexEquals(buildDropIndex("idx", "sch1", false), "drop index idx");
         assertDropIndexEquals(buildDropIndex("idx", "sch1", true), "drop index if exists idx");
         assertDropIndexEquals(buildDropIndex("idx", "sch1", true), "drop index if exists sch1.idx");
@@ -546,6 +555,75 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testParseDropTable() throws Exception {
+        // Schema that is not set defaults to default schema of connection which is sch1
+        assertDropTableEquals(buildDropTable("sch1", "tbl", false), "drop table tbl");
+        assertDropTableEquals(buildDropTable("sch1", "tbl", true), "drop table if exists tbl");
+        assertDropTableEquals(buildDropTable("sch1", "tbl", true), "drop table if exists sch1.tbl");
+        assertDropTableEquals(buildDropTable("sch1", "tbl", false), "drop table sch1.tbl");
+
+        // Message is null as long as it may differ from system to system, so we just check for exceptions
+        assertParseThrows("drop table schema2.", DbException.class, null);
+        assertParseThrows("drop table", DbException.class, null);
+        assertParseThrows("drop table if exists", DbException.class, null);
+        assertParseThrows("drop table if exists schema2.", DbException.class, null);
+    }
+
+    /** */
+    public void testParseCreateTable() throws Exception {
+        assertCreateTableEquals(
+            buildCreateTable("sch1", "Person", "cache", F.asList("id", "city"),
+                true, c("id", Value.INT), c("city", Value.STRING), c("name", Value.STRING),
+                c("surname", Value.STRING), c("age", Value.INT)),
+            "CREATE TABLE IF NOT EXISTS sch1.\"Person\" (\"id\" integer, \"city\" varchar," +
+                " \"name\" varchar, \"surname\" varchar, \"age\" integer, PRIMARY KEY (\"id\", \"city\")) WITH " +
+                "\"cacheTemplate=cache\"");
+
+        assertCreateTableEquals(
+            buildCreateTable("sch1", "Person", "cache", F.asList("id"),
+                false, c("id", Value.INT), c("city", Value.STRING), c("name", Value.STRING),
+                c("surname", Value.STRING), c("age", Value.INT)),
+            "CREATE TABLE sch1.\"Person\" (\"id\" integer PRIMARY KEY, \"city\" varchar," +
+                " \"name\" varchar, \"surname\" varchar, \"age\" integer) WITH " +
+                "\"cacheTemplate=cache\"");
+
+        assertParseThrows("create table Person (id int)",
+            IgniteSQLException.class, "No PRIMARY KEY defined for CREATE TABLE");
+
+        assertParseThrows("create table Person (id int) AS SELECT 2 * 2",
+            IgniteSQLException.class, "CREATE TABLE ... AS ... syntax is not supported");
+
+        assertParseThrows("create table Person (id int primary key)",
+            IgniteSQLException.class, "No cache value related columns found");
+
+        assertParseThrows("create table Person (id int primary key, age int null)",
+            IgniteSQLException.class, "Mandatory param is missing [paramName=cacheTemplate]");
+
+        assertParseThrows("create table Person (id int primary key, age int not null) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Non nullable columns are forbidden");
+
+        assertParseThrows("create table Person (id int primary key, age int unique) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Too many constraints - only PRIMARY KEY is supported for CREATE TABLE");
+
+        assertParseThrows("create table Person (id int auto_increment primary key, age int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "AUTO_INCREMENT columns are not supported");
+
+        assertParseThrows("create table Person (id int primary key check id > 0, age int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Column CHECK constraints are not supported [colName=ID]");
+
+        assertParseThrows("create table Person (id int as age * 2 primary key, age int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Computed columns are not supported [colName=ID]");
+
+        assertParseThrows("create table Person (id int primary key, age int default 5) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "DEFAULT expressions are not supported [colName=AGE]");
+
+        assertParseThrows("create table Int (_key int primary key, _val int) WITH \"cacheTemplate=cache\"",
+            IgniteSQLException.class, "Direct specification of _KEY and _VAL columns is forbidden");
+    }
+
+    /**
      * @param sql Statement.
      * @param exCls Exception class.
      * @param msg Expected message.
@@ -610,6 +688,112 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Parse SQL and compare it to expected instance of DROP TABLE.
+     */
+    private void assertCreateTableEquals(GridSqlCreateTable exp, String sql) throws Exception {
+        Prepared prepared = parse(sql);
+
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+
+        assertTrue(stmt instanceof GridSqlCreateTable);
+
+        assertCreateTableEquals(exp, (GridSqlCreateTable) stmt);
+    }
+
+    /**
+     * Test two instances of {@link GridSqlDropTable} for equality.
+     */
+    private static void assertCreateTableEquals(GridSqlCreateTable exp, GridSqlCreateTable actual) {
+        assertEqualsIgnoreCase(exp.schemaName(), actual.schemaName());
+        assertEqualsIgnoreCase(exp.tableName(), actual.tableName());
+        assertEquals(exp.templateCacheName(), actual.templateCacheName());
+        assertEquals(exp.primaryKeyColumns(), actual.primaryKeyColumns());
+        assertEquals(new ArrayList<>(exp.columns().keySet()), new ArrayList<>(actual.columns().keySet()));
+
+        for (Map.Entry<String, GridSqlColumn> col : exp.columns().entrySet()) {
+            GridSqlColumn val = actual.columns().get(col.getKey());
+
+            assertNotNull(val);
+
+            assertEquals(col.getValue().columnName(), val.columnName());
+            assertEquals(col.getValue().column().getType(), val.column().getType());
+        }
+
+        assertEquals(exp.ifNotExists(), actual.ifNotExists());
+    }
+
+    /**
+     *
+     */
+    private static GridSqlCreateTable buildCreateTable(String schema, String tbl, String tplCacheName,
+        Collection<String> pkColNames, boolean ifNotExists, GridSqlColumn... cols) {
+        GridSqlCreateTable res = new GridSqlCreateTable();
+
+        res.schemaName(schema);
+
+        res.tableName(tbl);
+
+        res.templateCacheName(tplCacheName);
+
+        res.primaryKeyColumns(new LinkedHashSet<>(pkColNames));
+
+        LinkedHashMap<String, GridSqlColumn> m = new LinkedHashMap<>();
+
+        for (GridSqlColumn col : cols)
+            m.put(col.columnName(), col);
+
+        res.columns(m);
+
+        res.ifNotExists(ifNotExists);
+
+        return res;
+    }
+
+    /**
+     * @param name Column name.
+     * @param type Column data type.
+     * @return {@link GridSqlColumn} with given name and type.
+     */
+    private static GridSqlColumn c(String name, int type) {
+        return new GridSqlColumn(new Column(name, type), null, name);
+    }
+
+    /**
+     * Parse SQL and compare it to expected instance of DROP TABLE.
+     */
+    private void assertDropTableEquals(GridSqlDropTable exp, String sql) throws Exception {
+        Prepared prepared = parse(sql);
+
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
+
+        assertTrue(stmt instanceof GridSqlDropTable);
+
+        assertDropTableEquals(exp, (GridSqlDropTable) stmt);
+    }
+
+    /**
+     * Test two instances of {@link GridSqlDropTable} for equality.
+     */
+    private static void assertDropTableEquals(GridSqlDropTable exp, GridSqlDropTable actual) {
+        assertEqualsIgnoreCase(exp.schemaName(), actual.schemaName());
+        assertEqualsIgnoreCase(exp.tableName(), actual.tableName());
+        assertEquals(exp.ifExists(), actual.ifExists());
+    }
+
+    /**
+     *
+     */
+    private static GridSqlDropTable buildDropTable(String schema, String tbl, boolean ifExists) {
+        GridSqlDropTable res = new GridSqlDropTable();
+
+        res.schemaName(schema);
+        res.tableName(tbl);
+        res.ifExists(ifExists);
+
+        return res;
+    }
+
+    /**
      * Test two instances of {@link GridSqlCreateIndex} for equality.
      */
     private static void assertCreateIndexEquals(GridSqlCreateIndex exp, GridSqlCreateIndex actual) {
@@ -723,6 +907,87 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param createTbl {@code CREATE TABLE} command.
+     * @return Corresponding SQL.
+     */
+    private static String createTableToSql(GridSqlCreateTable createTbl) {
+        GridStringBuilder b = new SB("CREATE TABLE ")
+            .a(createTbl.ifNotExists() ? "IF NOT EXISTS " : "")
+            .a("\n")
+            .a(Parser.quoteIdentifier(createTbl.schemaName()))
+            .a('.')
+            .a(Parser.quoteIdentifier(createTbl.tableName()))
+            .a("\n(");
+
+        boolean singleColPk = false;
+
+        boolean first = true;
+
+        for (GridSqlColumn col : createTbl.columns().values()) {
+            if (!first)
+                b.a(",\n");
+            else
+                first = false;
+
+            if (col.column().isPrimaryKey()) {
+                // Only one column may be marked PRIMARY KEY - multi-col PK is defined separately
+                assert !singleColPk;
+
+                singleColPk = true;
+            }
+
+            b.a('\t')
+                .a(col.getSQL())
+                .a(' ')
+                .a(col.resultType().sql())
+                .a(col.column().isPrimaryKey() ? " PRIMARY KEY" : "");
+        }
+
+        first = true;
+
+        if (!singleColPk && !F.isEmpty(createTbl.primaryKeyColumns())) {
+            b.a(",\n")
+                .a('\t')
+                .a("PRIMARY KEY (\n");
+
+            for (String col : createTbl.primaryKeyColumns()) {
+                GridSqlColumn pkCol = createTbl.columns().get(col);
+
+                assert pkCol != null;
+
+                if (!first)
+                    b.a(",\n");
+                else
+                    first = false;
+
+                b.a("\t\t")
+                    .a(pkCol.getSQL());
+            }
+
+            b.a("\n\t)");
+        }
+
+        b.a("\n)");
+
+        if (!F.isEmpty(createTbl.params())) {
+            b.a("\nWITH ");
+
+            first = true;
+
+            for (String p : createTbl.params()) {
+                if (!first)
+                    b.a(',');
+                else
+                    first = false;
+
+                b.a(Parser.quoteIdentifier(p));
+            }
+        }
+
+        return b.toString();
+    }
+
+    /**
      * @param qry Query.
      */
     private void checkQuery(String qry) throws Exception {
@@ -737,6 +1002,23 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         assertSqlEquals(U.firstNotNull(prepared.getPlanSQL(), prepared.getSQL()), res);
     }
 
+    /**
+     * @param qry Query.
+     */
+    private void checkCreateTable(String qry) throws Exception {
+        Prepared prepared = parse(qry);
+
+        assertTrue(prepared instanceof CreateTable);
+
+        GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(prepared);
+
+        String res = createTableToSql((GridSqlCreateTable)gridStmt);
+
+        System.out.println(normalizeSql(res));
+
+        assertSqlEquals(U.firstNotNull(prepared.getPlanSQL(), prepared.getSQL()), res);
+    }
+
     @QuerySqlFunction
     public static int cool1() {
         return 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8f5af33/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 6370cd0..14fb6ce 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -96,6 +96,7 @@ import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoord
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerBasicSelfTest;
+import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest;
 import org.apache.ignite.internal.processors.cache.index.QueryEntityValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
@@ -227,6 +228,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(H2DynamicIndexAtomicReplicatedSelfTest.class);
         suite.addTestSuite(H2DynamicIndexAtomicPartitionedSelfTest.class);
         suite.addTestSuite(H2DynamicIndexAtomicPartitionedNearSelfTest.class);
+        suite.addTestSuite(H2DynamicTableSelfTest.class);
 
         // Fields queries.
         suite.addTestSuite(SqlFieldsQuerySelfTest.class);