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/12/29 09:28:24 UTC

[07/15] ignite git commit: IGNITE-5949: SQL: added ALTER TABLE DROP COLUMN support. This closes #3258.

IGNITE-5949: SQL: added ALTER TABLE DROP COLUMN support. This closes #3258.


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

Branch: refs/heads/ignite-zk
Commit: a39468733d4d472ae5054bcd55634dde73c1c5a5
Parents: df3c4df
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Thu Dec 28 18:42:08 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 28 18:42:08 2017 +0300

----------------------------------------------------------------------
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |   2 +-
 .../processors/query/GridQueryIndexing.java     |  15 +
 .../processors/query/GridQueryProcessor.java    | 133 +++++++-
 .../internal/processors/query/QuerySchema.java  |  30 +-
 .../query/QueryTypeDescriptorImpl.java          |  19 ++
 .../internal/processors/query/QueryUtils.java   |  69 ++++
 .../SchemaAlterTableDropColumnOperation.java    |  95 ++++++
 ...IgniteClientCacheInitializationFailTest.java |   6 +
 .../processors/query/h2/IgniteH2Indexing.java   |  33 ++
 .../query/h2/ddl/DdlStatementsProcessor.java    |  55 ++++
 .../processors/query/h2/opt/GridH2Table.java    |  58 ++++
 .../h2/sql/GridSqlAlterTableDropColumn.java     | 113 +++++++
 .../query/h2/sql/GridSqlQueryParser.java        |  45 ++-
 ...ynamicColumnsAbstractConcurrentSelfTest.java | 238 +++++++++++---
 .../cache/index/DynamicColumnsAbstractTest.java |  59 +++-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  | 324 ++++++++++++++++++-
 .../IgnitePersistentStoreSchemaLoadTest.java    |   8 +-
 17 files changed, 1247 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index fb96f31..6f6d6c5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -115,7 +115,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
      * @throws SQLException if failed.
      */
     public void testUnsupportedSql() throws SQLException {
-        checkErrorState("ALTER TABLE \"test\".Integer DROP COLUMN _key", "0A000");
+        checkErrorState("ALTER TABLE \"test\".Integer MODIFY COLUMN _key CHAR", "0A000");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 4610025..2a34bfc 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
@@ -181,6 +181,21 @@ public interface GridQueryIndexing {
         boolean ifColNotExists) throws IgniteCheckedException;
 
     /**
+     * Drop columns from dynamic table.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param cols Columns to drop.
+     * @param ifTblExists Ignore operation if target table does not exist (instead of throwing an error).
+     * @param ifColExists Ignore operation if column does not exist (instead of throwing an error) - is honored only
+     *     for single column case.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+    public void dynamicDropColumn(String schemaName, String tblName, List<String> cols, boolean ifTblExists,
+        boolean ifColExists) throws IgniteCheckedException;
+
+    /**
      * Registers cache.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 2d1e392..dbe2e9b 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
@@ -87,6 +87,7 @@ import org.apache.ignite.internal.processors.query.schema.message.SchemaOperatio
 import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation;
+import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
@@ -774,6 +775,16 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                                         processDynamicAddColumn(typeDesc, opAddCol.columns());
                                     }
+                                    else if (op0 instanceof SchemaAlterTableDropColumnOperation) {
+                                        SchemaAlterTableDropColumnOperation opDropCol =
+                                            (SchemaAlterTableDropColumnOperation)op0;
+
+                                        QueryTypeDescriptorImpl typeDesc = tblTypMap.get(opDropCol.tableName());
+
+                                        assert typeDesc != null;
+
+                                        processDynamicDropColumn(typeDesc, opDropCol.columns());
+                                    }
                                     else
                                         assert false;
                                 }
@@ -1012,6 +1023,39 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
             }
         }
+        else if (op instanceof SchemaAlterTableDropColumnOperation) {
+            SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+            type = type(cacheName, op0.tableName());
+
+            if (type == null) {
+                if (op0.ifTableExists())
+                    nop = true;
+                else
+                    err = new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                        op0.tableName());
+            }
+            else {
+                for (String name : op0.columns()) {
+                    if (err != null)
+                        break;
+
+                    if (!type.hasField(name)) {
+                        if (op0.ifExists()) {
+                            assert op0.columns().size() == 1;
+
+                            nop = true;
+                        }
+                        else
+                            err = new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND, name);
+
+                        break;
+                    }
+
+                    err = QueryUtils.validateDropColumn(type, name);
+                }
+            }
+        }
         else
             err = new SchemaOperationException("Unsupported operation: " + op);
 
@@ -1142,6 +1186,53 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
             }
         }
+        else if (op instanceof SchemaAlterTableDropColumnOperation) {
+            SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+            QueryEntity e = tblMap.get(op0.tableName());
+
+            if (e == null) {
+                if (op0.ifTableExists())
+                    nop = true;
+                else
+                    err = new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                        op0.tableName());
+            }
+            else {
+                Map<String, String> aliases = e.getAliases();
+
+                for (String colName : op0.columns()) {
+                    if (err != null)
+                        break;
+
+                    String fldName = colName;
+
+                    if (!F.isEmpty(aliases)) {
+                        for (Map.Entry<String, String> a : aliases.entrySet()) {
+                            if (colName.equals(a.getValue())) {
+                                fldName = a.getKey();
+
+                                break;
+                            }
+                        }
+                    }
+
+                    if (!e.getFields().containsKey(fldName)) {
+                        if (op0.ifExists()) {
+                            assert op0.columns().size() == 1;
+
+                            nop = true;
+                        }
+                        else
+                            err = new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND, fldName);
+
+                        break;
+                    }
+
+                    err = QueryUtils.validateDropColumn(e, fldName, colName);
+                }
+            }
+        }
         else
             err = new SchemaOperationException("Unsupported operation: " + op);
 
@@ -1262,7 +1353,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     idxs.remove(idxKey);
                 }
                 else {
-                    assert op instanceof SchemaAlterTableAddColumnOperation;
+                    assert (op instanceof SchemaAlterTableAddColumnOperation ||
+                        op instanceof SchemaAlterTableDropColumnOperation);
 
                     // No-op - all processing is done at "local" stage
                     // as we must update both table and type descriptor atomically.
@@ -1353,6 +1445,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 idx.dynamicAddColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(),
                     op0.ifNotExists());
             }
+            else if (op instanceof SchemaAlterTableDropColumnOperation) {
+                SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+                processDynamicDropColumn(type, op0.columns());
+
+                idx.dynamicDropColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(),
+                    op0.ifExists());
+            }
             else
                 throw new SchemaOperationException("Unsupported operation: " + op);
         }
@@ -2226,6 +2326,24 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Entry point for drop column procedure.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Target table name.
+     * @param cols Columns to drop.
+     * @param ifTblExists Ignore operation if target table doesn't exist.
+     * @param ifExists Ignore operation if column does not exist.
+     */
+    public IgniteInternalFuture<?> dynamicColumnRemove(String cacheName, String schemaName, String tblName,
+        List<String> cols, boolean ifTblExists, boolean ifExists) {
+
+        SchemaAlterTableDropColumnOperation op = new SchemaAlterTableDropColumnOperation(UUID.randomUUID(), cacheName,
+            schemaName, tblName, cols, ifTblExists, ifExists);
+
+        return startIndexOperationDistributed(op);
+    }
+
+    /**
      * Start distributed index change operation.
      *
      * @param op Operation.
@@ -2317,6 +2435,19 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Remove fields from type descriptor.
+     *
+     * @param d Type descriptor to update.
+     * @param cols Columns to remove.
+     * @throws IgniteCheckedException
+     */
+    private void processDynamicDropColumn(QueryTypeDescriptorImpl d, List<String> cols)
+        throws IgniteCheckedException {
+        for (String field : cols)
+            d.removeProperty(field);
+    }
+
+    /**
      *
      * @param cacheName Cache name.
      * @param sql Query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
index 62a9ecd..5cbae29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
@@ -30,6 +30,7 @@ import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation;
+import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation;
 import org.apache.ignite.internal.util.typedef.F;
@@ -148,9 +149,7 @@ public class QuerySchema implements Serializable {
                     }
                 }
             }
-            else {
-                assert op instanceof SchemaAlterTableAddColumnOperation;
-
+            else if (op instanceof SchemaAlterTableAddColumnOperation) {
                 SchemaAlterTableAddColumnOperation op0 = (SchemaAlterTableAddColumnOperation)op;
 
                 int targetIdx = -1;
@@ -199,6 +198,31 @@ public class QuerySchema implements Serializable {
                 if (replaceTarget)
                     ((List<QueryEntity>)entities).set(targetIdx, target);
             }
+            else {
+                assert op instanceof SchemaAlterTableDropColumnOperation;
+
+                SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+                int targetIdx = -1;
+
+                for (int i = 0; i < entities.size(); i++) {
+                    QueryEntity entity = ((List<QueryEntity>)entities).get(i);
+
+                    if (F.eq(entity.getTableName(), op0.tableName())) {
+                        targetIdx = i;
+
+                        break;
+                    }
+                }
+
+                if (targetIdx == -1)
+                    return;
+
+                QueryEntity entity = ((List<QueryEntity>)entities).get(targetIdx);
+
+                for (String field : op0.columns())
+                    entity.getFields().remove(field);
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index 72adefd..de58a4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -384,6 +384,25 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     }
 
     /**
+     * Removes a property with specified name.
+     *
+     * @param name Name of a property to remove.
+     */
+    public void removeProperty(String name) throws IgniteCheckedException {
+        GridQueryProperty prop = props.remove(name);
+
+        if (prop == null)
+            throw new IgniteCheckedException("Property with name '" + name + "' does not exist.");
+
+        if (validateProps != null)
+            validateProps.remove(prop);
+
+        uppercaseProps.remove(name.toUpperCase());
+
+        fields.remove(name);
+    }
+
+    /**
      * @param schemaName Schema name.
      */
     public void schemaName(String schemaName) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 9584e05..91509f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -1238,6 +1238,75 @@ public class QueryUtils {
     }
 
     /**
+     * Checks if given column can be removed from table using its {@link QueryEntity}.
+     *
+     * @param entity Query entity.
+     * @param fieldName Name of the field of the key or value object.
+     * @param colName Name of the column.
+     * @return {@code null} if it's OK to remove the column and exception otherwise.
+     */
+    public static SchemaOperationException validateDropColumn(QueryEntity entity, String fieldName, String colName) {
+        if (F.eq(fieldName, entity.getKeyFieldName()) || KEY_FIELD_NAME.equalsIgnoreCase(fieldName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache key");
+
+        if (F.eq(fieldName, entity.getValueFieldName()) || VAL_FIELD_NAME.equalsIgnoreCase(fieldName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache value");
+
+        Set<String> keyFields = entity.getKeyFields();
+
+        if (keyFields != null && keyFields.contains(fieldName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it is a part of a cache key");
+
+        Collection<QueryIndex> indexes = entity.getIndexes();
+
+        if (indexes != null) {
+            for (QueryIndex idxDesc : indexes) {
+                if (idxDesc.getFields().containsKey(fieldName))
+                    return new SchemaOperationException("Cannot drop column \"" + colName +
+                        "\" because an index exists (\"" + idxDesc.getName() + "\") that uses the column.");
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * Checks if given column can be removed from the table using its {@link GridQueryTypeDescriptor}.
+     *
+     * @param type Type descriptor.
+     * @param colName Name of the column.
+     * @return {@code null} if it's OK to remove the column and exception otherwise.
+     */
+    public static SchemaOperationException validateDropColumn(GridQueryTypeDescriptor type, String colName) {
+        if (F.eq(colName, type.keyFieldName()) || KEY_FIELD_NAME.equalsIgnoreCase(colName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache key");
+
+        if (F.eq(colName, type.valueFieldName()) || VAL_FIELD_NAME.equalsIgnoreCase(colName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache value");
+
+        GridQueryProperty prop = type.property(colName);
+
+        if (prop != null && prop.key())
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it is a part of a cache key");
+
+        Collection<GridQueryIndexDescriptor> indexes = type.indexes().values();
+
+        for (GridQueryIndexDescriptor idxDesc : indexes) {
+            if (idxDesc.fields().contains(colName))
+                return new SchemaOperationException("Cannot drop column \"" + colName +
+                    "\" because an index exists (\"" + idxDesc.name() + "\") that uses the column.");
+        }
+
+        return null;
+    }
+
+    /**
      * Private constructor.
      */
     private QueryUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java
new file mode 100644
index 0000000..0437548
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java
@@ -0,0 +1,95 @@
+/*
+ * 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.schema.operation;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Schema alter table drop column operation.
+ */
+public class SchemaAlterTableDropColumnOperation extends SchemaAbstractAlterTableOperation {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Target table name. */
+    private final String tblName;
+
+    /** Columns to drop. */
+    private final List<String> cols;
+
+    /** Ignore operation if target table doesn't exist. */
+    private final boolean ifTblExists;
+
+    /** Ignore operation if column does not exist. */
+    private final boolean ifExists;
+
+    /**
+     * Constructor.
+     *
+     * @param opId Operation id.
+     * @param schemaName Schema name.
+     * @param tblName Target table name.
+     * @param cols Columns to drop.
+     * @param ifTblExists Ignore operation if target table doesn't exist.
+     * @param ifExists Ignore operation if column does not exist.
+     */
+    public SchemaAlterTableDropColumnOperation(UUID opId, String cacheName, String schemaName, String tblName,
+        List<String> cols, boolean ifTblExists, boolean ifExists) {
+        super(opId, cacheName, schemaName);
+
+        this.tblName = tblName;
+        this.cols = cols;
+        this.ifTblExists = ifTblExists;
+        this.ifExists = ifExists;
+    }
+
+    /**
+     * @return Ignore operation if table doesn't exist.
+     */
+    public boolean ifTableExists() {
+        return ifTblExists;
+    }
+
+    /**
+     * @return Columns to drop.
+     */
+    public List<String> columns() {
+        return cols;
+    }
+
+    /**
+     * @return Quietly abort this command if column does not exist (honored only in single column case).
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @return Target table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SchemaAlterTableDropColumnOperation.class, this, "parent", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index 366230d..4b93a46 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -292,6 +292,12 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
+        @Override public void dynamicDropColumn(String schemaName, String tblName, List<String> cols,
+            boolean ifTblExists, boolean ifColExists) throws IgniteCheckedException {
+            // No-op
+        }
+
+        /** {@inheritDoc} */
         @Override public void registerCache(String cacheName, String schemaName,
             GridCacheContext<?, ?> cctx) throws IgniteCheckedException {
             if (FAILED_CACHES.contains(cctx.name()) && cctx.kernalContext().clientNode())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 6fdcd27..7c451a5 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
@@ -144,6 +144,7 @@ import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Insert;
+import org.h2.command.dml.NoOperation;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
@@ -764,6 +765,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         clearCachedQueries();
     }
 
+    /** {@inheritDoc} */
+    @Override public void dynamicDropColumn(String schemaName, String tblName, List<String> cols, boolean ifTblExists,
+        boolean ifColExists) throws IgniteCheckedException {
+        // Locate table.
+        H2Schema schema = schemas.get(schemaName);
+
+        H2TableDescriptor desc = (schema != null ? schema.tableByName(tblName) : null);
+
+        if (desc == null) {
+            if (!ifTblExists)
+                throw new IgniteCheckedException("Table not found in internal H2 database [schemaName=" + schemaName +
+                    ",tblName=" + tblName + ']');
+            else
+                return;
+        }
+
+        desc.table().dropColumns(cols, ifColExists);
+
+        clearCachedQueries();
+    }
+
     /**
      * Execute DDL command.
      *
@@ -1576,6 +1598,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sqlQry + ']', e);
                         }
                     }
+
+                    if (prepared instanceof NoOperation) {
+                        QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(
+                            Collections.singletonList(Collections.singletonList(0L)), null, false);
+
+                        resCur.fieldsMeta(UPDATE_RESULT_META);
+
+                        res.add(resCur);
+
+                        continue;
+                    }
                 }
 
                 assert twoStepQry != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 68aab49..b198922 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
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils;
 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.GridSqlAlterTableAddColumn;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAlterTableDropColumn;
 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;
@@ -364,6 +365,60 @@ public class DdlStatementsProcessor {
                     }
                 }
             }
+            else if (stmt0 instanceof GridSqlAlterTableDropColumn) {
+                GridSqlAlterTableDropColumn cmd = (GridSqlAlterTableDropColumn)stmt0;
+
+                GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
+
+                if (tbl == null && cmd.ifTableExists()) {
+                    ctx.cache().createMissingQueryCaches();
+
+                    tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
+                }
+
+                if (tbl == null) {
+                    if (!cmd.ifTableExists())
+                        throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                            cmd.tableName());
+                }
+                else {
+                    assert tbl.rowDescriptor() != null;
+
+                    if (QueryUtils.isSqlType(tbl.rowDescriptor().type().valueClass()))
+                        throw new SchemaOperationException("Cannot drop column(s) because table was created " +
+                            "with " + PARAM_WRAP_VALUE + "=false option.");
+
+                    List<String> cols = new ArrayList<>(cmd.columns().length);
+
+                    GridQueryTypeDescriptor type = tbl.rowDescriptor().type();
+
+                    for (String colName : cmd.columns()) {
+                        if (!tbl.doesColumnExist(colName)) {
+                            if ((!cmd.ifExists() || cmd.columns().length != 1)) {
+                                throw new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND,
+                                    colName);
+                            }
+                            else {
+                                cols = null;
+
+                                break;
+                            }
+                        }
+
+                        SchemaOperationException err = QueryUtils.validateDropColumn(type, colName);
+
+                        if (err != null)
+                            throw err;
+
+                        cols.add(colName);
+                    }
+
+                    if (cols != null) {
+                        fut = ctx.query().dynamicColumnRemove(tbl.cacheName(), cmd.schemaName(),
+                            type.tableName(), cols, cmd.ifTableExists(), cmd.ifExists());
+                    }
+                }
+            }
             else
                 throw new IgniteSQLException("Unsupported DDL operation: " + sql,
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index cdffa16..c803a36 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -57,6 +57,7 @@ import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
 
 /**
@@ -939,6 +940,63 @@ public class GridH2Table extends TableBase {
         }
     }
 
+    /**
+     *
+     * @param cols
+     * @param ifExists
+     */
+    public void dropColumns(List<String> cols, boolean ifExists) {
+        assert !ifExists || cols.size() == 1;
+
+        lock(true);
+
+        try {
+            int size = columns.length;
+
+            for (String name : cols) {
+                if (!doesColumnExist(name)) {
+                    if (ifExists && cols.size() == 1)
+                        return;
+                    else
+                        throw new IgniteSQLException("Column does not exist [tblName=" + getName() +
+                            ", colName=" + name + ']');
+                }
+
+                size --;
+            }
+
+            assert size > DEFAULT_COLUMNS_COUNT;
+
+            Column[] newCols = new Column[size];
+
+            int dst = 0;
+
+            for (int i = 0; i < columns.length; i++) {
+                Column column = columns[i];
+
+                for (String name : cols) {
+                    if (F.eq(name, column.getName())) {
+                        column = null;
+
+                        break;
+                    }
+                }
+
+                if (column != null)
+                    newCols[dst++] = column;
+            }
+
+            setColumns(newCols);
+
+            desc.refreshMetadataFromTypeDescriptor();
+
+            setModified();
+        }
+        finally {
+            unlock(true);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public Column[] getColumns() {
         Boolean insertHack = INSERT_HACK.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java
new file mode 100644
index 0000000..6f8c923
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java
@@ -0,0 +1,113 @@
+/*
+ * 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;
+
+/**
+ * ALTER TABLE DROP COLUMN command data holder.
+ */
+public class GridSqlAlterTableDropColumn extends GridSqlStatement {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Target table name. */
+    private String tblName;
+
+    /** Columns to drop. */
+    private String[] cols;
+
+    /** Quietly abort this command if column does not exist (honored only in single column case). */
+    private boolean ifExists;
+
+    /** Quietly abort this command if target table does not exist. */
+    private boolean ifTblExists;
+
+    /**
+     * @return Columns to drop.
+     */
+    public String[] columns() {
+        return cols;
+    }
+
+    /**
+     * @param cols Columns to drop.
+     */
+    public void columns(String[] cols) {
+        this.cols = cols;
+    }
+
+    /**
+     * @return Quietly abort this command if column does not exist (honored only in single column case).
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @param ifExists Quietly abort this command if column does not exist (honored only in single column case).
+     */
+    public void ifExists(boolean ifExists) {
+        this.ifExists = ifExists;
+    }
+
+    /**
+     * @return Quietly abort this command if target table does not exist.
+     */
+    public boolean ifTableExists() {
+        return ifTblExists;
+    }
+
+    /**
+     * @param ifTblExists Quietly abort this command if target table does not exist.
+     */
+    public void ifTableExists(boolean ifTblExists) {
+        this.ifTblExists = ifTblExists;
+    }
+
+    /**
+     * @return Target table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Target table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 3b19e39..388231f 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
@@ -430,6 +430,10 @@ public class GridSqlQueryParser {
         getter(AlterTableAlterColumn.class, "columnsToAdd");
 
     /** */
+    private static final Getter<AlterTableAlterColumn, ArrayList<Column>> ALTER_COLUMN_REMOVE_COLS =
+        getter(AlterTableAlterColumn.class, "columnsToRemove");
+
+    /** */
     private static final Getter<AlterTableAlterColumn, Boolean> ALTER_COLUMN_IF_NOT_EXISTS =
         getter(AlterTableAlterColumn.class, "ifNotExists");
 
@@ -1150,6 +1154,9 @@ public class GridSqlQueryParser {
             case CommandInterface.ALTER_TABLE_ADD_COLUMN:
                 return parseAddColumn(stmt);
 
+            case CommandInterface.ALTER_TABLE_DROP_COLUMN:
+                return parseDropColumn(stmt);
+
             default: {
                 String stmtName = null;
 
@@ -1164,11 +1171,6 @@ public class GridSqlQueryParser {
                         stmtName = "ALTER COLUMN";
 
                         break;
-
-                    case CommandInterface.ALTER_TABLE_DROP_COLUMN:
-                        stmtName = "DROP COLUMN";
-
-                        break;
                 }
 
                 if (stmtName == null) {
@@ -1258,6 +1260,39 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * Parse {@code ALTER TABLE ... DROP COLUMN} statement.
+     * @param dropCol H2 statement.
+     * @see <a href="http://www.h2database.com/html/grammar.html#alter_table_add"></a>
+     */
+    private GridSqlStatement parseDropColumn(AlterTableAlterColumn dropCol) {
+        assert dropCol.getType() == CommandInterface.ALTER_TABLE_DROP_COLUMN;
+
+        GridSqlAlterTableDropColumn res = new GridSqlAlterTableDropColumn();
+
+        ArrayList<Column> h2DropCols = ALTER_COLUMN_REMOVE_COLS.get(dropCol);
+
+        String[] gridDropCols = new String[h2DropCols.size()];
+
+        for (int i = 0; i < h2DropCols.size(); i++)
+            gridDropCols[i] = h2DropCols.get(i).getName();
+
+        res.columns(gridDropCols);
+
+        if (gridDropCols.length == 1)
+            res.ifExists(!ALTER_COLUMN_IF_NOT_EXISTS.get(dropCol));
+
+        res.ifTableExists(ALTER_COLUMN_IF_TBL_EXISTS.get(dropCol));
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(dropCol);
+
+        res.schemaName(schema.getName());
+
+        res.tableName(ALTER_COLUMN_TBL_NAME.get(dropCol));
+
+        return res;
+    }
+
+    /**
      * @param name Param name.
      * @param val Param value.
      * @param res Table params to update.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
index 145947a..31ccd67 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -84,6 +85,9 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     /** SQL statement to create test table accompanied by template specification. */
     private final String createSql;
 
+    /** SQL statement to create test table with additional columns. */
+    private final String createSql4Cols;
+
     /** Latches to block certain index operations. */
     private static final ConcurrentHashMap<UUID, T3<CountDownLatch, AtomicBoolean, CountDownLatch>> BLOCKS =
         new ConcurrentHashMap<>();
@@ -103,7 +107,11 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     DynamicColumnsAbstractConcurrentSelfTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode) {
         this.cacheMode = cacheMode;
         this.atomicityMode = atomicityMode;
-        createSql =  CREATE_SQL + " WITH \"template=TPL\"";
+
+        final String template = " WITH \"template=TPL\"";
+
+        createSql =  CREATE_SQL + template;
+        createSql4Cols = CREATE_SQL_4_COLS + template;
     }
 
     /** {@inheritDoc} */
@@ -143,7 +151,26 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      *
      * @throws Exception If failed.
      */
-    public void testCoordinatorChange() throws Exception {
+    public void testAddColumnCoordinatorChange() throws Exception {
+        checkCoordinatorChange(true);
+    }
+
+    /**
+     * Make sure that coordinator migrates correctly between nodes.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropColumnCoordinatorChange() throws Exception {
+        checkCoordinatorChange(false);
+    }
+
+    /**
+     * Make sure that coordinator migrates correctly between nodes.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    public void checkCoordinatorChange(boolean addOrRemove) throws Exception {
         CountDownLatch finishLatch = new CountDownLatch(2);
 
         // Start servers.
@@ -159,12 +186,14 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(cli);
 
-        run(cli, createSql);
+        run(cli, addOrRemove ? createSql : createSql4Cols);
 
         // Test migration between normal servers.
         CountDownLatch idxLatch = blockIndexing(srv1Id);
 
-        IgniteInternalFuture<?> colFut1 = addCols(cli, QueryUtils.DFLT_SCHEMA, c("age", Integer.class.getName()));
+        IgniteInternalFuture<?> colFut1 = addOrRemove ?
+            addCols(cli, QueryUtils.DFLT_SCHEMA, c("age", Integer.class.getName())) :
+            dropCols(cli, QueryUtils.DFLT_SCHEMA, "AGE");
 
         U.await(idxLatch);
 
@@ -176,12 +205,15 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         colFut1.get();
 
         // Port number is for srv2.
-        checkTableState(srv2, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("age", Integer.class.getName()));
+        checkTableState(srv2, QueryUtils.DFLT_SCHEMA, TBL_NAME,
+            addOrRemove ? c("age", Integer.class.getName()) : c("CITY", String.class.getName()));
 
         // Test migration from normal server to non-affinity server.
         idxLatch = blockIndexing(srv2Id);
 
-        IgniteInternalFuture<?> colFut2 = addCols(cli, QueryUtils.DFLT_SCHEMA, c("city", String.class.getName()));
+        IgniteInternalFuture<?> colFut2 = addOrRemove ?
+            addCols(cli, QueryUtils.DFLT_SCHEMA, c("city", String.class.getName())) :
+            dropCols(cli, QueryUtils.DFLT_SCHEMA, "CITY");
 
         idxLatch.countDown();
 
@@ -196,7 +228,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         srv3.cache(QueryUtils.createTableCacheName(QueryUtils.DFLT_SCHEMA, "PERSON"));
 
         // Port number is for srv3.
-        checkTableState(srv3, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("city", String.class.getName()));
+        checkTableState(srv3, QueryUtils.DFLT_SCHEMA, TBL_NAME,
+            addOrRemove ? c("city", String.class.getName()) : c("NAME", String.class.getName()));
     }
 
     /**
@@ -220,12 +253,16 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         CountDownLatch idxLatch = blockIndexing(srv1);
 
-        QueryField c1 = c("age", Integer.class.getName());
-        QueryField c2 = c("city", String.class.getName());
+        QueryField c0 = c("ID", Integer.class.getName());
+        QueryField c1 = c("NAME", String.class.getName());
+        QueryField c2 = c("age", Integer.class.getName());
+        QueryField c3 = c("city", String.class.getName());
+
+        IgniteInternalFuture<?> colFut1 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c2);
 
-        IgniteInternalFuture<?> colFut1 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c1);
+        IgniteInternalFuture<?> colFut2 = dropCols(srv1, QueryUtils.DFLT_SCHEMA, c1.name());
 
-        IgniteInternalFuture<?> colFut2 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c2);
+        IgniteInternalFuture<?> colFut3 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c3);
 
         U.await(idxLatch);
 
@@ -236,36 +273,58 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         assert !colFut1.isDone();
         assert !colFut2.isDone();
+        assert !colFut3.isDone();
 
         unblockIndexing(srv1);
 
         colFut1.get();
         colFut2.get();
+        colFut3.get();
 
         U.await(finishLatch);
 
-        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c1, c2);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c0, c2, c3);
+    }
+
+    /**
+     * Test node join on pending add column operation.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNodeJoinOnPendingAddOperation() throws Exception {
+        checkNodeJoinOnPendingOperation(true);
     }
 
     /**
-     * Test node join on pending operation.
+     * Test node join on pending drop column operation.
      *
      * @throws Exception If failed.
      */
-    public void testNodeJoinOnPendingOperation() throws Exception {
+    public void testNodeJoinOnPendingDropOperation() throws Exception {
+        checkNodeJoinOnPendingOperation(false);
+    }
+
+    /**
+     * Check node join on pending operation.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    private void checkNodeJoinOnPendingOperation(boolean addOrRemove) throws Exception {
         CountDownLatch finishLatch = new CountDownLatch(4);
 
         IgniteEx srv1 = ignitionStart(serverConfiguration(1), finishLatch);
 
         createSqlCache(srv1);
 
-        run(srv1, createSql);
+        run(srv1, addOrRemove ? createSql : createSql4Cols);
 
         CountDownLatch idxLatch = blockIndexing(srv1);
 
-        QueryField c = c("age", Integer.class.getName());
+        QueryField c = c("AGE", Integer.class.getName());
 
-        IgniteInternalFuture<?> idxFut = addCols(srv1, QueryUtils.DFLT_SCHEMA, c);
+        IgniteInternalFuture<?> idxFut = addOrRemove ? addCols(srv1, QueryUtils.DFLT_SCHEMA, c) :
+            dropCols(srv1, QueryUtils.DFLT_SCHEMA, "CITY");
 
         U.await(idxLatch);
 
@@ -285,7 +344,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
-     * PUT/REMOVE data from cache and add column concurrently.
+     * PUT/REMOVE data from cache and add/drop column concurrently.
      *
      * @throws Exception If failed,
      */
@@ -302,7 +361,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(srv1);
 
-        run(srv1, createSql);
+        run(srv1, createSql4Cols);
 
         // Start data change operations from several threads.
         final AtomicBoolean stopped = new AtomicBoolean();
@@ -332,6 +391,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         addCols(srv1, QueryUtils.DFLT_SCHEMA, c("v", Integer.class.getName())).get();
 
+        dropCols(srv1, QueryUtils.DFLT_SCHEMA, "CITY").get();
+
         // Stop updates once index is ready.
         stopped.set(true);
 
@@ -340,7 +401,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         finishLatch.await();
 
         // Make sure new column is there.
-        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("v", Integer.class.getName()));
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("AGE", Integer.class.getName()),
+            c("v", Integer.class.getName()));
 
         run(srv1, "update person set \"v\" = case when mod(id, 2) <> 0 then substring(name, 7, length(name) - 6) " +
             "else null end");
@@ -417,7 +479,26 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      *
      * @throws Exception If failed.
      */
-    public void testConcurrentRebalance() throws Exception {
+    public void testAddConcurrentRebalance() throws Exception {
+        checkConcurrentRebalance(true);
+    }
+
+    /**
+     * Test index consistency on re-balance.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropConcurrentRebalance() throws Exception {
+        checkConcurrentRebalance(false);
+    }
+
+    /**
+     * Check index consistency on re-balance.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    public void checkConcurrentRebalance(boolean addOrRemove) throws Exception {
         // Start cache and populate it with data.
         IgniteEx srv1 = ignitionStart(serverConfiguration(1));
         Ignite srv2 = ignitionStart(serverConfiguration(2));
@@ -436,7 +517,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         QueryField c = c("salary", Double.class.getName());
 
-        final IgniteInternalFuture<?> idxFut = addCols(srv1, QueryUtils.DFLT_SCHEMA, c);
+        final IgniteInternalFuture<?> idxFut = addOrRemove ?
+            addCols(srv1, QueryUtils.DFLT_SCHEMA, c) : dropCols(srv1, QueryUtils.DFLT_SCHEMA, "NAME");
 
         U.await(idxLatch1);
         U.await(idxLatch2);
@@ -454,7 +536,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Validate index state.
         idxFut.get();
 
-        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, addOrRemove ? c : c("ID", Integer.class.getName()));
     }
 
     /**
@@ -473,7 +555,26 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      *
      * @throws Exception If failed.
      */
-    public void testConcurrentCacheDestroy() throws Exception {
+    public void testAddConcurrentCacheDestroy() throws Exception {
+        checkConcurrentCacheDestroy(true);
+    }
+
+    /**
+     * Check what happens in case cache is destroyed before operation is started.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropConcurrentCacheDestroy() throws Exception {
+        checkConcurrentCacheDestroy(false);
+    }
+
+    /**
+     * Check what happens in case cache is destroyed before operation is started.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    private void checkConcurrentCacheDestroy(boolean addOrRemove) throws Exception {
         // Start complex topology.
         Ignite srv1 = ignitionStart(serverConfiguration(1));
 
@@ -482,6 +583,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         Ignite cli = ignitionStart(clientConfiguration(4));
 
+        waitForDiscovery(srv1, grid(2), grid(3), cli);
+
         // Start cache and populate it with data.
         createSqlCache(cli);
 
@@ -494,7 +597,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         QueryField c = c("city", String.class.getName());
 
-        final IgniteInternalFuture<?> idxFut = addCols(srv1, QueryUtils.DFLT_SCHEMA, c);
+        final IgniteInternalFuture<?> idxFut = addOrRemove ?
+            addCols(srv1, QueryUtils.DFLT_SCHEMA, c) : dropCols(srv1, QueryUtils.DFLT_SCHEMA, "NAME");
 
         idxLatch.await();
 
@@ -515,8 +619,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
-     * Make sure that contended operations on the same index from different nodes do not hang when we issue both
-     * CREATE/DROP and SELECT statements.
+     * Make sure that contended operations on the same table from different nodes do not hang when we issue both
+     * ADD/DROP COLUMN and SELECT statements.
      *
      * @throws Exception If failed.
      */
@@ -538,16 +642,32 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         final AtomicInteger dynColCnt = new AtomicInteger();
 
+        final GridConcurrentHashSet<Integer> fields = new GridConcurrentHashSet<>();
+
         IgniteInternalFuture fut = multithreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 while (!stopped.get()) {
                     Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5));
 
-                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" +
-                        dynColCnt.getAndIncrement(), Integer.class.getName()));
+                    IgniteInternalFuture fut;
+
+                    int fieldNum = ThreadLocalRandom.current().nextInt(0, dynColCnt.get() + 1);
+
+                    boolean removed = fields.remove(fieldNum);
+
+                    if (removed)
+                        fut = dropCols(node, QueryUtils.DFLT_SCHEMA, "newCol" + fieldNum);
+                    else {
+                        fieldNum = dynColCnt.getAndIncrement();
 
+                        fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + fieldNum,
+                            Integer.class.getName()));
+                    }
                     try {
                         fut.get();
+
+                        if (!removed)
+                            fields.add(fieldNum);
                     }
                     catch (SchemaOperationException e) {
                         // No-op.
@@ -657,6 +777,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         reconnectClientNode(srv, cli, restartCache, dynamicCache, new RunnableX() {
             @Override public void run() throws Exception {
                 addCols(srv, schemaName, cols).get();
+
+                dropCols(srv, schemaName, "NAME").get();
             }
         });
 
@@ -713,7 +835,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
-     * Test concurrent node start/stop along with index operations. Nothing should hang.
+     * Test concurrent node start/stop along with add/drop column operations. Nothing should hang.
      *
      * @throws Exception If failed.
      */
@@ -782,16 +904,32 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
             }
         }, 1);
 
+        final GridConcurrentHashSet<Integer> fields = new GridConcurrentHashSet<>();
+
         IgniteInternalFuture idxFut = multithreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 while (!stopped.get()) {
                     Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5));
 
-                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" +
-                        dynColCnt.getAndIncrement(), Integer.class.getName()));
+                    IgniteInternalFuture fut;
+
+                    int fieldNum = ThreadLocalRandom.current().nextInt(0, dynColCnt.get() + 1);
+
+                    boolean removed = fields.remove(fieldNum);
+
+                    if (removed)
+                        fut = dropCols(node, QueryUtils.DFLT_SCHEMA, "newCol" + fieldNum);
+                    else {
+                        fieldNum = dynColCnt.getAndIncrement();
+
+                        fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + fieldNum, Integer.class.getName()));
+                    }
 
                     try {
                         fut.get();
+
+                        if (!removed)
+                            fields.add(fieldNum);
                     }
                     catch (SchemaOperationException e) {
                         // No-op.
@@ -816,7 +954,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Make sure cache is operational at this point.
         createSqlCache(cli);
 
-        QueryField[] expCols = new QueryField[dynColCnt.get()];
+        QueryField[] expCols = new QueryField[fields.size()];
 
         // Too many index columns kills indexing internals, have to limit number of the columns
         // to build the index on.
@@ -828,8 +966,14 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         String idxQry = "CREATE INDEX idx ON " + TBL_NAME + '(';
 
+        Integer[] sorted = fields.toArray(new Integer[fields.size()]);
+
+        Arrays.sort(sorted);
+
         for (int i = 0; i < expCols.length; i++) {
-            expCols[i] = c("newCol" + i, Integer.class.getName());
+            int fieldNum = sorted[i];
+
+            expCols[i] = c("newCol" + fieldNum, Integer.class.getName());
 
             if (i >= idxColsCnt)
                 continue;
@@ -840,9 +984,9 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                 idxQry += ", ";
             }
 
-            updQry += "\"newCol" + i + "\" = id + ?";
+            updQry += "\"newCol" + fieldNum + "\" = id + ?";
 
-            idxQry += "\"newCol" + i + '"';
+            idxQry += "\"newCol" + fieldNum + '"';
 
             args[i] = i;
         }
@@ -952,6 +1096,14 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
             super.dynamicAddColumn(schemaName, tblName, cols, ifTblExists, ifColNotExists);
         }
+
+        /** {@inheritDoc} */
+        @Override public void dynamicDropColumn(String schemaName, String tblName, List<String> cols,
+            boolean ifTblExists, boolean ifColExists) throws IgniteCheckedException {
+            awaitIndexing(ctx.localNodeId());
+
+            super.dynamicDropColumn(schemaName, tblName, cols, ifTblExists, ifColExists);
+        }
     }
 
     /**
@@ -969,6 +1121,20 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
+     *
+     * @param node Target node.
+     * @param schemaName Schema name.
+     * @param flds Columns to remove.
+     * @return DDL operation future.
+     */
+    private static IgniteInternalFuture<?> dropCols(Ignite node, String schemaName, String... flds) {
+        final String cacheName = F.eq(schemaName, QueryUtils.DFLT_SCHEMA) ? CACHE_NAME : "idx";
+
+        return ((IgniteEx)node).context().query().dynamicColumnRemove(cacheName, schemaName, TBL_NAME,
+            Arrays.asList(flds), false, false);
+    }
+
+    /**
      * Start SQL cache on given node.
      * @param node Node to create cache on.
      * @return Created cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
index 611f857..e507217 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
@@ -54,6 +54,10 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
     /** SQL to create test table. */
     final static String CREATE_SQL = "CREATE TABLE IF NOT EXISTS Person (id int primary key, name varchar)";
 
+    /** SQL to create test table with additional columns to drop. */
+    final static String CREATE_SQL_4_COLS = "CREATE TABLE IF NOT EXISTS Person (id int primary key, " +
+        "name varchar, age int, city varchar)";
+
     /** SQL to drop test table. */
     final static String DROP_SQL = "DROP TABLE Person";
 
@@ -66,8 +70,9 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
      * @param schemaName Schema name to look for the table in.
      * @param tblName Table name to check.
      * @param cols Columns whose presence must be checked.
+     * @return Number of other columns.
      */
-    static void checkTableState(IgniteEx node, String schemaName, String tblName, QueryField... cols)
+    static int checkTableState(IgniteEx node, String schemaName, String tblName, QueryField... cols)
         throws SQLException {
         List<QueryField> flds = new ArrayList<>();
 
@@ -103,6 +108,39 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
 
             assertEquals(exp.isNullable(), act.isNullable());
         }
+
+        return flds.size() - cols.length;
+    }
+
+    /**
+     * Checks presence of specific table column and returns it.
+     *
+     * @param node Node to check.
+     * @param schemaName Schema name to look for the table in.
+     * @param tblName Table name to check.
+     * @param colName Column name whose presence must be checked.
+     * @return field or {@code null} if not found.
+     * @throws SQLException if failed.
+     */
+    static QueryField getColumnMeta(IgniteEx node, String schemaName, String tblName, String colName)
+        throws SQLException {
+        try (Connection c = connect(node)) {
+            try (ResultSet rs = c.getMetaData().getColumns(null, schemaName, tblName, colName)) {
+                while (rs.next()) {
+                    String name = rs.getString("COLUMN_NAME");
+
+                    short type = rs.getShort("DATA_TYPE");
+
+                    String typeClsName = DataType.getTypeClassName(DataType.convertSQLTypeToValueType(type));
+
+                    short nullable = rs.getShort("NULLABLE");
+
+                    return new QueryField(name, typeClsName, nullable == 1);
+                }
+            }
+        }
+
+        return null;
     }
 
     /**
@@ -203,4 +241,23 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
             }
         }, IgniteSQLException.class, msg);
     }
+
+    /**
+     * Run specified statement expected to throw an exception with specified class and message.
+     *
+     * @param sql Statement.
+     * @param cls Expected exception class.
+     * @param msg Expected message.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    protected void assertThrowsAnyCause(final Ignite node, final String sql, Class<? extends Throwable> cls,
+        String msg) {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                run(node, sql);
+
+                return null;
+            }
+        }, cls, msg);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
index eb6d7e6..4e28ae9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.query.QueryField;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.testframework.config.GridTestProperties;
+import org.h2.jdbc.JdbcSQLException;
 
 import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
 
@@ -99,10 +100,11 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
      * @param schemaName Schema name.
      * @param tblName Table name.
      * @param cols Columns to look for.
+     * @return Number of other columns.
      * @throws SQLException if failed.
      */
-    private void checkTableState(String schemaName, String tblName, QueryField... cols) throws SQLException {
-        checkTableState(grid(nodeIndex()), schemaName, tblName, cols);
+    private int checkTableState(String schemaName, String tblName, QueryField... cols) throws SQLException {
+        return checkTableState(grid(nodeIndex()), schemaName, tblName, cols);
     }
 
     /**
@@ -185,9 +187,11 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
 
         run(cache, "CREATE INDEX cidx2 ON City(name)");
 
-        run(cache, "INSERT INTO City(id, name, population, state) values (5, 'New York', 15000000, 'New York')," +
+        run(cache, "INSERT INTO City(id, name, population, state_name) values (5, 'New York', 15000000, 'New York')," +
             "(7, 'Denver', 3000000, 'Colorado')");
 
+        run(cache, "ALTER TABLE City DROP COLUMN state_name");
+
         List<List<?>> res = run(cache, "SELECT p.name from Person p join City c on p.city = c.name where " +
             "c.population > 5000000 order by p.name");
 
@@ -253,10 +257,10 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
 
         checkTableState("City", "CITY", c);
 
-        run(cache, "INSERT INTO \"City\".City (_key, id, name, state, population) values " +
+        run(cache, "INSERT INTO \"City\".City (_key, id, name, state_name, population) values " +
             "(1, 1, 'Washington', 'DC', 2500000)");
 
-        List<List<?>> res = run(cache, "select _key, id, name, state, population from \"City\".City");
+        List<List<?>> res = run(cache, "select _key, id, name, state_name, population from \"City\".City");
 
         assertEquals(Collections.singletonList(Arrays.asList(1, 1, "Washington", "DC", 2500000)), res);
 
@@ -340,6 +344,302 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
     }
 
     /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumn() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR)");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("A", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+
+            run("ALTER TABLE test DROP COLUMN a");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+
+            run("ALTER TABLE test DROP COLUMN IF EXISTS a");
+
+            assertThrowsAnyCause("ALTER TABLE test DROP COLUMN a", JdbcSQLException.class, "Column \"A\" not found");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDroppedColumnMeta() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR)");
+
+            QueryField fld = getColumnMeta(grid(nodeIndex()), QueryUtils.DFLT_SCHEMA, "TEST", "A");
+
+            assertEquals("A", fld.name());
+            assertEquals(Integer.class.getName(), fld.typeName());
+
+            run("ALTER TABLE test DROP COLUMN a");
+
+            assertNull(getColumnMeta(grid(nodeIndex()), QueryUtils.DFLT_SCHEMA, "TEST", "A"));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropMultipleColumns() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR, c INT)");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("A", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true),
+                new QueryField("C", Integer.class.getName(), true)));
+
+            run("ALTER TABLE test DROP COLUMN a, c");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropNonExistingColumn() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT)");
+
+            assertThrowsAnyCause("ALTER TABLE test DROP COLUMN b", JdbcSQLException.class, "Column \"B\" not found");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnNonExistingTable() throws Exception {
+        assertThrowsAnyCause("ALTER TABLE nosuchtable DROP COLUMN a", JdbcSQLException.class,
+            "Table \"NOSUCHTABLE\" not found");
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnIfTableExists() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR)");
+
+            run("ALTER TABLE IF EXISTS test DROP COLUMN a");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnIfExists() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT)");
+
+            run("ALTER TABLE IF EXISTS test DROP COLUMN IF EXISTS a");
+
+            run("ALTER TABLE IF EXISTS test DROP COLUMN IF EXISTS b");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnIndexPresent() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b INT)");
+
+            run("CREATE INDEX b_index ON test(b)");
+
+            assertThrows("ALTER TABLE test DROP COLUMN b",
+                "Cannot drop column \"B\" because an index exists (\"B_INDEX\") that uses the column.");
+
+            run("DROP INDEX b_index");
+
+            run("ALTER TABLE test DROP COLUMN b");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("A", Integer.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnOnRealClassValuedTable() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, x VARCHAR) with \"wrap_value=false\"");
+
+            assertThrows("ALTER TABLE test DROP COLUMN x",
+                "Cannot drop column(s) because table was created with WRAP_VALUE=false option.");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnThatIsPartOfKey() throws Exception {
+        try {
+            run("CREATE TABLE test(id INT, a INT, b CHAR, PRIMARY KEY(id, a))");
+
+            assertThrows("ALTER TABLE test DROP COLUMN a",
+                "Cannot drop column \"A\" because it is a part of a cache key");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnThatIsKey() throws Exception {
+        try {
+            run("CREATE TABLE test(id INT PRIMARY KEY, a INT, b CHAR)");
+
+            assertThrows("ALTER TABLE test DROP COLUMN id",
+                "Cannot drop column \"ID\" because it represents an entire cache key");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnThatIsValue() throws Exception {
+        try {
+            run("CREATE TABLE test(id INT PRIMARY KEY, a INT, b CHAR)");
+
+            assertThrows("ALTER TABLE test DROP COLUMN _val",
+                "Cannot drop column \"_VAL\" because it represents an entire cache value");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     * Test that we can drop columns dynamically from tables associated
+     * with non dynamic caches storing user types as well.
+     *
+     * @throws SQLException if failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testDropColumnFromNonDynamicCacheWithRealValueType() throws SQLException {
+        CacheConfiguration<Integer, City> ccfg = defaultCacheConfiguration().setName("City")
+            .setIndexedTypes(Integer.class, City.class);
+
+        IgniteCache<Integer, ?> cache = ignite(nodeIndex()).getOrCreateCache(ccfg);
+
+        run(cache, "INSERT INTO \"City\".City (_key, id, name, state_name) VALUES " +
+            "(1, 1, 'Washington', 'DC')");
+
+        run(cache, "ALTER TABLE \"City\".City DROP COLUMN state_name");
+
+        doSleep(500);
+
+        QueryField c = c("NAME", String.class.getName());
+
+        checkTableState("City", "CITY", c);
+
+        run(cache, "INSERT INTO \"City\".City (_key, id, name) VALUES " +
+            "(2, 2, 'New York')");
+
+        assertThrowsAnyCause("SELECT state_name FROM \"City\".City",
+            JdbcSQLException.class, "Column \"STATE_NAME\" not found");
+
+        List<List<?>> res = run(cache, "SELECT _key, id, name FROM \"City\".City WHERE id = 1");
+
+        assertEquals(Collections.singletonList(Arrays.asList(1, 1, "Washington")), res);
+
+        res = run(cache, "SELECT * FROM \"City\".City WHERE id = 2");
+
+        assertEquals(Collections.singletonList(Arrays.asList(2, "New York")), res);
+
+        if (!Boolean.valueOf(GridTestProperties.getProperty(BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER))) {
+            City city = (City)cache.get(1);
+
+            assertEquals(1, city.id());
+            assertEquals("Washington", city.name());
+            assertEquals("DC", city.state());
+
+            city = (City)cache.get(2);
+
+            assertEquals(2, city.id());
+            assertEquals("New York", city.name());
+            assertEquals(null, city.state());
+        }
+        else {
+            BinaryObject city = (BinaryObject)cache.withKeepBinary().get(1);
+
+            assertEquals(1, (int)city.field("id"));
+            assertEquals("Washington", (String)city.field("name"));
+            assertEquals("DC", (String)city.field("state"));
+
+            city = (BinaryObject)cache.withKeepBinary().get(2);
+
+            assertEquals(2, (int)city.field("id"));
+            assertEquals("New York", (String)city.field("name"));
+            assertEquals(null, (String)city.field("state"));
+        }
+
+        cache.destroy();
+    }
+
+    /**
      * Test that {@code ADD COLUMN} fails for tables that have flat value.
      * @param tblName table name.
      */
@@ -364,6 +664,18 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
     }
 
     /**
+     * Run specified statement expected to throw an exception of specified class and message.
+     *
+     * @param sql Statement.
+     * @param cls Expected exception class.
+     * @param msg Expected message.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    protected void assertThrowsAnyCause(final String sql, Class<? extends Throwable> cls, String msg) {
+        assertThrowsAnyCause(grid(nodeIndex()), sql, cls, msg);
+    }
+
+    /**
      * Execute SQL command and return resulting dataset.
      * @param sql Statement.
      * @return result.
@@ -383,7 +695,7 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
         private String name;
 
         /** City state. */
-        @QuerySqlField
+        @QuerySqlField(name = "state_name")
         private String state;
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
index 1474954..2f6977a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
@@ -285,8 +285,12 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
                 .getAll();
 
         node.context().query().querySqlFieldsNoCache(
-            new SqlFieldsQuery("alter table \"Person\" add column \"age\" int").setSchema(schema), false)
-                .getAll();
+            new SqlFieldsQuery("alter table \"Person\" add column (\"age\" int, \"city\" char)")
+            .setSchema(schema), false).getAll();
+
+        node.context().query().querySqlFieldsNoCache(
+            new SqlFieldsQuery("alter table \"Person\" drop column \"city\"").setSchema(schema), false)
+            .getAll();
     }
 
     /**