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/04/19 08:46:22 UTC

[41/50] [abbrv] ignite git commit: IGNITE-4565: Implemented CREATE INDEX and DROP INDEX. This closes #1773. This closes #1804.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaIndexDropOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaIndexDropOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaIndexDropOperation.java
new file mode 100644
index 0000000..da60560
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaIndexDropOperation.java
@@ -0,0 +1,68 @@
+/*
+ * 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 org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.UUID;
+
+/**
+ * Schema index drop operation.
+ */
+public class SchemaIndexDropOperation extends SchemaIndexAbstractOperation {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index name. */
+    private final String idxName;
+
+    /** Ignore operation if index doesn't exist. */
+    private final boolean ifExists;
+
+    /**
+     * Constructor.
+     *
+     * @param opId Operation id.
+     * @param space Space.
+     * @param idxName Index name.
+     * @param ifExists Ignore operation if index doesn't exist.
+     */
+    public SchemaIndexDropOperation(UUID opId, String space, String idxName, boolean ifExists) {
+        super(opId, space);
+
+        this.idxName = idxName;
+        this.ifExists = ifExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String indexName() {
+        return idxName;
+    }
+
+    /**
+     * @return Ignore operation if index doesn't exist.
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SchemaIndexDropOperation.class, this, "parent", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 1545b8c..a060f7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
 import org.apache.ignite.internal.processors.dr.GridDrType;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.jetbrains.annotations.Nullable;
@@ -853,6 +854,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** {@inheritDoc} */
+    @Override public void updateIndex(SchemaIndexCacheVisitorClosure clo, long link) throws IgniteCheckedException,
+        GridCacheEntryRemovedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean deleted() {
         return false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 77d0ea7..ff67b77 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -66,6 +66,7 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 null,
                 null,
                 null,
+                null,
                 U.allPluginProviders()
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
index b53387f..4404e9c 100644
--- a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
+++ b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
@@ -101,7 +101,7 @@ public class GridH2IndexingGeoSelfTest extends GridCacheAbstractSelfTest {
             List<List<?>> res = cache.query(new SqlFieldsQuery("explain select _key from Geometry where _val && ?")
                 .setArgs(r.read("POLYGON((5 70, 5 80, 30 80, 30 70, 5 70))")).setLocal(true)).getAll();
 
-            assertTrue("__ explain: " + res, res.get(0).get(0).toString().contains("_val_idx"));
+            assertTrue("__ explain: " + res, res.get(0).get(0).toString().toLowerCase().contains("_val_idx"));
         }
         finally {
             cache.destroy();
@@ -167,7 +167,7 @@ public class GridH2IndexingGeoSelfTest extends GridCacheAbstractSelfTest {
 
             // Check explaint request.
             assertTrue(F.first(cache.query(new SqlFieldsQuery("explain select * from EnemyCamp " +
-                "where coords && 'POINT(25 75)'")).getAll()).get(0).toString().contains("coords_idx"));
+                "where coords && 'POINT(25 75)'")).getAll()).get(0).toString().toLowerCase().contains("coords_idx"));
         }
         finally {
             cache.destroy();

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 7129691..e8dc73b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -41,9 +41,11 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.CacheOperationContext;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -51,7 +53,6 @@ import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
-import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
@@ -73,14 +74,18 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.command.Prepared;
+import org.h2.command.dml.Delete;
+import org.h2.command.dml.Insert;
+import org.h2.command.dml.Merge;
+import org.h2.command.dml.Update;
 import org.h2.table.Column;
 import org.h2.value.DataType;
 import org.h2.value.Value;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.createJdbcSqlException;
+import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
 
 /**
  *
@@ -90,11 +95,10 @@ public class DmlStatementsProcessor {
     private final static int DFLT_DML_RERUN_ATTEMPTS = 4;
 
     /** Indexing. */
-    private final IgniteH2Indexing indexing;
+    private IgniteH2Indexing idx;
 
-    /** Set of binary type ids for which warning about missing identity in configuration has been printed. */
-    private final static Set<Integer> WARNED_TYPES =
-        Collections.newSetFromMap(new ConcurrentHashMap8<Integer, Boolean>());
+    /** Logger. */
+    private IgniteLogger log;
 
     /** Default size for update plan cache. */
     private static final int PLAN_CACHE_SIZE = 1024;
@@ -102,15 +106,16 @@ public class DmlStatementsProcessor {
     /** Update plans cache. */
     private final ConcurrentMap<String, ConcurrentMap<String, UpdatePlan>> planCache = new ConcurrentHashMap<>();
 
-    /** Dummy metadata for update result. */
-    private final static List<GridQueryFieldMetadata> UPDATE_RESULT_META = Collections.<GridQueryFieldMetadata>
-        singletonList(new IgniteH2Indexing.SqlFieldMetadata(null, null, "UPDATED", Long.class.getName()));
-
     /**
-     * @param indexing indexing.
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     * @param idx indexing.
      */
-    DmlStatementsProcessor(IgniteH2Indexing indexing) {
-        this.indexing = indexing;
+    public void start(GridKernalContext ctx, IgniteH2Indexing idx) {
+        this.idx = idx;
+
+        log = ctx.log(DmlStatementsProcessor.class);
     }
 
     /**
@@ -251,7 +256,7 @@ public class DmlStatementsProcessor {
 
             final ArrayList<List<?>> data = new ArrayList<>(plan.rowsNum);
 
-            final GridQueryFieldsResult res = indexing.queryLocalSqlFields(cctx.name(), plan.selectQry,
+            final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,
                 F.asList(args), null, false, 0, null);
 
             QueryCursorImpl<List<?>> stepCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
@@ -335,10 +340,10 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>) indexing.queryTwoStep(cctx, newFieldsQry, cancel);
+            cur = (QueryCursorImpl<List<?>>) idx.queryTwoStep(cctx, newFieldsQry, cancel);
         }
         else {
-            final GridQueryFieldsResult res = indexing.queryLocalSqlFields(cctx.name(), plan.selectQry,
+            final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,
                 F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);
 
             cur = new QueryCursorImpl<>(new Iterable<List<?>>() {
@@ -476,7 +481,7 @@ public class DmlStatementsProcessor {
         while (it.hasNext()) {
             List<?> e = it.next();
             if (e.size() != 2) {
-                U.warn(indexing.getLogger(), "Invalid row size on DELETE - expected 2, got " + e.size());
+                U.warn(log, "Invalid row size on DELETE - expected 2, got " + e.size());
                 continue;
             }
 
@@ -1069,6 +1074,16 @@ public class DmlStatementsProcessor {
         }
     }
 
+    /**
+     * Check whether statement is DML statement.
+     *
+     * @param stmt Statement.
+     * @return {@code True} if this is DML.
+     */
+    static boolean isDmlStatement(Prepared stmt) {
+        return stmt instanceof Merge || stmt instanceof Insert || stmt instanceof Update || stmt instanceof Delete;
+    }
+
     /** Update result - modifications count and keys to re-run query with, if needed. */
     private final static class UpdateResult {
         /** Result to return for operations that affected 1 item - mostly to be used for fast updates and deletes. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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 531b760..4f0a9f9 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
@@ -78,10 +78,10 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -100,6 +100,9 @@ import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl;
+import org.apache.ignite.internal.processors.query.h2.ddl.DdlStatementsProcessor;
+import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode;
 import org.apache.ignite.internal.processors.query.h2.database.H2PkHashIndex;
 import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
 import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
@@ -107,8 +110,9 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasInnerI
 import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasLeafIO;
 import org.apache.ignite.internal.processors.query.h2.database.io.H2InnerIO;
 import org.apache.ignite.internal.processors.query.h2.database.io.H2LeafIO;
-import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2SystemIndexFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
@@ -116,17 +120,19 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2TreeIndex;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.processors.query.h2.opt.GridLuceneIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.lang.GridPlainRunnable;
 import org.apache.ignite.internal.util.lang.IgniteInClosure2X;
@@ -150,14 +156,15 @@ import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
+import org.h2.api.TableEngine;
 import org.h2.command.CommandInterface;
 import org.h2.command.Prepared;
+import org.h2.command.ddl.CreateTableData;
 import org.h2.command.dml.Insert;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Cursor;
 import org.h2.index.Index;
-import org.h2.index.SpatialIndex;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.jdbc.JdbcStatement;
@@ -167,6 +174,7 @@ import org.h2.result.SortOrder;
 import org.h2.server.web.WebServer;
 import org.h2.table.Column;
 import org.h2.table.IndexColumn;
+import org.h2.table.TableBase;
 import org.h2.tools.Server;
 import org.h2.util.JdbcUtils;
 import org.h2.value.DataType;
@@ -221,8 +229,7 @@ import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType
  */
 @SuppressWarnings({"UnnecessaryFullyQualifiedName", "NonFinalStaticVariableUsedInClassInitialization"})
 public class IgniteH2Indexing implements GridQueryIndexing {
-
-    /**
+    /*
      * Register IO for indexes.
      */
     static {
@@ -231,6 +238,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         H2ExtrasLeafIO.register();
     }
 
+    /** Spatial index class name. */
+    private static final String SPATIAL_IDX_CLS =
+        "org.apache.ignite.internal.processors.query.h2.opt.GridH2SpatialIndex";
+
     /** Default DB options. */
     private static final String DB_OPTIONS = ";LOCK_MODE=3;MULTI_THREADED=1;DB_CLOSE_ON_EXIT=FALSE" +
         ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0" +
@@ -241,6 +252,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         // Uncomment this setting to get debug output from H2 to sysout.
 //        ";TRACE_LEVEL_SYSTEM_OUT=3";
 
+    /** Dummy metadata for update result. */
+    public static final List<GridQueryFieldMetadata> UPDATE_RESULT_META = Collections.<GridQueryFieldMetadata>
+        singletonList(new SqlFieldMetadata(null, null, "UPDATED", Long.class.getName()));
+
     /** */
     private static final int PREPARED_STMT_CACHE_SIZE = 256;
 
@@ -371,10 +386,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     };
 
     /** */
-    private volatile GridKernalContext ctx;
+    protected volatile GridKernalContext ctx;
+
+    /** */
+    private final DmlStatementsProcessor dmlProc = new DmlStatementsProcessor();
 
     /** */
-    private final DmlStatementsProcessor dmlProc = new DmlStatementsProcessor(this);
+    private DdlStatementsProcessor ddlProc;
 
     /** */
     private final ConcurrentMap<String, GridH2Table> dataTables = new ConcurrentHashMap8<>();
@@ -419,13 +437,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @return Logger.
-     */
-    IgniteLogger getLogger() {
-        return log;
-    }
-
-    /**
      * @param c Connection.
      * @param sql SQL.
      * @param useStmtCache If {@code true} uses statement cache.
@@ -468,15 +479,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException {
-        return prepareStatement(connectionForSpace(space(schema)), sql, true);
+    @Override public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException {
+        return prepareStatement(connectionForSpace(space), sql, true);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteDataStreamer<?, ?> createStreamer(String spaceName, PreparedStatement nativeStmt,
         long autoFlushFreq, int nodeBufSize, int nodeParOps, boolean allowOverwrite) {
-        Prepared prep = GridSqlQueryParser.prepared((JdbcPreparedStatement) nativeStmt);
+        Prepared prep = GridSqlQueryParser.prepared(nativeStmt);
 
         if (!(prep instanceof Insert))
             throw new IgniteSQLException("Only INSERT operations are supported in streaming mode",
@@ -625,14 +636,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public void store(@Nullable String spaceName,
-        GridQueryTypeDescriptor type,
+        String typeName,
         KeyCacheObject k,
         int partId,
         CacheObject v,
         GridCacheVersion ver,
         long expirationTime,
         long link) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
 
         if (tbl == null)
             return; // Type was rejected.
@@ -700,7 +711,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']');
 
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+        TableDescriptor tbl = tableDescriptor(type.name(), spaceName);
 
         if (tbl == null)
             return;
@@ -803,12 +814,232 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         tbl.schema.tbls.remove(tbl.typeName());
     }
 
+    /**
+     * Add initial user index.
+     *
+     * @param spaceName Space name.
+     * @param desc Table descriptor.
+     * @param h2Idx User index.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2IndexBase h2Idx)
+        throws IgniteCheckedException {
+        GridH2Table h2Tbl = desc.tbl;
+
+        h2Tbl.proposeUserIndex(h2Idx);
+
+        try {
+            String sql = indexCreateSql(desc.fullTableName(), h2Idx, false, desc.schema.escapeAll());
+
+            executeSql(spaceName, sql);
+        }
+        catch (Exception e) {
+            // Rollback and re-throw.
+            h2Tbl.rollbackUserIndex(h2Idx.getName());
+
+            throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dynamicIndexCreate(@Nullable final String spaceName, final String tblName,
+        final QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor)
+        throws IgniteCheckedException {
+        // Locate table.
+        String schemaName = schema(spaceName);
+
+        Schema schema = schemas.get(schemaName);
+
+        TableDescriptor desc = (schema != null ? schema.tbls.get(tblName) : null);
+
+        if (desc == null)
+            throw new IgniteCheckedException("Table not found in internal H2 database [schemaName=" + schemaName +
+                ", tblName=" + tblName + ']');
+
+        GridH2Table h2Tbl = desc.tbl;
+
+        // Create index.
+        final GridH2IndexBase h2Idx = desc.createUserIndex(idxDesc);
+
+        h2Tbl.proposeUserIndex(h2Idx);
+
+        try {
+            // Populate index with existing cache data.
+            final GridH2RowDescriptor rowDesc = h2Tbl.rowDescriptor();
+
+            SchemaIndexCacheVisitorClosure clo = new SchemaIndexCacheVisitorClosure() {
+                @Override public void apply(KeyCacheObject key, int part, CacheObject val, GridCacheVersion ver,
+                    long expTime, long link) throws IgniteCheckedException {
+                    if (expTime == 0L)
+                        expTime = Long.MAX_VALUE;
+
+                    GridH2Row row = rowDesc.createRow(key, part, val, ver, expTime);
+
+                    row.link(link);
+
+                    h2Idx.put(row);
+                }
+            };
+
+            cacheVisitor.visit(clo);
+
+            // At this point index is in consistent state, promote it through H2 SQL statement, so that cached
+            // prepared statements are re-built.
+            String sql = indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists, schema.escapeAll());
+
+            executeSql(spaceName, sql);
+        }
+        catch (Exception e) {
+            // Rollback and re-throw.
+            h2Tbl.rollbackUserIndex(h2Idx.getName());
+
+            throw e;
+        }
+    }
+
     /** {@inheritDoc} */
+    @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+    @Override public void dynamicIndexDrop(@Nullable final String spaceName, String idxName, boolean ifExists)
+        throws IgniteCheckedException{
+        String schemaName = schema(spaceName);
+
+        Schema schema = schemas.get(schemaName);
+
+        String sql = indexDropSql(schemaName, idxName, ifExists, schema.escapeAll());
+
+        executeSql(spaceName, sql);
+    }
+
+    /**
+     * Execute DDL command.
+     *
+     * @param spaceName Space name.
+     * @param sql SQL.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void executeSql(String spaceName, String sql) throws IgniteCheckedException {
+        try {
+            Connection conn = connectionForSpace(spaceName);
+
+            try (PreparedStatement stmt = prepareStatement(conn, sql, false)) {
+                stmt.execute();
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to execute SQL statement on internal H2 database: " + sql, e);
+        }
+    }
+
+    /**
+     * Generate {@code CREATE INDEX} SQL statement for given params.
+     * @param fullTblName Fully qualified table name.
+     * @param h2Idx H2 index.
+     * @param ifNotExists Quietly skip index creation if it exists.
+     * @return Statement string.
+     */
+    private static String indexCreateSql(String fullTblName, GridH2IndexBase h2Idx, boolean ifNotExists,
+        boolean escapeAll) {
+        boolean spatial = F.eq(SPATIAL_IDX_CLS, h2Idx.getClass().getName());
+
+        GridStringBuilder sb = new SB("CREATE ")
+            .a(spatial ? "SPATIAL " : "")
+            .a("INDEX ")
+            .a(ifNotExists ? "IF NOT EXISTS " : "")
+            .a(escapeName(h2Idx.getName(), escapeAll))
+            .a(" ON ")
+            .a(fullTblName)
+            .a(" (");
+
+        boolean first = true;
+
+        for (IndexColumn col : h2Idx.getIndexColumns()) {
+            if (first)
+                first = false;
+            else
+                sb.a(", ");
+
+            sb.a("\"" + col.columnName + "\"").a(" ").a(col.sortType == SortOrder.ASCENDING ? "ASC" : "DESC");
+        }
+
+        sb.a(')');
+
+        return sb.toString();
+    }
+
+    /**
+     * Generate {@code CREATE INDEX} SQL statement for given params.
+     * @param schemaName <b>Quoted</b> schema name.
+     * @param idxName Index name.
+     * @param ifExists Quietly skip index drop if it exists.
+     * @param escapeAll Escape flag.
+     * @return Statement string.
+     */
+    private static String indexDropSql(String schemaName, String idxName, boolean ifExists, boolean escapeAll) {
+        return "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + schemaName + '.' + escapeName(idxName, escapeAll);
+    }
+
+    /**
+     * Create sorted index.
+     *
+     * @param schema Schema.
+     * @param name Index name,
+     * @param tbl Table.
+     * @param pk Primary key flag.
+     * @param cols Columns.
+     * @return Index.
+     */
+    private GridH2IndexBase createSortedIndex(Schema schema, String name, GridH2Table tbl, boolean pk,
+        List<IndexColumn> cols, int inlineSize) {
+        try {
+            GridCacheContext cctx = schema.cacheContext();
+
+            if (log.isDebugEnabled())
+                log.debug("Creating cache index [cacheId=" + cctx.cacheId() + ", idxName=" + name + ']');
+
+            final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
+
+            return new H2TreeIndex(cctx, tbl, name, pk, cols, inlineSize, segments);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Create spatial index.
+     *
+     * @param tbl Table.
+     * @param idxName Index name.
+     * @param cols Columns.
+     */
+    private GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, IndexColumn[] cols
+    ) {
+        try {
+            Class<?> cls = Class.forName(SPATIAL_IDX_CLS);
+
+            Constructor<?> ctor = cls.getConstructor(
+                GridH2Table.class,
+                String.class,
+                Integer.TYPE,
+                IndexColumn[].class);
+
+            if (!ctor.isAccessible())
+                ctor.setAccessible(true);
+
+            final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
+
+            return (GridH2IndexBase)ctor.newInstance(tbl, idxName, segments, cols);
+        }
+        catch (Exception e) {
+            throw new IgniteException("Failed to instantiate: " + SPATIAL_IDX_CLS, e);
+        }
+    }
+
     @SuppressWarnings("unchecked")
     @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(
-        @Nullable String spaceName, String qry, GridQueryTypeDescriptor type,
+        @Nullable String spaceName, String qry, String typeName,
         IndexingQueryFilter filters) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
 
         if (tbl != null && tbl.luceneIdx != null) {
             GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, spaceName,
@@ -828,9 +1059,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void unregisterType(@Nullable String spaceName, GridQueryTypeDescriptor type)
+    @Override public void unregisterType(@Nullable String spaceName, String typeName)
         throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
 
         if (tbl != null)
             removeTable(tbl);
@@ -860,9 +1091,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
 
-        Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt);
+        Prepared p = GridSqlQueryParser.prepared(stmt);
 
-        if (!p.isQuery()) {
+        if (DmlStatementsProcessor.isDmlStatement(p)) {
             SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
 
             if (params != null)
@@ -873,6 +1104,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filter, cancel);
         }
+        else if (DdlStatementsProcessor.isDdlStatement(p))
+            throw new IgniteSQLException("DDL statements are supported for the whole cluster only",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
         List<GridQueryFieldMetadata> meta;
 
@@ -1419,6 +1653,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 .distributedJoinMode(distributedJoinMode));
 
             PreparedStatement stmt = null;
+            Prepared prepared;
 
             boolean cachesCreated = false;
 
@@ -1434,7 +1669,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         catch (SQLException e) {
                             if (!cachesCreated && e.getErrorCode() == ErrorCode.SCHEMA_NOT_FOUND_1) {
                                 try {
-                                    ctx.cache().createMissingCaches();
+                                    ctx.cache().createMissingQueryCaches();
                                 }
                                 catch (IgniteCheckedException ignored) {
                                     throw new CacheException("Failed to create missing caches.", e);
@@ -1449,7 +1684,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     }
 
 
-                    Prepared prepared = GridSqlQueryParser.prepared(stmt);
+                    prepared = GridSqlQueryParser.prepared(stmt);
 
                     if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery) qry).isQuery() != prepared.isQuery())
                         throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
@@ -1470,12 +1705,23 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                 // It is a DML statement if we did not create a twoStepQuery.
                 if (twoStepQry == null) {
-                    try {
-                        return dmlProc.updateSqlFieldsTwoStep(cctx.namexx(), stmt, qry, cancel);
+                    if (DmlStatementsProcessor.isDmlStatement(prepared)) {
+                        try {
+                            return dmlProc.updateSqlFieldsTwoStep(cctx.namexx(), stmt, qry, cancel);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteSQLException("Failed to execute DML statement [stmt=" + sqlQry +
+                                ", params=" + Arrays.deepToString(qry.getArgs()) + "]", e);
+                        }
                     }
-                    catch (IgniteCheckedException e) {
-                        throw new IgniteSQLException("Failed to execute DML statement [qry=" + sqlQry + ", params=" +
-                            Arrays.deepToString(qry.getArgs()) + "]", e);
+
+                    if (DdlStatementsProcessor.isDdlStatement(prepared)) {
+                        try {
+                            return ddlProc.runDdlStatement(stmt);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sqlQry + ']', e);
+                        }
                     }
                 }
 
@@ -1660,7 +1906,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             Connection conn = connectionForThread(schemaName);
 
-            createTable(schema, tbl, conn);
+            createTable(spaceName, schema, tbl, conn);
 
             schema.add(tbl);
         }
@@ -1754,12 +2000,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Create db table by using given table descriptor.
      *
+     * @param spaceName Space name.
      * @param schema Schema.
      * @param tbl Table descriptor.
      * @param conn Connection.
      * @throws SQLException If failed to create db table.
+     * @throws IgniteCheckedException If failed.
      */
-    private void createTable(Schema schema, TableDescriptor tbl, Connection conn) throws SQLException {
+    private void createTable(String spaceName, Schema schema, TableDescriptor tbl, Connection conn)
+        throws SQLException, IgniteCheckedException {
         assert schema != null;
         assert tbl != null;
 
@@ -1783,12 +2032,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Creating DB table with SQL: " + sql);
 
-        GridH2RowDescriptor desc = new RowDescriptor(tbl.type(), schema);
+        GridH2RowDescriptor rowDesc = new RowDescriptor(tbl.type(), schema);
 
-        GridH2Table res = GridH2Table.Engine.createTable(conn, sql.toString(), desc, tbl, tbl.schema.spaceName);
+        H2RowFactory rowFactory = tbl.rowFactory(rowDesc);
 
-        if (dataTables.putIfAbsent(res.identifier(), res) != null)
-            throw new IllegalStateException("Table already exists: " + res.identifier());
+        GridH2Table h2Tbl = H2TableEngine.createTable(conn, sql.toString(), rowDesc, rowFactory, tbl);
+
+        for (GridH2IndexBase usrIdx : tbl.createUserIndexes())
+            addInitialUserIndex(spaceName, tbl, usrIdx);
+
+        if (dataTables.putIfAbsent(h2Tbl.identifier(), h2Tbl) != null)
+            throw new IllegalStateException("Table already exists: " + h2Tbl.identifier());
     }
 
     /**
@@ -1800,24 +2054,29 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Gets corresponding DB type from java class.
+     * Find table by name in given schema.
      *
-     * @param cls Java class.
-     * @return DB type name.
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @return Table or {@code null} if none found.
      */
-    private String dbTypeFromClass(Class<?> cls) {
-        return DBTypeEnum.fromClass(cls).dBTypeAsString();
+    public GridH2Table dataTable(String schemaName, String tblName) {
+        for (GridH2Table tbl : dataTables.values()) {
+            if (tbl.getSchema().getName().equals(schemaName) && tbl.getName().equals(tblName))
+                return tbl;
+        }
+
+        return null;
     }
 
     /**
-     * Gets table descriptor by value type.
+     * Gets corresponding DB type from java class.
      *
-     * @param spaceName Space name.
-     * @param type Value type descriptor.
-     * @return Table descriptor or {@code null} if not found.
+     * @param cls Java class.
+     * @return DB type name.
      */
-    @Nullable private TableDescriptor tableDescriptor(@Nullable String spaceName, GridQueryTypeDescriptor type) {
-        return tableDescriptor(type.name(), spaceName);
+    private String dbTypeFromClass(Class<?> cls) {
+        return DBTypeEnum.fromClass(cls).dBTypeAsString();
     }
 
     /**
@@ -1903,7 +2162,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     @Override public void rebuildIndexesFromHash(@Nullable String spaceName,
         GridQueryTypeDescriptor type) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+        TableDescriptor tbl = tableDescriptor(type.name(), spaceName);
 
         if (tbl == null)
             return;
@@ -1961,7 +2220,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public void markForRebuildFromHash(@Nullable String spaceName, GridQueryTypeDescriptor type) {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+        TableDescriptor tbl = tableDescriptor(type.name(), spaceName);
 
         if (tbl == null)
             return;
@@ -1975,12 +2234,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * Gets size (for tests only).
      *
      * @param spaceName Space name.
-     * @param type Type descriptor.
+     * @param typeName Type name.
      * @return Size.
      * @throws IgniteCheckedException If failed or {@code -1} if the type is unknown.
      */
-    long size(@Nullable String spaceName, GridQueryTypeDescriptor type) throws IgniteCheckedException {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
+    long size(@Nullable String spaceName, String typeName) throws IgniteCheckedException {
+        TableDescriptor tbl = tableDescriptor(typeName, spaceName);
 
         if (tbl == null)
             return -1;
@@ -2093,6 +2352,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     cleanupStatementCache();
                 }
             }, CLEANUP_STMT_CACHE_PERIOD, CLEANUP_STMT_CACHE_PERIOD);
+
+            ddlProc = new DdlStatementsProcessor();
+
+            dmlProc.start(ctx, this);
+            ddlProc.start(ctx, this);
         }
 
         if (JdbcUtils.serializer != null)
@@ -2255,7 +2519,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             log.debug("Stopping cache query index...");
 
 //        unregisterMBean(); TODO https://issues.apache.org/jira/browse/IGNITE-2139
-
         if (ctx != null && !ctx.cache().context().database().persistenceEnabled()) {
             for (Schema schema : schemas.values())
                 schema.onDrop();
@@ -2778,7 +3041,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Information about table in database.
      */
-    private class TableDescriptor implements GridH2Table.IndexesFactory {
+    private class TableDescriptor implements GridH2SystemIndexFactory {
         /** */
         private final String fullTblName;
 
@@ -2795,9 +3058,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         private GridLuceneIndex luceneIdx;
 
         /** */
-        private Index pkTreeIdx;
-
-        /** */
         private H2PkHashIndex pkHashIdx;
 
         /**
@@ -2808,7 +3068,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             this.type = type;
             this.schema = schema;
 
-            String tblName = escapeName(type.tableName() != null ? type.tableName() : type.name(), schema.escapeAll());
+            String tblName = escapeName(type.tableName(), schema.escapeAll());
 
             fullTblName = schema.schemaName + "." + tblName;
         }
@@ -2846,22 +3106,23 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return S.toString(TableDescriptor.class, this);
         }
 
-        /** {@inheritDoc} */
-        @Override public H2RowFactory createRowFactory(GridH2Table tbl) {
-            int cacheId = CU.cacheId(schema.ccfg.getName());
-
-            GridCacheContext cctx = ctx.cache().context().cacheContext(cacheId);
+        /**
+         * Create H2 row factory.
+         *
+         * @param rowDesc Row descriptor.
+         * @return H2 row factory.
+         */
+        H2RowFactory rowFactory(GridH2RowDescriptor rowDesc) {
+            GridCacheContext cctx = schema.cacheContext();
 
             if (cctx.affinityNode() && cctx.offheapIndex())
-                return new H2RowFactory(tbl.rowDescriptor(), cctx);
+                return new H2RowFactory(rowDesc, cctx);
 
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public ArrayList<Index> createIndexes(GridH2Table tbl) {
-            this.tbl = tbl;
-
+        @Override public ArrayList<Index> createSystemIndexes(GridH2Table tbl) {
             ArrayList<Index> idxs = new ArrayList<>();
 
             IndexColumn keyCol = tbl.indexColumn(KEY_COL, SortOrder.ASCENDING);
@@ -2870,25 +3131,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             if (affCol != null && equal(affCol, keyCol))
                 affCol = null;
 
-            int cacheId = CU.cacheId(schema.ccfg.getName());
-
             Index hashIdx = createHashIndex(
-                    cacheId,
-                    "_key_PK_hash",
-                    tbl,
-                    treeIndexColumns(new ArrayList<IndexColumn>(2), keyCol, affCol));
+                schema,
+                tbl,
+                "_key_PK_hash",
+                treeIndexColumns(new ArrayList<IndexColumn>(2), keyCol, affCol)
+            );
 
             if (hashIdx != null)
                 idxs.add(hashIdx);
 
             // Add primary key index.
-            idxs.add(createSortedIndex(
-                cacheId,
+            Index pkIdx = createSortedIndex(
+                schema,
                 "_key_PK",
                 tbl,
                 true,
                 treeIndexColumns(new ArrayList<IndexColumn>(2), keyCol, affCol),
-                -1));
+                -1
+            );
+
+            idxs.add(pkIdx);
 
             if (type().valueClass() == String.class) {
                 try {
@@ -2901,50 +3164,40 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             boolean affIdxFound = false;
 
-            for (Map.Entry<String, GridQueryIndexDescriptor> e : type.indexes().entrySet()) {
-                String name = e.getKey();
-                GridQueryIndexDescriptor idx = e.getValue();
+            GridQueryIndexDescriptor textIdx = type.textIndex();
 
-                if (idx.type() == QueryIndexType.FULLTEXT) {
-                    try {
-                        luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type);
-                    }
-                    catch (IgniteCheckedException e1) {
-                        throw new IgniteException(e1);
-                    }
+            if (textIdx != null) {
+                try {
+                    luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type);
                 }
-                else {
-                    List<IndexColumn> cols = new ArrayList<>(idx.fields().size() + 2);
-
-                    boolean escapeAll = schema.escapeAll();
+                catch (IgniteCheckedException e1) {
+                    throw new IgniteException(e1);
+                }
+            }
 
-                    for (String field : idx.fields()) {
-                        String fieldName = escapeAll ? field : escapeName(field, false).toUpperCase();
+            // Locate index where affinity column is first (if any).
+            if (affCol != null) {
+                for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
+                    if (idxDesc.type() != QueryIndexType.SORTED)
+                        continue;
 
-                        Column col = tbl.getColumn(fieldName);
+                    String firstField = idxDesc.fields().iterator().next();
 
-                        cols.add(tbl.indexColumn(col.getColumnId(),
-                            idx.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
-                    }
+                    String firstFieldName =
+                        schema.escapeAll() ? firstField : escapeName(firstField, false).toUpperCase();
 
-                    if (idx.type() == QueryIndexType.SORTED) {
-                        // We don't care about number of fields in affinity index, just affinity key must be the first.
-                        affIdxFound |= affCol != null && equal(cols.get(0), affCol);
+                    Column col = tbl.getColumn(firstFieldName);
 
-                        cols = treeIndexColumns(cols, keyCol, affCol);
+                    IndexColumn idxCol = tbl.indexColumn(col.getColumnId(),
+                        idxDesc.descending(firstField) ? SortOrder.DESCENDING : SortOrder.ASCENDING);
 
-                        idxs.add(createSortedIndex(cacheId, name, tbl, false, cols, idx.inlineSize()));
-                    }
-                    else if (idx.type() == QueryIndexType.GEOSPATIAL)
-                        idxs.add(createH2SpatialIndex(tbl, name, cols.toArray(new IndexColumn[cols.size()])));
-                    else
-                        throw new IllegalStateException("Index type: " + idx.type());
+                    affIdxFound |= equal(idxCol, affCol);
                 }
             }
 
             // Add explicit affinity key index if nothing alike was found.
             if (affCol != null && !affIdxFound) {
-                idxs.add(createSortedIndex(cacheId, "AFFINITY_KEY", tbl, false,
+                idxs.add(createSortedIndex(schema, "AFFINITY_KEY", tbl, false,
                     treeIndexColumns(new ArrayList<IndexColumn>(2), affCol, keyCol), -1));
             }
 
@@ -2952,121 +3205,92 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         /**
-         * @param cacheId Cache ID.
-         * @param name Index name,
-         * @param tbl Table.
-         * @param pk Primary key flag.
-         * @param cols Columns.
-         * @return Index.
+         * Get collection of user indexes.
+         *
+         * @return User indexes.
          */
-        private Index createSortedIndex(
-            int cacheId,
-            String name,
-            GridH2Table tbl,
-            boolean pk,
-            List<IndexColumn> cols,
-            int inlineSize
-        ) {
-            try {
-                GridCacheSharedContext<Object, Object> scctx = ctx.cache().context();
+        public Collection<GridH2IndexBase> createUserIndexes() {
+            assert tbl != null;
 
-                GridCacheContext cctx = scctx.cacheContext(cacheId);
+            ArrayList<GridH2IndexBase> res = new ArrayList<>();
 
-                if (log.isInfoEnabled())
-                    log.info("Creating cache index [cacheId=" + cctx.cacheId() + ", idxName=" + name + ']');
+            for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
+                GridH2IndexBase idx = createUserIndex(idxDesc);
 
-                final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
-
-                return new H2TreeIndex(cctx, tbl, name, pk, cols, inlineSize, segments);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
+                res.add(idx);
             }
+
+            return res;
         }
 
         /**
+         * Create user index.
+         *
+         * @param idxDesc Index descriptor.
          * @return Index.
          */
-        private Index createHashIndex(
-            int cacheId,
-            String name,
-            GridH2Table tbl,
-            List<IndexColumn> cols
-        ) {
-            GridCacheSharedContext<Object, Object> scctx = ctx.cache().context();
+        private GridH2IndexBase createUserIndex(GridQueryIndexDescriptor idxDesc) {
+            String name = schema.escapeAll() ? idxDesc.name() : escapeName(idxDesc.name(), false).toUpperCase();
 
-            GridCacheContext cctx = scctx.cacheContext(cacheId);
+            IndexColumn keyCol = tbl.indexColumn(KEY_COL, SortOrder.ASCENDING);
+            IndexColumn affCol = tbl.getAffinityKeyColumn();
 
-            if (cctx.affinityNode() && cctx.offheapIndex()) {
-                assert pkHashIdx == null : pkHashIdx;
+            List<IndexColumn> cols = new ArrayList<>(idxDesc.fields().size() + 2);
 
-                pkHashIdx = new H2PkHashIndex(
-                    cctx,
-                    tbl,
-                    name,
-                    cols);
+            boolean escapeAll = schema.escapeAll();
 
-                return pkHashIdx;
-            }
+            for (String field : idxDesc.fields()) {
+                String fieldName = escapeAll ? field : escapeName(field, false).toUpperCase();
 
-            return null;
-        }
+                Column col = tbl.getColumn(fieldName);
 
-        /**
-         *
-         */
-        void onDrop() {
-            dataTables.remove(tbl.identifier(), tbl);
+                cols.add(tbl.indexColumn(col.getColumnId(),
+                    idxDesc.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
+            }
 
-            tbl.destroy();
+            if (idxDesc.type() == QueryIndexType.SORTED) {
+                cols = treeIndexColumns(cols, keyCol, affCol);
 
-            U.closeQuiet(luceneIdx);
+                return createSortedIndex(schema, name, tbl, false, cols, idxDesc.inlineSize());
+            }
+            else if (idxDesc.type() == QueryIndexType.GEOSPATIAL)
+                return createSpatialIndex(tbl, name, cols.toArray(new IndexColumn[cols.size()]));
+
+            throw new IllegalStateException("Index type: " + idxDesc.type());
         }
 
         /**
+         * Create hash index.
+         *
+         * @param schema Schema.
          * @param tbl Table.
          * @param idxName Index name.
          * @param cols Columns.
+         * @return Index.
          */
-        private SpatialIndex createH2SpatialIndex(
-            GridH2Table tbl,
-            String idxName,
-            IndexColumn[] cols
-        ) {
-            String className = "org.apache.ignite.internal.processors.query.h2.opt.GridH2SpatialIndex";
-
-            try {
-                Class<?> cls = Class.forName(className);
+        private Index createHashIndex(Schema schema, GridH2Table tbl, String idxName, List<IndexColumn> cols) {
+            GridCacheContext cctx = schema.cacheContext();
 
-                Constructor<?> ctor = cls.getConstructor(
-                    GridH2Table.class,
-                    String.class,
-                    Integer.TYPE,
-                    IndexColumn[].class);
-
-                if (!ctor.isAccessible())
-                    ctor.setAccessible(true);
+            if (cctx.affinityNode() && cctx.offheapIndex()) {
+                assert pkHashIdx == null : pkHashIdx;
 
-                final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
+                pkHashIdx = new H2PkHashIndex(cctx, tbl, idxName, cols);
 
-                return (SpatialIndex)ctor.newInstance(tbl, idxName, segments, cols);
-            }
-            catch (Exception e) {
-                throw new IgniteException("Failed to instantiate: " + className, e);
+                return pkHashIdx;
             }
+
+            return null;
         }
 
         /**
-         * @param idxName Index name.
-         * @param tbl Table.
-         * @param pk Primary key flag.
-         * @param columns Index column list.
-         * @return
+         *
          */
-        private Index createTreeIndex(String idxName, GridH2Table tbl, boolean pk, List<IndexColumn> columns) {
-            final int segments = tbl.rowDescriptor().configuration().getQueryParallelism();
+        void onDrop() {
+            dataTables.remove(tbl.identifier(), tbl);
+
+            tbl.destroy();
 
-            return new GridH2TreeIndex(idxName, tbl, pk, columns, segments);
+            U.closeQuiet(luceneIdx);
         }
     }
 
@@ -3252,6 +3476,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         /**
+         * @return Cache context.
+         */
+        public GridCacheContext cacheContext() {
+            return cctx;
+        }
+
+        /**
          * @param tbl Table descriptor.
          */
         public void add(TableDescriptor tbl) {
@@ -3368,7 +3599,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         /** {@inheritDoc} */
         @Override public GridCacheContext<?, ?> context() {
-            return schema.cctx;
+            return schema.cacheContext();
         }
 
         /** {@inheritDoc} */
@@ -3625,4 +3856,63 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             lastUsage = U.currentTimeMillis();
         }
     }
+
+    /**
+     * H2 Table engine.
+     */
+    public static class H2TableEngine implements TableEngine {
+        /** */
+        private static GridH2RowDescriptor rowDesc0;
+
+        /** */
+        private static H2RowFactory rowFactory0;
+
+        /** */
+        private static TableDescriptor tblDesc0;
+
+        /** */
+        private static GridH2Table resTbl0;
+
+        /**
+         * Creates table using given connection, DDL clause for given type descriptor and list of indexes.
+         *
+         * @param conn Connection.
+         * @param sql DDL clause.
+         * @param rowDesc Row descriptor.
+         * @param rowFactory Row factory.
+         * @param tblDesc Table descriptor.
+         * @throws SQLException If failed.
+         * @return Created table.
+         */
+        public static synchronized GridH2Table createTable(Connection conn, String sql,
+            @Nullable GridH2RowDescriptor rowDesc, H2RowFactory rowFactory, TableDescriptor tblDesc)
+            throws SQLException {
+            rowDesc0 = rowDesc;
+            rowFactory0 = rowFactory;
+            tblDesc0 = tblDesc;
+
+            try {
+                try (Statement s = conn.createStatement()) {
+                    s.execute(sql + " engine \"" + H2TableEngine.class.getName() + "\"");
+                }
+
+                tblDesc.tbl = resTbl0;
+
+                return resTbl0;
+            }
+            finally {
+                resTbl0 = null;
+                tblDesc0 = null;
+                rowFactory0 = null;
+                rowDesc0 = null;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public TableBase createTable(CreateTableData createTblData) {
+            resTbl0 = new GridH2Table(createTblData, rowDesc0, rowFactory0, tblDesc0, tblDesc0.schema.spaceName);
+
+            return resTbl0;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/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
new file mode 100644
index 0000000..5b4b494
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -0,0 +1,208 @@
+/*
+ * 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.ddl;
+
+import java.sql.PreparedStatement;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropIndex;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
+import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+import org.h2.command.Prepared;
+import org.h2.command.ddl.CreateIndex;
+import org.h2.command.ddl.DropIndex;
+import org.h2.jdbc.JdbcPreparedStatement;
+
+import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
+
+/**
+ * DDL statements processor.<p>
+ * Contains higher level logic to handle operations as a whole and communicate with the client.
+ */
+public class DdlStatementsProcessor {
+    /** Kernal context. */
+    GridKernalContext ctx;
+
+    /** Indexing. */
+    IgniteH2Indexing idx;
+
+    /**
+     * Initialize message handlers and this' fields needed for further operation.
+     *
+     * @param ctx Kernal context.
+     * @param idx Indexing.
+     */
+    public void start(final GridKernalContext ctx, IgniteH2Indexing idx) {
+        this.ctx = ctx;
+        this.idx = idx;
+    }
+
+    /**
+     * Execute DDL statement.
+     *
+     * @param stmt H2 statement to parse and execute.
+     */
+    @SuppressWarnings("unchecked")
+    public QueryCursor<List<?>> runDdlStatement(PreparedStatement stmt)
+        throws IgniteCheckedException {
+        assert stmt instanceof JdbcPreparedStatement;
+
+        IgniteInternalFuture fut;
+
+        try {
+            GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt));
+
+            if (gridStmt instanceof GridSqlCreateIndex) {
+                GridSqlCreateIndex createIdx = (GridSqlCreateIndex)gridStmt;
+
+                String spaceName = idx.space(createIdx.schemaName());
+
+                QueryIndex newIdx = new QueryIndex();
+
+                newIdx.setName(createIdx.index().getName());
+
+                newIdx.setIndexType(createIdx.index().getIndexType());
+
+                LinkedHashMap<String, Boolean> flds = new LinkedHashMap<>();
+
+                GridH2Table tbl = idx.dataTable(createIdx.schemaName(), createIdx.tableName());
+
+                if (tbl == null)
+                    throw new IgniteSQLException("Table not found [schemaName=" + createIdx.schemaName() + ", " +
+                        "tblName=" + createIdx.tableName() + ']', IgniteQueryErrorCode.TABLE_NOT_FOUND);
+
+                assert tbl.rowDescriptor() != null;
+
+                // Let's replace H2's table and property names by those operated by GridQueryProcessor.
+                GridQueryTypeDescriptor typeDesc = tbl.rowDescriptor().type();
+
+                for (Map.Entry<String, Boolean> e : createIdx.index().getFields().entrySet()) {
+                    GridQueryProperty prop = typeDesc.property(e.getKey());
+
+                    if (prop == null)
+                        throw new IgniteSQLException("Property not found [typeName=" + typeDesc.name() + ", propName=" +
+                            e.getKey() + ']');
+
+                    flds.put(prop.name(), e.getValue());
+                }
+
+                newIdx.setFields(flds);
+
+                fut = ctx.query().dynamicIndexCreate(spaceName, typeDesc.tableName(), newIdx, createIdx.ifNotExists());
+            }
+            else if (gridStmt instanceof GridSqlDropIndex) {
+                GridSqlDropIndex dropIdx = (GridSqlDropIndex)gridStmt;
+
+                String spaceName = idx.space(dropIdx.schemaName());
+
+                fut = ctx.query().dynamicIndexDrop(spaceName, dropIdx.name(), dropIdx.ifExists());
+            }
+            else
+                throw new IgniteSQLException("Unexpected DDL operation [type=" + gridStmt.getClass() + ']',
+                    IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+
+            fut.get();
+
+            QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
+                (Collections.singletonList(0L)), null, false);
+
+            resCur.fieldsMeta(UPDATE_RESULT_META);
+
+            return resCur;
+        }
+        catch (SchemaOperationException e) {
+            throw convert(e);
+        }
+        catch (Exception e) {
+            throw new IgniteSQLException("DLL operation failed.", e);
+        }
+    }
+
+    /**
+     * @return {@link IgniteSQLException} with the message same as of {@code this}'s and
+     */
+    private IgniteSQLException convert(SchemaOperationException e) {
+        int sqlCode;
+
+        switch (e.code()) {
+            case SchemaOperationException.CODE_CACHE_NOT_FOUND:
+                sqlCode = IgniteQueryErrorCode.CACHE_NOT_FOUND;
+
+                break;
+
+            case SchemaOperationException.CODE_TABLE_NOT_FOUND:
+                sqlCode = IgniteQueryErrorCode.TABLE_NOT_FOUND;
+
+                break;
+
+            case SchemaOperationException.CODE_TABLE_EXISTS:
+                sqlCode = IgniteQueryErrorCode.TABLE_ALREADY_EXISTS;
+
+                break;
+
+            case SchemaOperationException.CODE_COLUMN_NOT_FOUND:
+                sqlCode = IgniteQueryErrorCode.COLUMN_NOT_FOUND;
+
+                break;
+
+            case SchemaOperationException.CODE_COLUMN_EXISTS:
+                sqlCode = IgniteQueryErrorCode.COLUMN_ALREADY_EXISTS;
+
+                break;
+
+            case SchemaOperationException.CODE_INDEX_NOT_FOUND:
+                sqlCode = IgniteQueryErrorCode.INDEX_NOT_FOUND;
+
+                break;
+
+            case SchemaOperationException.CODE_INDEX_EXISTS:
+                sqlCode = IgniteQueryErrorCode.INDEX_ALREADY_EXISTS;
+
+                break;
+
+            default:
+                sqlCode = IgniteQueryErrorCode.UNKNOWN;
+        }
+
+        return new IgniteSQLException(e.getMessage(), sqlCode);
+    }
+
+    /**
+     * @param cmd Statement.
+     * @return Whether {@code cmd} is a DDL statement we're able to handle.
+     */
+    public static boolean isDdlStatement(Prepared cmd) {
+        return cmd instanceof CreateIndex || cmd instanceof DropIndex;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index 67e294a..7163834 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -376,7 +376,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
 
     /** {@inheritDoc} */
     @Override public void remove(Session ses) {
-        throw DbException.getUnsupportedException("remove index");
+        // No-op: destroyed from owning table.
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PrimaryScanIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PrimaryScanIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PrimaryScanIndex.java
new file mode 100644
index 0000000..097b34e
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2PrimaryScanIndex.java
@@ -0,0 +1,87 @@
+/*
+ * 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.opt;
+
+import org.h2.engine.Session;
+import org.h2.result.SortOrder;
+import org.h2.table.TableFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper type for primary key.
+ */
+@SuppressWarnings("PackageVisibleInnerClass")
+public class GridH2PrimaryScanIndex extends GridH2ScanIndex<GridH2IndexBase> {
+    /** */
+    static final String SCAN_INDEX_NAME_SUFFIX = "__SCAN_";
+
+    /** Parent table. */
+    private final GridH2Table tbl;
+
+    /** */
+    private final GridH2IndexBase hashIdx;
+
+    /**
+     * Constructor.
+     *
+     * @param tbl Table.
+     * @param treeIdx Tree index.
+     * @param hashIdx Hash index.
+     */
+    GridH2PrimaryScanIndex(GridH2Table tbl, GridH2IndexBase treeIdx, @Nullable GridH2IndexBase hashIdx) {
+        super(treeIdx);
+
+        this.tbl = tbl;
+        this.hashIdx = hashIdx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridH2IndexBase delegate() {
+        boolean rebuildFromHashInProgress = tbl.rebuildFromHashInProgress();
+
+        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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SystemIndexFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SystemIndexFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SystemIndexFactory.java
new file mode 100644
index 0000000..f150b6a
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SystemIndexFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.opt;
+
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.h2.index.Index;
+
+import java.util.ArrayList;
+
+/**
+ * Factory for system table indexes.
+ */
+public interface GridH2SystemIndexFactory {
+    /**
+     * 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> createSystemIndexes(GridH2Table tbl);
+}