http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 98d227c..a9c1a20 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
@@ -17,12 +17,10 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Set;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
@@ -35,11 +33,9 @@ import 
org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
-import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.h2.api.TableEngine;
 import org.h2.command.ddl.CreateTableData;
 import org.h2.engine.Session;
 import org.h2.index.Index;
@@ -50,7 +46,6 @@ import org.h2.result.SearchRow;
 import org.h2.result.SortOrder;
 import org.h2.table.IndexColumn;
 import org.h2.table.TableBase;
-import org.h2.table.TableFilter;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -72,7 +67,10 @@ public class GridH2Table extends TableBase {
     private final GridH2RowDescriptor desc;
 
     /** */
-    private final ArrayList<Index> idxs;
+    private volatile ArrayList<Index> idxs;
+
+    /** */
+    private final Map<String, GridH2IndexBase> tmpIdxs = new HashMap<>();
 
     /** */
     private final ReadWriteLock lock;
@@ -81,7 +79,7 @@ public class GridH2Table extends TableBase {
     private boolean destroyed;
 
     /** */
-    private final Set<Session> sessions = Collections.newSetFromMap(new 
ConcurrentHashMap8<Session,Boolean>());
+    private final ConcurrentMap<Session, Boolean> sessions = new 
ConcurrentHashMap8<>();
 
     /** */
     private final AtomicReference<Object[]> actualSnapshot = new 
AtomicReference<>();
@@ -106,11 +104,12 @@ public class GridH2Table extends TableBase {
      *
      * @param createTblData Table description.
      * @param desc Row descriptor.
+     * @param rowFactory Row factory.
      * @param idxsFactory Indexes factory.
      * @param spaceName Space name.
      */
-    public GridH2Table(CreateTableData createTblData, @Nullable 
GridH2RowDescriptor desc, IndexesFactory idxsFactory,
-        @Nullable String spaceName) {
+    public GridH2Table(CreateTableData createTblData, @Nullable 
GridH2RowDescriptor desc, H2RowFactory rowFactory,
+        GridH2SystemIndexFactory idxsFactory, @Nullable String spaceName) {
         super(createTblData);
 
         assert idxsFactory != null;
@@ -143,18 +142,18 @@ public class GridH2Table extends TableBase {
             }
         }
 
+        this.rowFactory = rowFactory;
+
         // Indexes must be created in the end when everything is ready.
-        rowFactory = idxsFactory.createRowFactory(this);
-        idxs = idxsFactory.createIndexes(this);
+        idxs = idxsFactory.createSystemIndexes(this);
 
         assert idxs != null;
 
         // Add scan index at 0 which is required by H2.
-        if (idxs.size() >= 2
-                && index(0).getIndexType().isHash())
-            idxs.add(0, new ScanIndex(index(1), index(0)));
+        if (idxs.size() >= 2 && index(0).getIndexType().isHash())
+            idxs.add(0, new GridH2PrimaryScanIndex(this, index(1), index(0)));
         else
-            idxs.add(0, new ScanIndex(index(0), null));
+            idxs.add(0, new GridH2PrimaryScanIndex(this, index(0), null));
 
         snapshotEnabled = desc == null || desc.snapshotableIndex();
 
@@ -238,6 +237,8 @@ public class GridH2Table extends TableBase {
             desc.guard().begin();
 
         try {
+            ensureNotDestroyed();
+
             GridH2AbstractKeyValueRow row = 
(GridH2AbstractKeyValueRow)pk.findOne(searchRow);
 
             if (row == null)
@@ -266,13 +267,21 @@ public class GridH2Table extends TableBase {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"LockAcquiredButNotSafelyReleased", 
"SynchronizationOnLocalVariableOrMethodParameter", "unchecked"})
-    @Override public boolean lock(@Nullable final Session ses, boolean 
exclusive, boolean force) {
-        if (ses != null) {
-            if (!sessions.add(ses))
-                return false;
+    @Override public boolean lock(Session ses, boolean exclusive, boolean 
force) {
+        Boolean putRes = sessions.putIfAbsent(ses, exclusive);
+
+        // In accordance with base method semantics, we'll return true if we 
were already exclusively locked
+        if (putRes != null)
+            return putRes;
+
+        ses.addLock(this);
+
+        lock(exclusive);
+
+        if (destroyed) {
+            unlock(exclusive);
 
-            ses.addLock(this);
+            throw new IllegalStateException("Table " + identifier() + " 
already destroyed.");
         }
 
         if (snapshotInLock())
@@ -320,6 +329,8 @@ public class GridH2Table extends TableBase {
         }
 
         try {
+            ensureNotDestroyed();
+
             // Try again inside of the lock.
             snapshots = actualSnapshot.get();
 
@@ -362,12 +373,6 @@ public class GridH2Table extends TableBase {
 
             throw new IgniteInterruptedException("Thread got interrupted while 
trying to acquire table lock.", e);
         }
-
-        if (destroyed) {
-            unlock(exclusive);
-
-            throw new IllegalStateException("Table " + identifier() + " 
already destroyed.");
-        }
     }
 
     /**
@@ -388,12 +393,6 @@ public class GridH2Table extends TableBase {
             throw new IgniteInterruptedException("Thread got interrupted while 
trying to acquire table lock.", e);
         }
 
-        if (destroyed) {
-            unlock(exclusive);
-
-            throw new IllegalStateException("Table " + identifier() + " 
already destroyed.");
-        }
-
         return true;
     }
 
@@ -409,6 +408,14 @@ public class GridH2Table extends TableBase {
     }
 
     /**
+     * Check if table is not destroyed.
+     */
+    private void ensureNotDestroyed() {
+        if (destroyed)
+            throw new IllegalStateException("Table " + identifier() + " 
already destroyed.");
+    }
+
+    /**
      * Must be called inside of write lock because when using multiple indexes 
we have to ensure that all of them have
      * the same contents at snapshot taking time.
      *
@@ -462,6 +469,8 @@ public class GridH2Table extends TableBase {
         lock(true);
 
         try {
+            ensureNotDestroyed();
+
             assert sessions.isEmpty() : sessions;
 
             destroyed = true;
@@ -475,12 +484,16 @@ public class GridH2Table extends TableBase {
     }
 
     /** {@inheritDoc} */
-    @Override public void unlock(@Nullable Session ses) {
-        if (ses != null && !sessions.remove(ses))
+    @Override public void unlock(Session ses) {
+        Boolean exclusive = sessions.remove(ses);
+
+        if (exclusive == null)
             return;
 
         if (snapshotInLock())
             releaseSnapshots();
+
+        unlock(exclusive);
     }
 
     /**
@@ -599,6 +612,8 @@ public class GridH2Table extends TableBase {
             desc.guard().begin();
 
         try {
+            ensureNotDestroyed();
+
             GridH2IndexBase pk = pk();
 
             if (!del) {
@@ -623,18 +638,11 @@ public class GridH2Table extends TableBase {
                 while (++i < len) {
                     GridH2IndexBase idx = index(i);
 
-                    assert !idx.getIndexType().isUnique() : "Unique indexes 
are not supported: " + idx;
-
-                    GridH2Row old2 = idx.put(row);
-
-                    if (old2 != null) { // Row was replaced in index.
-                        if (!eq(pk, old2, old))
-                            throw new IllegalStateException("Row conflict 
should never happen, unique indexes are " +
-                                "not supported [idx=" + idx + ", old=" + old + 
", old2=" + old2 + ']');
-                    }
-                    else if (old != null) // Row was not replaced, need to 
remove manually.
-                        idx.removex(old);
+                    addToIndex(idx, pk, row, old, false);
                 }
+
+                for (GridH2IndexBase idx : tmpIdxs.values())
+                    addToIndex(idx, pk, row, old, true);
             }
             else {
                 //  index(1) is PK, get full row from there (search row here 
contains only key but no other columns).
@@ -656,6 +664,9 @@ public class GridH2Table extends TableBase {
                         assert eq(pk, res, old) : "\n" + old + "\n" + res + 
"\n" + i + " -> " + index(i).getName();
                     }
 
+                    for (GridH2IndexBase idx : tmpIdxs.values())
+                        idx.remove(old);
+
                     size.decrement();
                 }
                 else
@@ -676,6 +687,31 @@ public class GridH2Table extends TableBase {
     }
 
     /**
+     * Add row to index.
+     *
+     * @param idx Index to add row to.
+     * @param pk Primary key index.
+     * @param row Row to add to index.
+     * @param old Previous row state, if any.
+     * @param tmp {@code True} if this is proposed index which may be not 
consistent yet.
+     */
+    private void addToIndex(GridH2IndexBase idx, Index pk, GridH2Row row, 
GridH2Row old, boolean tmp) {
+        assert !idx.getIndexType().isUnique() : "Unique indexes are not 
supported: " + idx;
+
+        GridH2Row old2 = idx.put(row);
+
+        if (old2 != null) { // Row was replaced in index.
+            if (!tmp) {
+                if (!eq(pk, old2, old))
+                    throw new IllegalStateException("Row conflict should never 
happen, unique indexes are " +
+                        "not supported [idx=" + idx + ", old=" + old + ", 
old2=" + old2 + ']');
+            }
+        }
+        else if (old != null) // Row was not replaced, need to remove manually.
+            idx.removex(old);
+    }
+
+    /**
      * Check row equality.
      *
      * @param pk Primary key index.
@@ -716,9 +752,119 @@ public class GridH2Table extends TableBase {
     }
 
     /** {@inheritDoc} */
-    @Override public Index addIndex(Session ses, String s, int i, 
IndexColumn[] idxCols, IndexType idxType,
-        boolean b, String s1) {
-        throw DbException.getUnsupportedException("addIndex");
+    @Override public Index addIndex(Session ses, String idxName, int idxId, 
IndexColumn[] cols, IndexType idxType,
+        boolean create, String idxComment) {
+        return commitUserIndex(ses, idxName);
+    }
+
+    /**
+     * Add index that is in an intermediate state and is still being built, 
thus is not used in queries until it is
+     * promoted.
+     *
+     * @param idx Index to add.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void proposeUserIndex(Index idx) throws IgniteCheckedException {
+        assert idx instanceof GridH2IndexBase;
+
+        lock(true);
+
+        try {
+            ensureNotDestroyed();
+
+            for (Index oldIdx : idxs) {
+                if (F.eq(oldIdx.getName(), idx.getName()))
+                    throw new IgniteCheckedException("Index already exists: " 
+ idx.getName());
+            }
+
+            Index oldTmpIdx = tmpIdxs.put(idx.getName(), (GridH2IndexBase)idx);
+
+            assert oldTmpIdx == null;
+        }
+        finally {
+            unlock(true);
+        }
+    }
+
+    /**
+     * Promote temporary index to make it usable in queries.
+     *
+     * @param ses H2 session.
+     * @param idxName Index name.
+     * @return Temporary index with given name.
+     */
+    private Index commitUserIndex(Session ses, String idxName) {
+        lock(true);
+
+        try {
+            ensureNotDestroyed();
+
+            Index idx = tmpIdxs.remove(idxName);
+
+            assert idx != null;
+
+            ArrayList<Index> newIdxs = new ArrayList<>(idxs.size() + 1);
+
+            newIdxs.addAll(idxs);
+
+            newIdxs.add(idx);
+
+            idxs = newIdxs;
+
+            database.addSchemaObject(ses, idx);
+
+            setModified();
+
+            return idx;
+        }
+        finally {
+            unlock(true);
+        }
+    }
+
+    /**
+     * Remove user index without promoting it.
+     *
+     * @param idxName Index name.
+     */
+    public void rollbackUserIndex(String idxName) {
+        lock(true);
+
+        try {
+            ensureNotDestroyed();
+
+            GridH2IndexBase rmvIdx = tmpIdxs.remove(idxName);
+
+            assert rmvIdx != null;
+        }
+        finally {
+            unlock(true);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeIndex(Index h2Idx) {
+        lock(true);
+
+        try {
+            ArrayList<Index> idxs = new ArrayList<>(this.idxs);
+
+            for (int i = 2; i < idxs.size(); i++) {
+                GridH2IndexBase idx = (GridH2IndexBase)idxs.get(i);
+
+                if (idx != h2Idx)
+                    continue;
+
+                idxs.remove(i);
+
+                this.idxs = idxs;
+
+                return;
+            }
+        }
+        finally {
+            unlock(true);
+        }
     }
 
     /** {@inheritDoc} */
@@ -848,136 +994,4 @@ public class GridH2Table extends TableBase {
         return rowFactory;
     }
 
-    /**
-     * H2 Table engine.
-     */
-    @SuppressWarnings({"PublicInnerClass", 
"FieldAccessedSynchronizedAndUnsynchronized"})
-    public static class Engine implements TableEngine {
-        /** */
-        private static GridH2RowDescriptor rowDesc;
-
-        /** */
-        private static IndexesFactory idxsFactory;
-
-        /** */
-        private static GridH2Table resTbl;
-
-        /** */
-        private static String spaceName;
-
-        /** {@inheritDoc} */
-        @Override public TableBase createTable(CreateTableData createTblData) {
-            resTbl = new GridH2Table(createTblData, rowDesc, idxsFactory, 
spaceName);
-
-            return resTbl;
-        }
-
-        /**
-         * Creates table using given connection, DDL clause for given type 
descriptor and list of indexes.
-         *
-         * @param conn Connection.
-         * @param sql DDL clause.
-         * @param desc Row descriptor.
-         * @param factory Indexes factory.
-         * @param space Space name.
-         * @throws SQLException If failed.
-         * @return Created table.
-         */
-        public static synchronized GridH2Table createTable(Connection conn, 
String sql,
-            @Nullable GridH2RowDescriptor desc, IndexesFactory factory, String 
space)
-            throws SQLException {
-            rowDesc = desc;
-            idxsFactory = factory;
-            spaceName = space;
-
-            try {
-                try (Statement s = conn.createStatement()) {
-                    s.execute(sql + " engine \"" + Engine.class.getName() + 
"\"");
-                }
-
-                return resTbl;
-            }
-            finally {
-                resTbl = null;
-                idxsFactory = null;
-                rowDesc = null;
-            }
-        }
-    }
-
-    /**
-     * Type which can create indexes list for given table.
-     */
-    @SuppressWarnings({"PackageVisibleInnerClass", "PublicInnerClass"})
-    public static interface IndexesFactory {
-        /**
-         * Create list of indexes. First must be primary key, after that all 
unique indexes and
-         * only then non-unique indexes.
-         * All indexes must be subtypes of {@link H2TreeIndex}.
-         *
-         * @param tbl Table to create indexes for.
-         * @return List of indexes.
-         */
-        ArrayList<Index> createIndexes(GridH2Table tbl);
-
-        /**
-         * @param tbl Table.
-         * @return Data store.
-         */
-        H2RowFactory createRowFactory(GridH2Table tbl);
-    }
-
-    /**
-     * Wrapper type for primary key.
-     */
-    @SuppressWarnings("PackageVisibleInnerClass")
-    class ScanIndex extends GridH2ScanIndex<GridH2IndexBase> {
-        /** */
-        static final String SCAN_INDEX_NAME_SUFFIX = "__SCAN_";
-
-        /** */
-        private final GridH2IndexBase hashIdx;
-
-        /**
-         * Constructor.
-         */
-        private ScanIndex(GridH2IndexBase treeIdx, GridH2IndexBase hashIdx) {
-            super(treeIdx);
-
-            this.hashIdx = hashIdx;
-        }
-
-        /**
-         *
-         */
-        @Override protected GridH2IndexBase delegate() {
-            if (hashIdx != null)
-                return rebuildFromHashInProgress ? hashIdx : super.delegate();
-            else {
-                assert !rebuildFromHashInProgress;
-
-                return super.delegate();
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public double getCost(Session ses, int[] masks, 
TableFilter[] filters, int filter,
-            SortOrder sortOrder) {
-            long rows = getRowCountApproximation();
-            double baseCost = getCostRangeIndex(masks, rows, filters, filter, 
sortOrder, true);
-            int mul = delegate().getDistributedMultiplier(ses, filters, 
filter);
-
-            return mul * baseCost;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getPlanSQL() {
-            return delegate().getTable().getSQL() + "." + 
SCAN_INDEX_NAME_SUFFIX;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getName() {
-            return delegate().getName() + SCAN_INDEX_NAME_SUFFIX;
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
index b73bb96..4395024 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
@@ -116,15 +116,7 @@ public class GridLuceneIndex implements AutoCloseable {
             throw new IgniteCheckedException(e);
         }
 
-        GridQueryIndexDescriptor idx = null;
-
-        for (GridQueryIndexDescriptor descriptor : type.indexes().values()) {
-            if (descriptor.type() == QueryIndexType.FULLTEXT) {
-                idx = descriptor;
-
-                break;
-            }
-        }
+        GridQueryIndexDescriptor idx = type.textIndex();
 
         if (idx != null) {
             Collection<String> fields = idx.fields();

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateIndex.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateIndex.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateIndex.java
new file mode 100644
index 0000000..50d455c
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateIndex.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.sql;
+
+import java.util.Map;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.h2.command.Parser;
+
+/**
+ * CREATE INDEX statement.
+ */
+public class GridSqlCreateIndex extends GridSqlStatement {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Attempt to create the index only if it does not exist. */
+    private boolean ifNotExists;
+
+    /** Index to create. */
+    private QueryIndex idx;
+
+    /**
+     * @return Schema name for new index.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name for new index.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return whether attempt to create the index should be made only if it 
does not exist.
+     */
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+    /**
+     * @param ifNotExists whether attempt to create the index should be made 
only if it does not exist.
+     */
+    public void ifNotExists(boolean ifNotExists) {
+        this.ifNotExists = ifNotExists;
+    }
+
+    /**
+     * @return Index to create.
+     */
+    public QueryIndex index() {
+        return idx;
+    }
+
+    /**
+     * @param idx Index to create.
+     */
+    public void index(QueryIndex idx) {
+        this.idx = idx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StringBuilder sb = new StringBuilder("CREATE ")
+            .append(idx.getIndexType() == QueryIndexType.GEOSPATIAL ? "SPATIAL 
" : "")
+            .append("INDEX ").append(ifNotExists ? "IF NOT EXISTS " : "")
+            .append(Parser.quoteIdentifier(schemaName)).append('.')
+            .append(Parser.quoteIdentifier(idx.getName())).append(" ON ")
+            .append(Parser.quoteIdentifier(tblName)).append(" (");
+
+        boolean first = true;
+
+        for (Map.Entry<String, Boolean> e : idx.getFields().entrySet()) {
+            if (first)
+                first = false;
+            else
+                sb.append(", ");
+
+            sb.append(Parser.quoteIdentifier(e.getKey())).append(e.getValue() 
? " ASC" : " DESC");
+        }
+
+        sb.append(')');
+
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropIndex.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropIndex.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropIndex.java
new file mode 100644
index 0000000..c844b49
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDropIndex.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.sql;
+
+import org.h2.command.Parser;
+
+/**
+ * DROP INDEX statement.
+ */
+public class GridSqlDropIndex extends GridSqlStatement {
+    /** Index name. */
+    private String name;
+
+    /** Schema name. */
+    private String schemaName;
+
+    /** Attempt to drop the index only if it exists. */
+    private boolean ifExists;
+
+    /**
+     * @return Index name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @param name Index name.
+     */
+    public void name(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return whether attempt to drop the index should be made only if it 
exists.
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @param ifExists whether attempt to drop the index should be made only 
if it exists.
+     */
+    public void ifExists(boolean ifExists) {
+        this.ifExists = ifExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + 
Parser.quoteIdentifier(schemaName) + '.' +
+            Parser.quoteIdentifier(name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 0f940e9..199a157 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
@@ -28,9 +28,16 @@ import java.util.List;
 import java.util.Map;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.h2.command.Command;
 import org.h2.command.CommandContainer;
 import org.h2.command.Prepared;
+import org.h2.command.ddl.CreateIndex;
+import org.h2.command.ddl.DropIndex;
+import org.h2.command.ddl.SchemaCommand;
 import org.h2.command.dml.Delete;
 import org.h2.command.dml.Explain;
 import org.h2.command.dml.Insert;
@@ -63,8 +70,10 @@ import org.h2.expression.ValueExpression;
 import org.h2.index.ViewIndex;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.result.SortOrder;
+import org.h2.schema.Schema;
 import org.h2.table.Column;
 import org.h2.table.FunctionTable;
+import org.h2.table.IndexColumn;
 import org.h2.table.RangeTable;
 import org.h2.table.Table;
 import org.h2.table.TableBase;
@@ -302,6 +311,48 @@ public class GridSqlQueryParser {
         GridSqlQueryParser.<Command, Prepared>getter(CommandContainer.class, 
"prepared");
 
     /** */
+    private static final Getter<CreateIndex, String> CREATE_INDEX_NAME = 
getter(CreateIndex.class, "indexName");
+
+    /** */
+    private static final Getter<CreateIndex, String> CREATE_INDEX_TABLE_NAME = 
getter(CreateIndex.class, "tableName");
+
+    /** */
+    private static final Getter<CreateIndex, IndexColumn[]> 
CREATE_INDEX_COLUMNS = getter(CreateIndex.class,
+        "indexColumns");
+
+    /** */
+    private static final Getter<CreateIndex, Boolean> CREATE_INDEX_SPATIAL = 
getter(CreateIndex.class, "spatial");
+
+    /** */
+    private static final Getter<CreateIndex, Boolean> CREATE_INDEX_PRIMARY_KEY 
= getter(CreateIndex.class,
+        "primaryKey");
+
+    /** */
+    private static final Getter<CreateIndex, Boolean> CREATE_INDEX_UNIQUE = 
getter(CreateIndex.class, "unique");
+
+    /** */
+    private static final Getter<CreateIndex, Boolean> CREATE_INDEX_HASH = 
getter(CreateIndex.class, "hash");
+
+    /** */
+    private static final Getter<CreateIndex, Boolean> 
CREATE_INDEX_IF_NOT_EXISTS = getter(CreateIndex.class,
+        "ifNotExists");
+
+    /** */
+    private static final Getter<IndexColumn, String> INDEX_COLUMN_NAME = 
getter(IndexColumn.class, "columnName");
+
+    /** */
+    private static final Getter<IndexColumn, Integer> INDEX_COLUMN_SORT_TYPE = 
getter(IndexColumn.class, "sortType");
+
+    /** */
+    private static final Getter<DropIndex, String> DROP_INDEX_NAME = 
getter(DropIndex.class, "indexName");
+
+    /** */
+    private static final Getter<DropIndex, Boolean> DROP_INDEX_IF_EXISTS = 
getter(DropIndex.class, "ifExists");
+
+    /** */
+    private static final Getter<SchemaCommand, Schema> SCHEMA_COMMAND_SCHEMA = 
getter(SchemaCommand.class, "schema");
+
+    /** */
     private final IdentityHashMap<Object, Object> h2ObjToGridObj = new 
IdentityHashMap<>();
 
     /** */
@@ -659,6 +710,72 @@ public class GridSqlQueryParser {
         return res;
     }
 
+
+
+    /**
+     * Parse {@code DROP INDEX} statement.
+     *
+     * @param dropIdx {@code DROP INDEX} statement.
+     * @see <a href="http://h2database.com/html/grammar.html#drop_index";>H2 
{@code DROP INDEX} spec.</a>
+     */
+    private GridSqlDropIndex parseDropIndex(DropIndex dropIdx) {
+        GridSqlDropIndex res = new GridSqlDropIndex();
+
+        res.name(DROP_INDEX_NAME.get(dropIdx));
+        res.schemaName(SCHEMA_COMMAND_SCHEMA.get(dropIdx).getName());
+        res.ifExists(DROP_INDEX_IF_EXISTS.get(dropIdx));
+
+        return res;
+    }
+
+    /**
+     * Parse {@code CREATE INDEX} statement.
+     *
+     * @param createIdx {@code CREATE INDEX} statement.
+     * @see <a href="http://h2database.com/html/grammar.html#create_index";>H2 
{@code CREATE INDEX} spec.</a>
+     */
+    private GridSqlCreateIndex parseCreateIndex(CreateIndex createIdx) {
+        if (CREATE_INDEX_HASH.get(createIdx) || 
CREATE_INDEX_PRIMARY_KEY.get(createIdx) ||
+            CREATE_INDEX_UNIQUE.get(createIdx))
+            throw new IgniteSQLException("Only SPATIAL modifier is supported 
for CREATE INDEX",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        GridSqlCreateIndex res = new GridSqlCreateIndex();
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(createIdx);
+
+        String tblName = CREATE_INDEX_TABLE_NAME.get(createIdx);
+
+        res.schemaName(schema.getName());
+        res.tableName(tblName);
+        res.ifNotExists(CREATE_INDEX_IF_NOT_EXISTS.get(createIdx));
+
+        QueryIndex idx = new QueryIndex();
+
+        idx.setName(CREATE_INDEX_NAME.get(createIdx));
+        idx.setIndexType(CREATE_INDEX_SPATIAL.get(createIdx) ? 
QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
+
+        IndexColumn[] cols = CREATE_INDEX_COLUMNS.get(createIdx);
+
+        LinkedHashMap<String, Boolean> flds = new LinkedHashMap<>(cols.length);
+
+        for (IndexColumn col : CREATE_INDEX_COLUMNS.get(createIdx)) {
+            int sortType = INDEX_COLUMN_SORT_TYPE.get(col);
+
+            if ((sortType & SortOrder.NULLS_FIRST) != 0 || (sortType & 
SortOrder.NULLS_LAST) != 0)
+                throw new IgniteSQLException("NULLS FIRST and NULLS LAST 
modifiers are not supported for index columns",
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            flds.put(INDEX_COLUMN_NAME.get(col), (sortType & 
SortOrder.DESCENDING) == 0);
+        }
+
+        idx.setFields(flds);
+
+        res.index(idx);
+
+        return res;
+    }
+
     /**
      * @param sortOrder Sort order.
      * @param qry Query.
@@ -722,6 +839,12 @@ public class GridSqlQueryParser {
         if (stmt instanceof Explain)
             return parse(EXPLAIN_COMMAND.get((Explain)stmt)).explain(true);
 
+        if (stmt instanceof CreateIndex)
+            return parseCreateIndex((CreateIndex)stmt);
+
+        if (stmt instanceof DropIndex)
+            return parseDropIndex((DropIndex)stmt);
+
         throw new CacheException("Unsupported SQL statement: " + stmt);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
new file mode 100644
index 0000000..a865b18
--- /dev/null
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractSchemaSelfTest.java
@@ -0,0 +1,512 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl;
+import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for dynamic schema changes.
+ */
+@SuppressWarnings("unchecked")
+public class AbstractSchemaSelfTest extends GridCommonAbstractTest {
+    /** Cache. */
+    protected static final String CACHE_NAME = "cache";
+
+    /** Table name. */
+    protected static final String TBL_NAME = tableName(ValueClass.class);
+
+    /** Table name 2. */
+    protected static final String TBL_NAME_2 = tableName(ValueClass2.class);
+
+    /** Index name 1. */
+    protected static final String IDX_NAME_1 = "idx_1";
+
+    /** Index name 2. */
+    protected static final String IDX_NAME_2 = "idx_2";
+
+    /** Index name 3. */
+    protected static final String IDX_NAME_3 = "idx_3";
+
+    /** Key ID field. */
+    protected static final String FIELD_KEY = "id";
+
+    /** Field 1. */
+    protected static final String FIELD_NAME_1 = "field1";
+
+    /** Field 1. */
+    protected static final String FIELD_NAME_2 = "field2";
+
+    /** Field 3. */
+    protected static final String FIELD_NAME_3 = "field3";
+
+    /**
+     * Get type on the given node for the given cache and table name. Type 
must exist.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @return Type.
+     */
+    protected static QueryTypeDescriptorImpl typeExisting(IgniteEx node, 
String cacheName, String tblName) {
+        QueryTypeDescriptorImpl res = type(node, cacheName, tblName);
+
+        assertNotNull(res);
+
+        return res;
+    }
+
+    /**
+     * Get type on the given node for the given cache and table name.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @return Type.
+     */
+    @Nullable protected static QueryTypeDescriptorImpl type(IgniteEx node, 
String cacheName, String tblName) {
+        return types(node, cacheName).get(tblName);
+    }
+
+    /**
+     * Get available types on the given node for the given cache.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @return Map from table name to type.
+     */
+    protected static Map<String, QueryTypeDescriptorImpl> types(IgniteEx node, 
String cacheName) {
+        Map<String, QueryTypeDescriptorImpl> res = new HashMap<>();
+
+        Collection<GridQueryTypeDescriptor> descs = 
node.context().query().types(cacheName);
+
+        for (GridQueryTypeDescriptor desc : descs) {
+            QueryTypeDescriptorImpl desc0 = (QueryTypeDescriptorImpl)desc;
+
+            res.put(desc0.tableName(), desc0);
+        }
+
+        return res;
+    }
+
+    /**
+     * Assert index state on all nodes.
+     *
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @param idxName Index name.
+     * @param fields Fields.
+     */
+    protected static void assertIndex(String cacheName, String tblName, String 
idxName,
+        IgniteBiTuple<String, Boolean>... fields) {
+        for (Ignite node : Ignition.allGrids())
+            assertIndex((IgniteEx)node, cacheName, tblName, idxName, fields);
+    }
+
+    /**
+     * Assert index state on particular node.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @param idxName Index name.
+     * @param fields Fields.
+     */
+    protected static void assertIndex(IgniteEx node, String cacheName, String 
tblName, String idxName,
+        IgniteBiTuple<String, Boolean>... fields) {
+        assertIndexDescriptor(node, cacheName, tblName, idxName, fields);
+
+        if (affinityNode(node, cacheName)) {
+            QueryTypeDescriptorImpl typeDesc = typeExisting(node, cacheName, 
tblName);
+
+            assertIndex(typeDesc, idxName, fields);
+        }
+    }
+
+    /**
+     * Make sure index exists in cache descriptor.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @param idxName Index name.
+     * @param fields Fields.
+     */
+    protected static void assertIndexDescriptor(IgniteEx node, String 
cacheName, String tblName, String idxName,
+        IgniteBiTuple<String, Boolean>... fields) {
+        awaitCompletion();
+
+        DynamicCacheDescriptor desc = 
node.context().cache().cacheDescriptor(cacheName);
+
+        assert desc != null;
+
+        for (QueryEntity entity : desc.schema().entities()) {
+            if (F.eq(tblName, QueryUtils.tableName(entity))) {
+                for (QueryIndex idx : entity.getIndexes()) {
+                    if (F.eq(QueryUtils.indexName(entity, idx), idxName)) {
+                        LinkedHashMap<String, Boolean> idxFields = 
idx.getFields();
+
+                        assertEquals(idxFields.size(), fields.length);
+
+                        int i = 0;
+
+                        for (String idxField : idxFields.keySet()) {
+                            assertEquals(idxField, fields[i].get1());
+                            assertEquals(idxFields.get(idxField), 
fields[i].get2());
+
+                            i++;
+                        }
+
+                        return;
+                    }
+                }
+            }
+        }
+
+        fail("Index not found [cacheName=" + cacheName + ", tlbName=" + 
tblName + ", idxName=" + idxName + ']');
+    }
+
+    /**
+     * Assert index state.
+     *
+     * @param typeDesc Type descriptor.
+     * @param idxName Index name.
+     * @param fields Fields (order is important).
+     */
+    protected static void assertIndex(QueryTypeDescriptorImpl typeDesc, String 
idxName,
+        IgniteBiTuple<String, Boolean>... fields) {
+        QueryIndexDescriptorImpl idxDesc = typeDesc.index(idxName);
+
+        assertNotNull(idxDesc);
+
+        assertEquals(idxName, idxDesc.name());
+        assertEquals(typeDesc, idxDesc.typeDescriptor());
+        assertEquals(QueryIndexType.SORTED, idxDesc.type());
+
+        List<String> fieldNames = new ArrayList<>(idxDesc.fields());
+
+        assertEquals(fields.length, fieldNames.size());
+
+        for (int i = 0; i < fields.length; i++) {
+            String expFieldName = fields[i].get1();
+            boolean expFieldAsc = fields[i].get2();
+
+            assertEquals("Index field mismatch [pos=" + i + ", expField=" + 
expFieldName +
+                ", actualField=" + fieldNames.get(i) + ']', expFieldName, 
fieldNames.get(i));
+
+            boolean fieldAsc = !idxDesc.descending(expFieldName);
+
+            assertEquals("Index field sort mismatch [pos=" + i + ", field=" + 
expFieldName +
+                ", expAsc=" + expFieldAsc + ", actualAsc=" + fieldAsc + ']', 
expFieldAsc, fieldAsc);
+        }
+    }
+
+    /**
+     * Assert index doesn't exist on all nodes.
+     *
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @param idxName Index name.
+     */
+    protected static void assertNoIndex(String cacheName, String tblName, 
String idxName) {
+        for (Ignite node : Ignition.allGrids())
+            assertNoIndex((IgniteEx)node, cacheName, tblName, idxName);
+    }
+
+    /**
+     * Assert index doesn't exist on particular node.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @param idxName Index name.
+     */
+    protected static void assertNoIndex(IgniteEx node, String cacheName, 
String tblName, String idxName) {
+        assertNoIndexDescriptor(node, cacheName, tblName, idxName);
+
+        if (affinityNode(node, cacheName)) {
+            QueryTypeDescriptorImpl typeDesc = typeExisting(node, cacheName, 
tblName);
+
+            assertNoIndex(typeDesc, idxName);
+        }
+    }
+
+    /**
+     * Assert index doesn't exist in particular node's cache descriptor.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @param tblName Table name.
+     * @param idxName Index name.
+     */
+    protected static void assertNoIndexDescriptor(IgniteEx node, String 
cacheName, String tblName, String idxName) {
+        awaitCompletion();
+
+        DynamicCacheDescriptor desc = 
node.context().cache().cacheDescriptor(cacheName);
+
+        if (desc == null)
+            return;
+
+        for (QueryEntity entity : desc.schema().entities()) {
+            for (QueryIndex idx : entity.getIndexes()) {
+                if (F.eq(idxName, QueryUtils.indexName(entity, idx)))
+                    fail("Index exists: " + idxName);
+            }
+        }
+    }
+
+    /**
+     * Await completion (hopefully) of pending operations.
+     */
+    private static void awaitCompletion() {
+        try {
+            U.sleep(100);
+        }
+        catch (IgniteInterruptedCheckedException e) {
+            fail();
+        }
+    }
+
+    /**
+     * Assert index doesn't exist.
+     *
+     * @param typeDesc Type descriptor.
+     * @param idxName Index name.
+     */
+    protected static void assertNoIndex(QueryTypeDescriptorImpl typeDesc, 
String idxName) {
+        assertNull(typeDesc.index(idxName));
+    }
+
+    /**
+     * Check whether this is affinity node for cache.
+     *
+     * @param node Node.
+     * @param cacheName Cache name.
+     * @return {@code True} if affinity node.
+     */
+    private static boolean affinityNode(IgniteEx node, String cacheName) {
+        if (node.configuration().isClientMode())
+            return false;
+
+        DynamicCacheDescriptor cacheDesc = 
node.context().cache().cacheDescriptor(cacheName);
+
+        IgnitePredicate<ClusterNode> filter = 
cacheDesc.cacheConfiguration().getNodeFilter();
+
+        return filter == null || filter.apply(node.localNode());
+    }
+
+    /**
+     * Get table name for class.
+     *
+     * @param cls Class.
+     * @return Table name.
+     */
+    protected static String tableName(Class cls) {
+        return cls.getSimpleName();
+    }
+
+    /**
+     * Convenient method for index creation.
+     *
+     * @param name Name.
+     * @param fields Fields.
+     * @return Index.
+     */
+    protected static QueryIndex index(String name, IgniteBiTuple<String, 
Boolean>... fields) {
+        QueryIndex idx = new QueryIndex();
+
+        idx.setName(name);
+
+        LinkedHashMap<String, Boolean> fields0 = new LinkedHashMap<>();
+
+        for (IgniteBiTuple<String, Boolean> field : fields)
+            fields0.put(field.getKey(), field.getValue());
+
+        idx.setFields(fields0);
+
+        return idx;
+    }
+
+    /**
+     * Get query processor.
+     *
+     * @param node Node.
+     * @return Query processor.
+     */
+    protected static GridQueryProcessor queryProcessor(Ignite node) {
+        return ((IgniteEx)node).context().query();
+    }
+
+    /**
+     * Field for index state check (ascending).
+     *
+     * @param name Name.
+     * @return Field.
+     */
+    protected static IgniteBiTuple<String, Boolean> field(String name) {
+        return field(name, true);
+    }
+
+    /**
+     * Field for index state check.
+     *
+     * @param name Name.
+     * @param asc Ascending flag.
+     * @return Field.
+     */
+    protected static IgniteBiTuple<String, Boolean> field(String name, boolean 
asc) {
+        return F.t(name, asc);
+    }
+
+    /**
+     * @param fieldName Field name.
+     * @return Alias.
+     */
+    protected static String alias(String fieldName) {
+        return fieldName + "_alias";
+    }
+
+    /**
+     * Key class.
+     */
+    public static class KeyClass {
+        /** ID. */
+        @QuerySqlField
+        private long id;
+
+        /**
+         * Constructor.
+         *
+         * @param id ID.
+         */
+        public KeyClass(long id) {
+            this.id = id;
+        }
+
+        /**
+         * @return ID.
+         */
+        public long id() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            KeyClass keyClass = (KeyClass) o;
+
+            return id == keyClass.id;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return (int) (id ^ (id >>> 32));
+        }
+    }
+
+    /**
+     * Key class.
+     */
+    public static class ValueClass {
+        /** Field 1. */
+        @QuerySqlField
+        private String field1;
+
+        /**
+         * Constructor.
+         *
+         * @param field1 Field 1.
+         */
+        public ValueClass(String field1) {
+            this.field1 = field1;
+        }
+
+        /**
+         * @return Field 1
+         */
+        public String field1() {
+            return field1;
+        }
+    }
+
+    /**
+     * Key class.
+     */
+    public static class ValueClass2 {
+        /** Field 1. */
+        @QuerySqlField(name = "field1")
+        private String field;
+
+        /**
+         * Constructor.
+         *
+         * @param field Field 1.
+         */
+        public ValueClass2(String field) {
+            this.field = field;
+        }
+
+        /**
+         * @return Field 1
+         */
+        public String field() {
+            return field;
+        }
+    }
+
+    /**
+     * Runnable which can throw checked exceptions.
+     */
+    protected interface RunnableX {
+        /**
+         * Do run.
+         *
+         * @throws Exception If failed.
+         */
+        public void run() throws Exception;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
new file mode 100644
index 0000000..6bc1576
--- /dev/null
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
@@ -0,0 +1,950 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import 
org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ * Tests for dynamic index creation.
+ */
+@SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
+public abstract class DynamicIndexAbstractBasicSelfTest extends 
DynamicIndexAbstractSelfTest {
+    /** Node index for regular server (coordinator). */
+    protected static final int IDX_SRV_CRD = 0;
+
+    /** Node index for regular server (not coordinator). */
+    protected static final int IDX_SRV_NON_CRD = 1;
+
+    /** Node index for regular client. */
+    protected static final int IDX_CLI = 2;
+
+    /** Node index for server which doesn't pass node filter. */
+    protected static final int IDX_SRV_FILTERED = 3;
+
+    /** Node index for client with near-only cache. */
+    protected static final int IDX_CLI_NEAR_ONLY = 4;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        for (IgniteConfiguration cfg : configurations())
+            Ignition.start(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        node().destroyCache(CACHE_NAME);
+
+        super.afterTest();
+    }
+
+    /**
+     * Initialize cache for tests.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     */
+    private void initialize(CacheMode mode, CacheAtomicityMode atomicityMode, 
boolean near) {
+        node().getOrCreateCache(cacheConfiguration(mode, atomicityMode, near));
+
+        grid(IDX_CLI_NEAR_ONLY).getOrCreateNearCache(CACHE_NAME, new 
NearCacheConfiguration<>());
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+
+        loadInitialData();
+    }
+
+    /**
+     * Create cache with the given cache mode and atomicity mode.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Whether near cache should be initialized.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<KeyClass, ValueClass> 
cacheConfiguration(CacheMode mode,
+        CacheAtomicityMode atomicityMode, boolean near) {
+        CacheConfiguration<KeyClass, ValueClass> ccfg = cacheConfiguration();
+
+        ccfg.setCacheMode(mode);
+        ccfg.setAtomicityMode(atomicityMode);
+
+        if (near)
+            ccfg.setNearConfiguration(new NearCacheConfiguration<KeyClass, 
ValueClass>());
+
+        return ccfg;
+    }
+
+    /**
+     * Load initial data.
+     */
+    private void loadInitialData() {
+        put(node(), 0, KEY_BEFORE);
+    }
+
+    /**
+     * Test simple index create for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreatePartitionedAtomic() throws Exception {
+        checkCreate(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test simple index create for PARTITIONED ATOMIC cache with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreatePartitionedAtomicNear() throws Exception {
+        checkCreate(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test simple index create for PARTITIONED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreatePartitionedTransactional() throws Exception {
+        checkCreate(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test simple index create for PARTITIONED TRANSACTIONAL cache with near 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreatePartitionedTransactionalNear() throws Exception {
+        checkCreate(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test simple index create for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateReplicatedAtomic() throws Exception {
+        checkCreate(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test simple index create for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateReplicatedTransactional() throws Exception {
+        checkCreate(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check normal create operation.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkCreate(CacheMode mode, CacheAtomicityMode atomicityMode, 
boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        final IgniteEx node = node();
+
+        final QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1));
+
+        queryProcessor(node).dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx, 
false).get();
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1, field(FIELD_NAME_1));
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node).dynamicIndexCreate(CACHE_NAME, TBL_NAME, 
idx, false).get();
+            }
+        }, SchemaOperationException.CODE_INDEX_EXISTS);
+
+        queryProcessor(node).dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx, 
true).get();
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1, field(FIELD_NAME_1));
+
+        assertSimpleIndexOperations(SQL_SIMPLE_FIELD_1);
+
+        assertIndexUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_1, SQL_ARG_1);
+    }
+
+    /**
+     * Test composite index creation for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateCompositePartitionedAtomic() throws Exception {
+        checkCreateComposite(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test composite index creation for PARTITIONED ATOMIC cache with near 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateCompositePartitionedAtomicNear() throws Exception {
+        checkCreateComposite(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test composite index creation for PARTITIONED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateCompositePartitionedTransactional() throws Exception 
{
+        checkCreateComposite(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test composite index creation for PARTITIONED TRANSACTIONAL cache with 
near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateCompositePartitionedTransactionalNear() throws 
Exception {
+        checkCreateComposite(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test composite index creation for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateCompositeReplicatedAtomic() throws Exception {
+        checkCreateComposite(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test composite index creation for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateCompositeReplicatedTransactional() throws Exception {
+        checkCreateComposite(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check composite index creation.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkCreateComposite(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        final QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1), 
field(alias(FIELD_NAME_2)));
+
+        queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx, 
false).get();
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1, field(FIELD_NAME_1), 
field(alias(FIELD_NAME_2)));
+
+        assertCompositeIndexOperations(SQL_COMPOSITE);
+
+        assertIndexUsed(IDX_NAME_1, SQL_COMPOSITE, SQL_ARG_1, SQL_ARG_2);
+    }
+
+    /**
+     * Test create when cache doesn't exist for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoCachePartitionedAtomic() throws Exception {
+        checkCreateNotCache(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test create when cache doesn't exist for PARTITIONED ATOMIC cache with 
near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoCachePartitionedAtomicNear() throws Exception {
+        checkCreateNotCache(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test create when cache doesn't exist for PARTITIONED TRANSACTIONAL 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoCachePartitionedTransactional() throws Exception {
+        checkCreateNotCache(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test create when cache doesn't exist for PARTITIONED TRANSACTIONAL 
cache with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoCachePartitionedTransactionalNear() throws 
Exception {
+        checkCreateNotCache(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test create when cache doesn't exist for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoCacheReplicatedAtomic() throws Exception {
+        checkCreateNotCache(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test create when cache doesn't exist for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoCacheReplicatedTransactional() throws Exception {
+        checkCreateNotCache(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check create when cache doesn't exist.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkCreateNotCache(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        final QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1));
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexCreate(randomString(), 
TBL_NAME, idx, false).get();
+            }
+        }, SchemaOperationException.CODE_CACHE_NOT_FOUND);
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoTablePartitionedAtomic() throws Exception {
+        checkCreateNoTable(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED ATOMIC cache with 
near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoTablePartitionedAtomicNear() throws Exception {
+        checkCreateNoTable(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED TRANSACTIONAL 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoTablePartitionedTransactional() throws Exception {
+        checkCreateNoTable(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED TRANSACTIONAL 
cache with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoTablePartitionedTransactionalNear() throws 
Exception {
+        checkCreateNoTable(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test create when table doesn't exist for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoTableReplicatedAtomic() throws Exception {
+        checkCreateNoTable(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test create when table doesn't exist for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoTableReplicatedTransactional() throws Exception {
+        checkCreateNoTable(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check create when table doesn't exist.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkCreateNoTable(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        final QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1));
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, 
randomString(), idx, false).get();
+            }
+        }, SchemaOperationException.CODE_TABLE_NOT_FOUND);
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoColumnPartitionedAtomic() throws Exception {
+        checkCreateNoColumn(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED ATOMIC cache with 
near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoColumnPartitionedAtomicNear() throws Exception {
+        checkCreateNoColumn(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED TRANSACTIONAL 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoColumnPartitionedTransactional() throws Exception {
+        checkCreateNoColumn(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test create when table doesn't exist for PARTITIONED TRANSACTIONAL 
cache with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoColumnPartitionedTransactionalNear() throws 
Exception {
+        checkCreateNoColumn(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test create when table doesn't exist for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoColumnReplicatedAtomic() throws Exception {
+        checkCreateNoColumn(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test create when table doesn't exist for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateNoColumnReplicatedTransactional() throws Exception {
+        checkCreateNoColumn(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check create when table doesn't exist.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkCreateNoColumn(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        final QueryIndex idx = index(IDX_NAME_1, field(randomString()));
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, 
TBL_NAME, idx, false).get();
+            }
+        }, SchemaOperationException.CODE_COLUMN_NOT_FOUND);
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+    }
+
+    /**
+     * Test index creation on aliased column for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateColumnWithAliasPartitionedAtomic() throws Exception {
+        checkCreateColumnWithAlias(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test index creation on aliased column for PARTITIONED ATOMIC cache with 
near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateColumnWithAliasPartitionedAtomicNear() throws 
Exception {
+        checkCreateColumnWithAlias(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test index creation on aliased column for PARTITIONED TRANSACTIONAL 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateColumnWithAliasPartitionedTransactional() throws 
Exception {
+        checkCreateColumnWithAlias(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test index creation on aliased column for PARTITIONED TRANSACTIONAL 
cache with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateColumnWithAliasPartitionedTransactionalNear() throws 
Exception {
+        checkCreateColumnWithAlias(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test index creation on aliased column for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateColumnWithAliasReplicatedAtomic() throws Exception {
+        checkCreateColumnWithAlias(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test index creation on aliased column for REPLICATED TRANSACTIONAL 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateColumnWithAliasReplicatedTransactional() throws 
Exception {
+        checkCreateColumnWithAlias(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check index creation on aliased column.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkCreateColumnWithAlias(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near)
+        throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_2));
+
+                queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, 
TBL_NAME, idx, false).get();
+            }
+        }, SchemaOperationException.CODE_COLUMN_NOT_FOUND);
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+
+        QueryIndex idx = index(IDX_NAME_1, field(alias(FIELD_NAME_2)));
+
+        queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx, 
false).get();
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1, 
field(alias(FIELD_NAME_2)));
+
+        assertSimpleIndexOperations(SQL_SIMPLE_FIELD_2);
+
+        assertIndexUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_2, SQL_ARG_1);
+    }
+
+    /**
+     * Test simple index drop for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropPartitionedAtomic() throws Exception {
+        checkDrop(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test simple index drop for PARTITIONED ATOMIC cache with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropPartitionedAtomicNear() throws Exception {
+        checkDrop(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test simple index drop for PARTITIONED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropPartitionedTransactional() throws Exception {
+        checkDrop(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test simple index drop for PARTITIONED TRANSACTIONAL cache with near 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropPartitionedTransactionalNear() throws Exception {
+        checkDrop(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test simple index drop for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropReplicatedAtomic() throws Exception {
+        checkDrop(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test simple index drop for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropReplicatedTransactional() throws Exception {
+        checkDrop(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check simple index drop.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    public void checkDrop(CacheMode mode, CacheAtomicityMode atomicityMode, 
boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1));
+
+        queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx, 
false).get();
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1, field(FIELD_NAME_1));
+
+        assertIndexUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_1, SQL_ARG_1);
+
+        assertSimpleIndexOperations(SQL_SIMPLE_FIELD_1);
+
+        loadInitialData();
+
+        queryProcessor(node()).dynamicIndexDrop(CACHE_NAME, IDX_NAME_1, 
false).get();
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+
+        assertSimpleIndexOperations(SQL_SIMPLE_FIELD_1);
+
+        assertIndexNotUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_1, SQL_ARG_1);
+    }
+
+    /**
+     * Test drop when there is no index for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoIndexPartitionedAtomic() throws Exception {
+        checkDropNoIndex(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test drop when there is no index for PARTITIONED ATOMIC cache with near 
cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoIndexPartitionedAtomicNear() throws Exception {
+        checkDropNoIndex(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test drop when there is no index for PARTITIONED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoIndexPartitionedTransactional() throws Exception {
+        checkDropNoIndex(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test drop when there is no index for PARTITIONED TRANSACTIONAL cache 
with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoIndexPartitionedTransactionalNear() throws Exception 
{
+        checkDropNoIndex(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test drop when there is no index for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoIndexReplicatedAtomic() throws Exception {
+        checkDropNoIndex(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test drop when there is no index for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoIndexReplicatedTransactional() throws Exception {
+        checkDropNoIndex(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check drop when there is no index.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkDropNoIndex(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexDrop(CACHE_NAME, 
IDX_NAME_1, false).get();
+            }
+        }, SchemaOperationException.CODE_INDEX_NOT_FOUND);
+
+        queryProcessor(node()).dynamicIndexDrop(CACHE_NAME, IDX_NAME_1, 
true).get();
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+    }
+
+    /**
+     * Test drop when cache doesn't exist for PARTITIONED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoCachePartitionedAtomic() throws Exception {
+        checkDropNoCache(PARTITIONED, ATOMIC, false);
+    }
+
+    /**
+     * Test drop when cache doesn't exist for PARTITIONED ATOMIC cache with 
near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoCachePartitionedAtomicNear() throws Exception {
+        checkDropNoCache(PARTITIONED, ATOMIC, true);
+    }
+
+    /**
+     * Test drop when cache doesn't exist for PARTITIONED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoCachePartitionedTransactional() throws Exception {
+        checkDropNoCache(PARTITIONED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Test drop when cache doesn't exist for PARTITIONED TRANSACTIONAL cache 
with near cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoCachePartitionedTransactionalNear() throws Exception 
{
+        checkDropNoCache(PARTITIONED, TRANSACTIONAL, true);
+    }
+
+    /**
+     * Test drop when cache doesn't exist for REPLICATED ATOMIC cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoCacheReplicatedAtomic() throws Exception {
+        checkDropNoCache(REPLICATED, ATOMIC, false);
+    }
+
+    /**
+     * Test drop when cache doesn't exist for REPLICATED TRANSACTIONAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropNoCacheReplicatedTransactional() throws Exception {
+        checkDropNoCache(REPLICATED, TRANSACTIONAL, false);
+    }
+
+    /**
+     * Check drop when cache doesn't exist.
+     *
+     * Check drop when there is no index.
+     *
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near flag.
+     * @throws Exception If failed.
+     */
+    private void checkDropNoCache(CacheMode mode, CacheAtomicityMode 
atomicityMode, boolean near) throws Exception {
+        initialize(mode, atomicityMode, near);
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexDrop(randomString(), 
"my_idx", false).get();
+            }
+        }, SchemaOperationException.CODE_CACHE_NOT_FOUND);
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+    }
+
+    /**
+     * Test that operations fail on LOCAL cache.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFailOnLocalCache() throws Exception {
+        for (Ignite node : Ignition.allGrids()) {
+            if (!node.configuration().isClientMode())
+                
node.getOrCreateCache(cacheConfiguration().setCacheMode(LOCAL));
+        }
+
+        final QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1));
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexCreate(CACHE_NAME, 
TBL_NAME, idx, true).get();
+            }
+        }, SchemaOperationException.CODE_GENERIC);
+
+        assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
+
+        assertSchemaException(new RunnableX() {
+            @Override public void run() throws Exception {
+                queryProcessor(node()).dynamicIndexDrop(CACHE_NAME, 
IDX_NAME_1, true).get();
+            }
+        }, SchemaOperationException.CODE_GENERIC);
+    }
+
+    /**
+     * Get node which should be used to start operations.
+     *
+     * @return If failed.
+     */
+    protected IgniteEx node() {
+        return grid(nodeIndex());
+    }
+
+    /**
+     * Get index of the node which should be used to start operations.
+     *
+     * @return If failed.
+     */
+    protected abstract int nodeIndex();
+
+    /**
+     * Get configurations to be used in test.
+     *
+     * @return Configurations.
+     * @throws Exception If failed.
+     */
+    protected List<IgniteConfiguration> configurations() throws Exception {
+        return Arrays.asList(
+            serverCoordinatorConfiguration(IDX_SRV_CRD),
+            serverConfiguration(IDX_SRV_NON_CRD),
+            clientConfiguration(IDX_CLI),
+            serverConfiguration(IDX_SRV_FILTERED, true),
+            clientConfiguration(IDX_CLI_NEAR_ONLY)
+        );
+    }
+
+    /**
+     * Get server coordinator configuration.
+     *
+     * @param idx Index.
+     * @return Configuration.
+     * @throws Exception If failed.
+     */
+    protected IgniteConfiguration serverCoordinatorConfiguration(int idx) 
throws Exception {
+        return serverConfiguration(idx);
+    }
+
+    /**
+     * Assert FIELD_1 index usage.
+     *
+     * @param sql Simple SQL.
+     */
+    private void assertSimpleIndexOperations(String sql) {
+        for (Ignite node : Ignition.allGrids())
+            assertSqlSimpleData(node, sql, KEY_BEFORE - SQL_ARG_1);
+
+        put(node(), KEY_BEFORE, KEY_AFTER);
+
+        for (Ignite node : Ignition.allGrids())
+            assertSqlSimpleData(node, sql, KEY_AFTER - SQL_ARG_1);
+
+        remove(node(), 0, KEY_BEFORE);
+
+        for (Ignite node : Ignition.allGrids())
+            assertSqlSimpleData(node, sql, KEY_AFTER - KEY_BEFORE);
+
+        remove(node(), KEY_BEFORE, KEY_AFTER);
+
+        for (Ignite node : Ignition.allGrids())
+            assertSqlSimpleData(node, sql, 0);
+    }
+
+    /**
+     * Assert composite index usage.
+     *
+     * @param sql Simple SQL.
+     */
+    private void assertCompositeIndexOperations(String sql) {
+        for (Ignite node : Ignition.allGrids())
+            assertSqlCompositeData(node, sql, KEY_BEFORE - SQL_ARG_2);
+
+        put(node(), KEY_BEFORE, KEY_AFTER);
+
+        for (Ignite node : Ignition.allGrids())
+            assertSqlCompositeData(node, sql, KEY_AFTER - SQL_ARG_2);
+
+        remove(node(), 0, KEY_BEFORE);
+
+        for (Ignite node : Ignition.allGrids())
+            assertSqlCompositeData(node, sql, KEY_AFTER - KEY_BEFORE);
+
+        remove(node(), KEY_BEFORE, KEY_AFTER);
+
+        for (Ignite node : Ignition.allGrids())
+            assertSqlCompositeData(node, sql, 0);
+    }
+}

Reply via email to