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 09:22:53 UTC

[03/51] ignite git commit: IGNITE-5304: SQL: decoupled DDL from schema. This closes #2017.

IGNITE-5304: SQL: decoupled DDL from schema. This closes #2017.


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

Branch: refs/heads/ignite-5075-pds
Commit: 40851c7ff44c0ec97d9cca53923c8db0a38f78cd
Parents: 427ca69
Author: devozerov <vo...@gridgain.com>
Authored: Sat May 27 15:07:01 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Sat May 27 15:07:01 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryIndexing.java     |  9 ++-
 .../processors/query/GridQueryProcessor.java    | 32 +++++-----
 .../processors/query/QueryIndexKey.java         | 28 +++------
 .../processors/query/h2/IgniteH2Indexing.java   | 32 +++++-----
 .../query/h2/ddl/DdlStatementsProcessor.java    | 66 ++++++++++++--------
 .../DynamicIndexAbstractConcurrentSelfTest.java |  8 +--
 6 files changed, 83 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 77c581c..9d66c0a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -142,27 +142,26 @@ public interface GridQueryIndexing {
     /**
      * Create new index locally.
      *
-     * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param tblName Table name.
      * @param idxDesc Index descriptor.
      * @param ifNotExists Ignore operation if index exists (instead of throwing an error).
      * @param cacheVisitor Cache visitor
      * @throws IgniteCheckedException if failed.
      */
-    public void dynamicIndexCreate(String cacheName, String tblName, QueryIndexDescriptorImpl idxDesc,
+    public void dynamicIndexCreate(String schemaName, String tblName, QueryIndexDescriptorImpl idxDesc,
         boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException;
 
     /**
      * Remove index from the cache.
      *
-     * @param cacheName cache name.
+     * @param schemaName Schema name.
      * @param idxName Index name.
      * @param ifExists Ignore operation if index does not exist (instead of throwing an error).
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-    public void dynamicIndexDrop(String cacheName, String idxName, boolean ifExists)
-        throws IgniteCheckedException;
+    public void dynamicIndexDrop(String schemaName, String idxName, boolean ifExists) throws IgniteCheckedException;
 
     /**
      * Registers cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/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 3233fd2..76cde17 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
@@ -933,7 +933,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (err == null) {
                 String idxName = op0.index().getName();
 
-                QueryIndexKey idxKey = new QueryIndexKey(cacheName, idxName);
+                QueryIndexKey idxKey = new QueryIndexKey(op.schemaName(), idxName);
 
                 if (idxs.get(idxKey) != null) {
                     if (op0.ifNotExists())
@@ -948,7 +948,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String idxName = op0.indexName();
 
-            QueryIndexDescriptorImpl oldIdx = idxs.get(new QueryIndexKey(cacheName, idxName));
+            QueryIndexDescriptorImpl oldIdx = idxs.get(new QueryIndexKey(op.schemaName(), idxName));
 
             if (oldIdx == null) {
                 if (op0.ifExists())
@@ -1159,7 +1159,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (log.isDebugEnabled())
                 log.debug("Local operation finished successfully [opId=" + op.id() + ']');
 
-            String cacheName = op.cacheName();
+            String schemaName = op.schemaName();
 
             try {
                 if (op instanceof SchemaIndexCreateOperation) {
@@ -1169,7 +1169,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     QueryIndexDescriptorImpl idxDesc = type.index(op0.indexName());
 
-                    QueryIndexKey idxKey = new QueryIndexKey(cacheName, op0.indexName());
+                    QueryIndexKey idxKey = new QueryIndexKey(schemaName, op0.indexName());
 
                     idxs.put(idxKey, idxDesc);
                 }
@@ -1180,7 +1180,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     QueryUtils.processDynamicIndexChange(op0.indexName(), null, type);
 
-                    QueryIndexKey idxKey = new QueryIndexKey(cacheName, op0.indexName());
+                    QueryIndexKey idxKey = new QueryIndexKey(schemaName, op0.indexName());
 
                     idxs.remove(idxKey);
                 }
@@ -1251,12 +1251,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 SchemaIndexCacheVisitor visitor =
                     new SchemaIndexCacheVisitorImpl(this, cache.context(), cacheName, op0.tableName(), cancelTok);
 
-                idx.dynamicIndexCreate(cacheName, op0.tableName(), idxDesc, op0.ifNotExists(), visitor);
+                idx.dynamicIndexCreate(op0.schemaName(), op0.tableName(), idxDesc, op0.ifNotExists(), visitor);
             }
             else if (op instanceof SchemaIndexDropOperation) {
                 SchemaIndexDropOperation op0 = (SchemaIndexDropOperation) op;
 
-                idx.dynamicIndexDrop(cacheName, op0.indexName(), op0.ifExists());
+                idx.dynamicIndexDrop(op0.schemaName(), op0.indexName(), op0.ifExists());
             }
             else
                 throw new SchemaOperationException("Unsupported operation: " + op);
@@ -1293,10 +1293,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         CacheConfiguration<?, ?> newCfg = new CacheConfiguration<>(templateCfg);
 
         newCfg.setName(entity.getTableName());
-
         newCfg.setQueryEntities(Collections.singleton(entity));
 
-        // We want to preserve user specified names as they are
+        // Preserve user specified names as they are.
         newCfg.setSqlEscapeAll(true);
 
         boolean res = ctx.grid().getOrCreateCache0(newCfg).get2();
@@ -1308,14 +1307,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * Drop table by destroying its cache if it's an 1:1 per cache table.
      *
-     * @param schemaName Schema name.
+     * @param cacheName Cache 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);
+    public void dynamicTableDrop(String cacheName, String tblName, boolean ifExists) throws SchemaOperationException {
+        boolean res = ctx.grid().destroyCache0(cacheName);
 
         if (!res && !ifExists)
             throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName);
@@ -1352,13 +1351,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         types.put(altTypeId, desc);
 
                     for (QueryIndexDescriptorImpl idx : desc.indexes0()) {
-                        QueryIndexKey idxKey = new QueryIndexKey(cacheName, idx.name());
+                        QueryIndexKey idxKey = new QueryIndexKey(schemaName, idx.name());
 
                         QueryIndexDescriptorImpl oldIdx = idxs.putIfAbsent(idxKey, idx);
 
                         if (oldIdx != null) {
                             throw new IgniteException("Duplicate index name [cache=" + cacheName +
-                                ", idxName=" + idx.name() + ", existingTable=" + oldIdx.typeDescriptor().tableName() +
+                                ", schemaName=" + schemaName + ", idxName=" + idx.name() +
+                                ", existingTable=" + oldIdx.typeDescriptor().tableName() +
                                 ", table=" + desc.tableName() + ']');
                         }
                     }
@@ -1409,9 +1409,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             while (idxIt.hasNext()) {
                 Map.Entry<QueryIndexKey, QueryIndexDescriptorImpl> idxEntry = idxIt.next();
 
-                QueryIndexKey idxKey = idxEntry.getKey();
-
-                if (F.eq(cacheName, idxKey.cacheName()))
+                if (F.eq(cacheName, idxEntry.getValue().typeDescriptor().cacheName()))
                     idxIt.remove();
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
index 7fdb805..fea6594 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
@@ -29,8 +29,8 @@ public class QueryIndexKey implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Cache name. */
-    private final String cacheName;
+    /** Schema name. */
+    private final String schemaName;
 
     /** Name. */
     private final String name;
@@ -38,31 +38,17 @@ public class QueryIndexKey implements Serializable {
     /**
      * Constructor.
      *
-     * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param name Name.
      */
-    public QueryIndexKey(String cacheName, String name) {
-        this.cacheName = cacheName;
+    public QueryIndexKey(String schemaName, String name) {
+        this.schemaName = schemaName;
         this.name = name;
     }
 
-    /**
-     * @return Cache name.
-     */
-    public String cacheName() {
-        return cacheName;
-    }
-
-    /**
-     * @return Name.
-     */
-    public String name() {
-        return name;
-    }
-
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + (name != null ? name.hashCode() : 0);
+        return 31 * (schemaName != null ? schemaName.hashCode() : 0) + (name != null ? name.hashCode() : 0);
     }
 
     /** {@inheritDoc} */
@@ -75,7 +61,7 @@ public class QueryIndexKey implements Serializable {
 
         QueryIndexKey other = (QueryIndexKey)o;
 
-        return F.eq(name, other.name) && F.eq(cacheName, other.cacheName);
+        return F.eq(name, other.name) && F.eq(schemaName, other.schemaName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 71d3ad5..0291b0a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -670,12 +670,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Add initial user index.
      *
-     * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param desc Table descriptor.
      * @param h2Idx User index.
      * @throws IgniteCheckedException If failed.
      */
-    private void addInitialUserIndex(String cacheName, H2TableDescriptor desc, GridH2IndexBase h2Idx)
+    private void addInitialUserIndex(String schemaName, H2TableDescriptor desc, GridH2IndexBase h2Idx)
         throws IgniteCheckedException {
         GridH2Table h2Tbl = desc.table();
 
@@ -684,7 +684,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             String sql = H2Utils.indexCreateSql(desc.fullTableName(), h2Idx, false);
 
-            executeSql(cacheName, sql);
+            executeSql(schemaName, sql);
         }
         catch (Exception e) {
             // Rollback and re-throw.
@@ -695,12 +695,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void dynamicIndexCreate(final String cacheName, final String tblName,
+    @Override public void dynamicIndexCreate(final String schemaName, final String tblName,
         final QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor)
         throws IgniteCheckedException {
         // Locate table.
-        String schemaName = schema(cacheName);
-
         H2Schema schema = schemas.get(schemaName);
 
         H2TableDescriptor desc = (schema != null ? schema.tableByName(tblName) : null);
@@ -740,7 +738,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             // prepared statements are re-built.
             String sql = H2Utils.indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists);
 
-            executeSql(cacheName, sql);
+            executeSql(schemaName, sql);
         }
         catch (Exception e) {
             // Rollback and re-throw.
@@ -752,25 +750,23 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-    @Override public void dynamicIndexDrop(final String cacheName, String idxName, boolean ifExists)
+    @Override public void dynamicIndexDrop(final String schemaName, String idxName, boolean ifExists)
         throws IgniteCheckedException{
-        String schemaName = schema(cacheName);
-
         String sql = H2Utils.indexDropSql(schemaName, idxName, ifExists);
 
-        executeSql(cacheName, sql);
+        executeSql(schemaName, sql);
     }
 
     /**
      * Execute DDL command.
      *
-     * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param sql SQL.
      * @throws IgniteCheckedException If failed.
      */
-    private void executeSql(String cacheName, String sql) throws IgniteCheckedException {
+    private void executeSql(String schemaName, String sql) throws IgniteCheckedException {
         try {
-            Connection conn = connectionForCache(cacheName);
+            Connection conn = connectionForSchema(schemaName);
 
             try (PreparedStatement stmt = prepareStatement(conn, sql, false)) {
                 stmt.execute();
@@ -1609,7 +1605,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             Connection conn = connectionForThread(schemaName);
 
-            createTable(cacheName, schema, tbl, conn);
+            createTable(schemaName, schema, tbl, conn);
 
             schema.add(tbl);
         }
@@ -1653,14 +1649,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Create db table by using given table descriptor.
      *
-     * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param schema Schema.
      * @param tbl Table descriptor.
      * @param conn Connection.
      * @throws SQLException If failed to create db table.
      * @throws IgniteCheckedException If failed.
      */
-    private void createTable(String cacheName, H2Schema schema, H2TableDescriptor tbl, Connection conn)
+    private void createTable(String schemaName, H2Schema schema, H2TableDescriptor tbl, Connection conn)
         throws SQLException, IgniteCheckedException {
         assert schema != null;
         assert tbl != null;
@@ -1693,7 +1689,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridH2Table h2Tbl = H2TableEngine.createTable(conn, sql.toString(), rowDesc, rowFactory, tbl);
 
         for (GridH2IndexBase usrIdx : tbl.createUserIndexes())
-            addInitialUserIndex(cacheName, tbl, usrIdx);
+            addInitialUserIndex(schemaName, tbl, usrIdx);
 
         if (dataTables.putIfAbsent(h2Tbl.identifier(), h2Tbl) != null)
             throw new IllegalStateException("Table already exists: " + h2Tbl.identifierString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/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 7157d1f..e324ed9 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
@@ -88,34 +88,34 @@ public class DdlStatementsProcessor {
         throws IgniteCheckedException {
         assert stmt instanceof JdbcPreparedStatement;
 
-        IgniteInternalFuture fut;
+        IgniteInternalFuture fut = null;
 
         try {
-            GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt));
+            GridSqlStatement stmt0 = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt));
 
-            if (gridStmt instanceof GridSqlCreateIndex) {
-                GridSqlCreateIndex createIdx = (GridSqlCreateIndex)gridStmt;
+            if (stmt0 instanceof GridSqlCreateIndex) {
+                GridSqlCreateIndex cmd = (GridSqlCreateIndex)stmt0;
 
                 QueryIndex newIdx = new QueryIndex();
 
-                newIdx.setName(createIdx.index().getName());
+                newIdx.setName(cmd.index().getName());
 
-                newIdx.setIndexType(createIdx.index().getIndexType());
+                newIdx.setIndexType(cmd.index().getIndexType());
 
                 LinkedHashMap<String, Boolean> flds = new LinkedHashMap<>();
 
-                GridH2Table tbl = idx.dataTable(createIdx.schemaName(), createIdx.tableName());
+                GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
 
                 if (tbl == null)
                     throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
-                        createIdx.tableName());
+                        cmd.tableName());
 
                 assert tbl.rowDescriptor() != null;
 
                 // Let's replace H2's table and property names by those operated by GridQueryProcessor.
                 GridQueryTypeDescriptor typeDesc = tbl.rowDescriptor().type();
 
-                for (Map.Entry<String, Boolean> e : createIdx.index().getFields().entrySet()) {
+                for (Map.Entry<String, Boolean> e : cmd.index().getFields().entrySet()) {
                     GridQueryProperty prop = typeDesc.property(e.getKey());
 
                     if (prop == null)
@@ -126,39 +126,51 @@ public class DdlStatementsProcessor {
 
                 newIdx.setFields(flds);
 
-                fut = ctx.query().dynamicIndexCreate(tbl.cacheName(), createIdx.schemaName(), typeDesc.tableName(),
-                    newIdx, createIdx.ifNotExists());
+                fut = ctx.query().dynamicIndexCreate(tbl.cacheName(), cmd.schemaName(), typeDesc.tableName(),
+                    newIdx, cmd.ifNotExists());
             }
-            else if (gridStmt instanceof GridSqlDropIndex) {
-                GridSqlDropIndex dropIdx = (GridSqlDropIndex)gridStmt;
+            else if (stmt0 instanceof GridSqlDropIndex) {
+                GridSqlDropIndex cmd = (GridSqlDropIndex)stmt0;
 
-                GridH2Table tbl = idx.dataTableForIndex(dropIdx.schemaName(), dropIdx.indexName());
+                GridH2Table tbl = idx.dataTableForIndex(cmd.schemaName(), cmd.indexName());
 
                 if (tbl != null)
-                    fut = ctx.query().dynamicIndexDrop(tbl.cacheName(), dropIdx.schemaName(), dropIdx.indexName(),
-                        dropIdx.ifExists());
+                    fut = ctx.query().dynamicIndexDrop(tbl.cacheName(), cmd.schemaName(), cmd.indexName(),
+                        cmd.ifExists());
                 else {
-                    if (dropIdx.ifExists())
+                    if (cmd.ifExists())
                         fut = new GridFinishedFuture();
                     else
                         throw new SchemaOperationException(SchemaOperationException.CODE_INDEX_NOT_FOUND,
-                            dropIdx.indexName());
+                            cmd.indexName());
                 }
             }
-            else if (gridStmt instanceof GridSqlCreateTable) {
-                GridSqlCreateTable createTbl = (GridSqlCreateTable)gridStmt;
+            else if (stmt0 instanceof GridSqlCreateTable) {
+                GridSqlCreateTable cmd = (GridSqlCreateTable)stmt0;
 
-                ctx.query().dynamicTableCreate(createTbl.schemaName(), toQueryEntity(createTbl),
-                    createTbl.templateCacheName(), createTbl.ifNotExists());
+                GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
 
-                fut = null;
+                if (tbl != null) {
+                    if (!cmd.ifNotExists())
+                        throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_EXISTS,
+                            cmd.tableName());
+                }
+                else
+                    ctx.query().dynamicTableCreate(cmd.schemaName(), toQueryEntity(cmd), cmd.templateCacheName(),
+                        cmd.ifNotExists());
             }
-            else if (gridStmt instanceof GridSqlDropTable) {
-                GridSqlDropTable dropTbl = (GridSqlDropTable)gridStmt;
+            else if (stmt0 instanceof GridSqlDropTable) {
+                GridSqlDropTable cmd = (GridSqlDropTable)stmt0;
 
-                ctx.query().dynamicTableDrop(dropTbl.schemaName(), dropTbl.tableName(), dropTbl.ifExists());
+                GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
 
-                fut = null;
+                if (tbl == null) {
+                    if (!cmd.ifExists())
+                        throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                            cmd.tableName());
+                }
+                else
+                    ctx.query().dynamicTableDrop(tbl.cacheName(), cmd.tableName(), cmd.ifExists());
             }
             else
                 throw new IgniteSQLException("Unsupported DDL operation: " + sql,

http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
index 5d93c12..9936a81 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
@@ -1043,20 +1043,20 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     private static class BlockingIndexing extends IgniteH2Indexing {
         /** {@inheritDoc} */
-        @Override public void dynamicIndexCreate(@NotNull String cacheName, String tblName,
+        @Override public void dynamicIndexCreate(@NotNull String schemaName, String tblName,
             QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor)
             throws IgniteCheckedException {
             awaitIndexing(ctx.localNodeId());
 
-            super.dynamicIndexCreate(cacheName, tblName, idxDesc, ifNotExists, cacheVisitor);
+            super.dynamicIndexCreate(schemaName, tblName, idxDesc, ifNotExists, cacheVisitor);
         }
 
         /** {@inheritDoc} */
-        @Override public void dynamicIndexDrop(@NotNull String cacheName, String idxName, boolean ifExists)
+        @Override public void dynamicIndexDrop(@NotNull String schemaName, String idxName, boolean ifExists)
             throws IgniteCheckedException{
             awaitIndexing(ctx.localNodeId());
 
-            super.dynamicIndexDrop(cacheName, idxName, ifExists);
+            super.dynamicIndexDrop(schemaName, idxName, ifExists);
         }
     }
 }