You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/12/05 12:38:08 UTC

[04/52] ignite git commit: IGNITE-2294: Implemented DML.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/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 6da8758..f4ffbc7 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
@@ -68,6 +68,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
@@ -79,6 +80,7 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
@@ -87,6 +89,7 @@ import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 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.opt.GridH2DefaultTableEngine;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
@@ -98,6 +101,7 @@ 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;
@@ -129,6 +133,7 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.CommandInterface;
+import org.h2.command.Prepared;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
@@ -314,7 +319,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 c = DriverManager.getConnection(dbUrl);
             }
             catch (SQLException e) {
-                throw new IgniteException("Failed to initialize DB connection: " + dbUrl, e);
+                throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
             }
 
             conns.add(c);
@@ -327,6 +332,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private volatile GridKernalContext ctx;
 
     /** */
+    private final DmlStatementsProcessor dmlProc = new DmlStatementsProcessor(this);
+
+    /** */
     private final ConcurrentMap<String, GridH2Table> dataTables = new ConcurrentHashMap8<>();
 
     /** Statement cache. */
@@ -369,6 +377,13 @@ 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.
@@ -410,6 +425,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             return c.prepareStatement(sql);
     }
 
+    /** {@inheritDoc} */
+    @Override public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException {
+        return prepareStatement(connectionForSpace(schema), sql, false);
+    }
+
     /**
      * Gets DB connection.
      *
@@ -437,7 +457,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 c.schema(schema);
             }
             catch (SQLException e) {
-                throw new IgniteCheckedException("Failed to set schema for DB connection for thread [schema=" +
+                throw new IgniteSQLException("Failed to set schema for DB connection for thread [schema=" +
                     schema + "]", e);
             }
             finally {
@@ -492,7 +512,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         catch (SQLException e) {
             onSqlException();
 
-            throw new IgniteCheckedException("Failed to execute statement: " + sql, e);
+            throw new IgniteSQLException("Failed to execute statement: " + sql, e);
         }
         finally {
             U.close(stmt, log);
@@ -732,8 +752,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         catch (SQLException e) {
             onSqlException();
 
-            throw new IgniteCheckedException("Failed to drop database index table [type=" + tbl.type().name() +
-                ", table=" + tbl.fullTableName() + "]", e);
+            throw new IgniteSQLException("Failed to drop database index table [type=" + tbl.type().name() +
+                ", table=" + tbl.fullTableName() + "]", IgniteQueryErrorCode.TABLE_DROP_FAILED, e);
         }
         finally {
             U.close(stmt, log);
@@ -776,9 +796,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         initLocalQueryContext(conn, enforceJoinOrder, filters);
 
+        Prepared p = null;
+
         try {
             final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
 
+            p = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
+
+            if (!p.isQuery()) {
+                GridH2QueryContext.clearThreadLocal();
+
+                SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
+
+                if (params != null)
+                    fldsQry.setArgs(params.toArray());
+
+                fldsQry.setEnforceJoinOrder(enforceJoinOrder);
+                fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
+
+                return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filters, cancel);
+            }
+
             List<GridQueryFieldMetadata> meta;
 
             try {
@@ -797,7 +835,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             };
         }
         finally {
-            GridH2QueryContext.clearThreadLocal();
+            if (p == null || p.isQuery())
+                GridH2QueryContext.clearThreadLocal();
         }
     }
 
@@ -874,16 +913,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             throw new IgniteCheckedException("Failed to parse SQL query: " + sql, e);
         }
 
-        switch (commandType(stmt)) {
-            case CommandInterface.SELECT:
-            case CommandInterface.CALL:
-            case CommandInterface.EXPLAIN:
-            case CommandInterface.ANALYZE:
-                break;
-            default:
-                throw new IgniteCheckedException("Failed to execute non-query SQL statement: " + sql);
-        }
-
         bindParameters(stmt, params);
 
         return stmt;
@@ -1055,7 +1084,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         final TableDescriptor tbl = tableDescriptor(spaceName, type);
 
         if (tbl == null)
-            throw new CacheException("Failed to find SQL table for type: " + type.name());
+            throw new IgniteSQLException("Failed to find SQL table for type: " + type.name(),
+                IgniteQueryErrorCode.TABLE_NOT_FOUND);
 
         String sql = generateQuery(qry, tbl);
 
@@ -1100,7 +1130,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         TableDescriptor tblDesc = tableDescriptor(type, space);
 
         if (tblDesc == null)
-            throw new CacheException("Failed to find SQL table for type: " + type);
+            throw new IgniteSQLException("Failed to find SQL table for type: " + type,
+                IgniteQueryErrorCode.TABLE_NOT_FOUND);
 
         String sql;
 
@@ -1120,7 +1151,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if(qry.getTimeout() > 0)
             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
 
-        final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry);
+        final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry, null);
 
         final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
             @Override public Iterator<Cache.Entry<K, V>> iterator() {
@@ -1161,7 +1192,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+        GridQueryCancel cancel) {
         final String space = cctx.name();
         final String sqlQry = qry.getSql();
 
@@ -1214,7 +1246,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             cachesCreated = true;
                         }
                         else
-                            throw new CacheException("Failed to parse query: " + sqlQry, e);
+                            throw new IgniteSQLException("Failed to parse query: " + sqlQry, e.getSQLState(),
+                                IgniteQueryErrorCode.PARSING);
                     }
                 }
             }
@@ -1222,6 +1255,22 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 GridH2QueryContext.clearThreadLocal();
             }
 
+            Prepared prepared = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
+
+            if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery) qry).isQuery() != prepared.isQuery())
+                throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
+                    IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+
+            if (!prepared.isQuery()) {
+                try {
+                    return dmlProc.updateSqlFieldsTwoStep(cctx.namexx(), stmt, qry, cancel);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteSQLException("Failed to execute DML statement [qry=" + sqlQry + ", params=" +
+                        Arrays.deepToString(qry.getArgs()) + "]", e);
+                }
+            }
+
             try {
                 bindParameters(stmt, F.asList(qry.getArgs()));
 
@@ -1271,7 +1320,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     Arrays.deepToString(qry.getArgs()) + "]", e);
             }
             catch (SQLException e) {
-                throw new CacheException(e);
+                throw new IgniteSQLException(e);
             }
             finally {
                 U.close(stmt, log);
@@ -1283,7 +1332,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         twoStepQry.pageSize(qry.getPageSize());
 
-        GridQueryCancel cancel = new GridQueryCancel();
+        if (cancel == null)
+            cancel = new GridQueryCancel();
 
         QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
             runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder, qry.getTimeout(), cancel), cancel);
@@ -1433,7 +1483,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param escapeAll Escape flag.
      * @return Escaped name.
      */
-    private static String escapeName(String name, boolean escapeAll) {
+    public static String escapeName(String name, boolean escapeAll) {
         if (name == null) // It is possible only for a cache name.
             return ESC_STR;
 
@@ -2557,7 +2607,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Special field set iterator based on database result set.
      */
-    private static class FieldsIterator extends GridH2ResultSetIterator<List<?>> {
+    public static class FieldsIterator extends GridH2ResultSetIterator<List<?>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2565,7 +2615,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @param data Data.
          * @throws IgniteCheckedException If failed.
          */
-        protected FieldsIterator(ResultSet data) throws IgniteCheckedException {
+        public FieldsIterator(ResultSet data) throws IgniteCheckedException {
             super(data, false, true);
         }
 
@@ -2607,7 +2657,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Field descriptor.
      */
-    private static class SqlFieldMetadata implements GridQueryFieldMetadata {
+    static class SqlFieldMetadata implements GridQueryFieldMetadata {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -3012,6 +3062,21 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         /** {@inheritDoc} */
+        @Override public void setColumnValue(Object key, Object val, Object colVal, int col) {
+            try {
+                props[col].setValue(key, val, colVal);
+            }
+            catch (IgniteCheckedException e) {
+                throw DbException.convert(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isColumnKeyProperty(int col) {
+            return props[col].key();
+        }
+
+        /** {@inheritDoc} */
         @Override public GridH2KeyValueRowOffheap createPointer(long ptr) {
             GridH2KeyValueRowOffheap row = schema.rowCache.get(ptr);
 
@@ -3033,6 +3098,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         @Override public boolean snapshotableIndex() {
             return snapshotableIdx;
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean quoteAllIdentifiers() {
+            return schema.escapeAll();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
new file mode 100644
index 0000000..dc90fe9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
@@ -0,0 +1,27 @@
+/*
+ * 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.dml;
+
+import org.apache.ignite.internal.util.lang.GridPlainClosure;
+
+/**
+ * Operand for fast UPDATE or DELETE (single item operation that does not involve any SELECT).
+ */
+public interface FastUpdateArgument extends GridPlainClosure<Object[], Object> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
new file mode 100644
index 0000000..cb47704
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArguments.java
@@ -0,0 +1,53 @@
+/*
+ * 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.dml;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Arguments for fast, query-less UPDATE or DELETE - key and, optionally, value and new value.
+ */
+public final class FastUpdateArguments {
+    /** Operand to compute key. */
+    public final FastUpdateArgument key;
+
+    /** Operand to compute value. */
+    public final FastUpdateArgument val;
+
+    /** Operand to compute new value. */
+    public final FastUpdateArgument newVal;
+
+    /** */
+    public FastUpdateArguments(FastUpdateArgument key, FastUpdateArgument val, FastUpdateArgument newVal) {
+        assert key != null && key != NULL_ARGUMENT;
+        assert val != null;
+        assert newVal != null;
+
+        this.key = key;
+        this.val = val;
+        this.newVal = newVal;
+    }
+
+    /** Operand that always evaluates as {@code null}. */
+    public final static FastUpdateArgument NULL_ARGUMENT = new FastUpdateArgument() {
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            return null;
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java
new file mode 100644
index 0000000..709d5c9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/KeyValueSupplier.java
@@ -0,0 +1,30 @@
+/*
+ * 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.dml;
+
+import java.util.List;
+import org.apache.ignite.internal.util.lang.GridPlainClosure;
+
+/**
+ * Method to construct new instances of keys and values on SQL MERGE and INSERT,
+ * as well as to build new values during UPDATE - a function that takes a row selected from DB
+ * and then transforms it into new object.
+ */
+public interface KeyValueSupplier extends GridPlainClosure<List<?>, Object> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java
new file mode 100644
index 0000000..0440648
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdateMode.java
@@ -0,0 +1,36 @@
+/*
+ * 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.dml;
+
+/**
+ * DML statement execution plan type - MERGE/INSERT from rows or subquery,
+ * or UPDATE/DELETE from subquery or literals/params based.
+ */
+public enum UpdateMode {
+    /** */
+    MERGE,
+
+    /** */
+    INSERT,
+
+    /** */
+    UPDATE,
+
+    /** */
+    DELETE,
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
new file mode 100644
index 0000000..5976f4c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.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.dml;
+
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Update plan - where to take data to update cache from and how to construct new keys and values, if needed.
+ */
+public final class UpdatePlan {
+    /** Initial statement to drive the rest of the logic. */
+    public final UpdateMode mode;
+
+    /** Target table to be affected by initial DML statement. */
+    public final GridH2Table tbl;
+
+    /** Column names to set or update. */
+    public final String[] colNames;
+
+    /** Method to create key for INSERT or MERGE, ignored for UPDATE and DELETE. */
+    public final KeyValueSupplier keySupplier;
+
+    /** Method to create value to put to cache, ignored for DELETE. */
+    public final KeyValueSupplier valSupplier;
+
+    /** Index of key column, if it's explicitly mentioned in column list of MERGE or INSERT,
+     * ignored for UPDATE and DELETE. */
+    public final int keyColIdx;
+
+    /** Index of value column, if it's explicitly mentioned in column list. Ignored for UPDATE and DELETE. */
+    public final int valColIdx;
+
+    /** SELECT statement built upon initial DML statement. */
+    public final String selectQry;
+
+    /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
+    public final boolean isLocSubqry;
+
+    /** Number of rows in rows based MERGE or INSERT. */
+    public final int rowsNum;
+
+    /** Arguments for fast UPDATE or DELETE. */
+    public final FastUpdateArguments fastUpdateArgs;
+
+    /** */
+    private UpdatePlan(UpdateMode mode, GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
+        int rowsNum, FastUpdateArguments fastUpdateArgs) {
+        this.colNames = colNames;
+        this.rowsNum = rowsNum;
+        assert mode != null;
+        assert tbl != null;
+
+        this.mode = mode;
+        this.tbl = tbl;
+        this.keySupplier = keySupplier;
+        this.valSupplier = valSupplier;
+        this.keyColIdx = keyColIdx;
+        this.valColIdx = valColIdx;
+        this.selectQry = selectQry;
+        this.isLocSubqry = isLocSubqry;
+        this.fastUpdateArgs = fastUpdateArgs;
+    }
+
+    /** */
+    public static UpdatePlan forMerge(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry,
+        int rowsNum) {
+        assert !F.isEmpty(colNames);
+
+        return new UpdatePlan(UpdateMode.MERGE, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+            selectQry, isLocSubqry, rowsNum, null);
+    }
+
+    /** */
+    public static UpdatePlan forInsert(GridH2Table tbl, String[] colNames, KeyValueSupplier keySupplier,
+        KeyValueSupplier valSupplier, int keyColIdx, int valColIdx, String selectQry, boolean isLocSubqry, int rowsNum) {
+        assert !F.isEmpty(colNames);
+
+        return new UpdatePlan(UpdateMode.INSERT, tbl, colNames, keySupplier, valSupplier, keyColIdx, valColIdx, selectQry,
+            isLocSubqry, rowsNum, null);
+    }
+
+    /** */
+    public static UpdatePlan forUpdate(GridH2Table tbl, String[] colNames, KeyValueSupplier valSupplier, int valColIdx,
+        String selectQry) {
+        assert !F.isEmpty(colNames);
+
+        return new UpdatePlan(UpdateMode.UPDATE, tbl, colNames, null, valSupplier, -1, valColIdx, selectQry,
+            false, 0, null);
+    }
+
+    /** */
+    public static UpdatePlan forDelete(GridH2Table tbl, String selectQry) {
+        return new UpdatePlan(UpdateMode.DELETE, tbl, null, null, null, -1, -1, selectQry, false, 0, null);
+    }
+
+    /** */
+    public static UpdatePlan forFastUpdate(UpdateMode mode, GridH2Table tbl, FastUpdateArguments fastUpdateArgs) {
+        assert mode == UpdateMode.UPDATE || mode == UpdateMode.DELETE;
+
+        return new UpdatePlan(mode, tbl, null, null, null, -1, -1, null, false, 0, fastUpdateArgs);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
new file mode 100644
index 0000000..15c94c3
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -0,0 +1,502 @@
+/*
+ * 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.dml;
+
+import java.lang.reflect.Constructor;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+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.DmlStatementsProcessor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.sql.DmlAstUtils;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDelete;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlInsert;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlMerge;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSelect;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUnion;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUpdate;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.h2.command.Prepared;
+import org.h2.table.Column;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.KEY_FIELD_NAME;
+import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.VAL_FIELD_NAME;
+
+/**
+ * Logic for building update plans performed by {@link DmlStatementsProcessor}.
+ */
+public final class UpdatePlanBuilder {
+    /** */
+    private UpdatePlanBuilder() {
+        // No-op.
+    }
+
+    /**
+     * Generate SELECT statements to retrieve data for modifications from and find fast UPDATE or DELETE args,
+     * if available.
+     *
+     * @param prepared H2's {@link Prepared}.
+     * @return Update plan.
+     */
+    public static UpdatePlan planForStatement(Prepared prepared,
+        @Nullable Integer errKeysPos) throws IgniteCheckedException {
+        assert !prepared.isQuery();
+
+        GridSqlStatement stmt = new GridSqlQueryParser().parse(prepared);
+
+        if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert)
+            return planForInsert(stmt);
+        else
+            return planForUpdate(stmt, errKeysPos);
+    }
+
+    /**
+     * Prepare update plan for INSERT or MERGE.
+     *
+     * @param stmt INSERT or MERGE statement.
+     * @return Update plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private static UpdatePlan planForInsert(GridSqlStatement stmt) throws IgniteCheckedException {
+        GridSqlQuery sel;
+
+        GridSqlElement target;
+
+        GridSqlColumn[] cols;
+
+        boolean isTableSubqry;
+
+        int rowsNum;
+
+        GridSqlTable tbl;
+
+        GridH2RowDescriptor desc;
+
+        if (stmt instanceof GridSqlInsert) {
+            GridSqlInsert ins = (GridSqlInsert) stmt;
+            target = ins.into();
+
+            tbl = gridTableForElement(target);
+            desc = tbl.dataTable().rowDescriptor();
+
+            cols = ins.columns();
+            sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query(), desc);
+            isTableSubqry = (ins.query() != null);
+            rowsNum = isTableSubqry ? 0 : ins.rows().size();
+        }
+        else if (stmt instanceof GridSqlMerge) {
+            GridSqlMerge merge = (GridSqlMerge) stmt;
+
+            target = merge.into();
+
+            tbl = gridTableForElement(target);
+            desc = tbl.dataTable().rowDescriptor();
+
+            // This check also protects us from attempts to update key or its fields directly -
+            // when no key except cache key can be used, it will serve only for uniqueness checks,
+            // not for updates, and hence will allow putting new pairs only.
+            // We don't quote _key and _val column names on CREATE TABLE, so they are always uppercase here.
+            GridSqlColumn[] keys = merge.keys();
+            if (keys.length != 1 || IgniteH2Indexing.KEY_FIELD_NAME.equals(keys[0].columnName()))
+                throw new CacheException("SQL MERGE does not support arbitrary keys");
+
+            cols = merge.columns();
+            sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query(), desc);
+            isTableSubqry = (merge.query() != null);
+            rowsNum = isTableSubqry ? 0 : merge.rows().size();
+        }
+        else throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
+                IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+
+        // Let's set the flag only for subqueries that have their FROM specified.
+        isTableSubqry = (isTableSubqry && (sel instanceof GridSqlUnion ||
+            (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
+
+        int keyColIdx = -1;
+        int valColIdx = -1;
+
+        boolean hasKeyProps = false;
+        boolean hasValProps = false;
+
+        if (desc == null)
+            throw new IgniteSQLException("Row descriptor undefined for table '" + tbl.dataTable().getName() + "'",
+                IgniteQueryErrorCode.NULL_TABLE_DESCRIPTOR);
+
+        GridCacheContext<?, ?> cctx = desc.context();
+
+        String[] colNames = new String[cols.length];
+
+        for (int i = 0; i < cols.length; i++) {
+            colNames[i] = cols[i].columnName();
+
+            if (isKeyColumn(cols[i].columnName(), desc)) {
+                keyColIdx = i;
+                continue;
+            }
+
+            if (isValColumn(cols[i].columnName(), desc)) {
+                valColIdx = i;
+                continue;
+            }
+
+            GridQueryProperty prop = desc.type().property(cols[i].columnName());
+
+            assert prop != null : "Property '" + cols[i].columnName() + "' not found.";
+
+            if (prop.key())
+                hasKeyProps = true;
+            else
+                hasValProps = true;
+        }
+
+        KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true);
+        KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false);
+
+        if (stmt instanceof GridSqlMerge)
+            return UpdatePlan.forMerge(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+                sel.getSQL(), !isTableSubqry, rowsNum);
+        else
+            return UpdatePlan.forInsert(tbl.dataTable(), colNames, keySupplier, valSupplier, keyColIdx, valColIdx,
+                sel.getSQL(), !isTableSubqry, rowsNum);
+    }
+
+    /**
+     * Prepare update plan for UPDATE or DELETE.
+     *
+     * @param stmt UPDATE or DELETE statement.
+     * @param errKeysPos index to inject param for re-run keys at. Null if it's not a re-run plan.
+     * @return Update plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    private static UpdatePlan planForUpdate(GridSqlStatement stmt, @Nullable Integer errKeysPos) throws IgniteCheckedException {
+        GridSqlElement target;
+
+        FastUpdateArguments fastUpdate;
+
+        UpdateMode mode;
+
+        if (stmt instanceof GridSqlUpdate) {
+            // Let's verify that user is not trying to mess with key's columns directly
+            verifyUpdateColumns(stmt);
+
+            GridSqlUpdate update = (GridSqlUpdate) stmt;
+            target = update.target();
+            fastUpdate = DmlAstUtils.getFastUpdateArgs(update);
+            mode = UpdateMode.UPDATE;
+        }
+        else if (stmt instanceof GridSqlDelete) {
+            GridSqlDelete del = (GridSqlDelete) stmt;
+            target = del.from();
+            fastUpdate = DmlAstUtils.getFastDeleteArgs(del);
+            mode = UpdateMode.DELETE;
+        }
+        else
+            throw new IgniteSQLException("Unexpected DML operation [cls=" + stmt.getClass().getName() + ']',
+                IgniteQueryErrorCode.UNEXPECTED_OPERATION);
+
+        GridSqlTable tbl = gridTableForElement(target);
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        GridH2RowDescriptor desc = gridTbl.rowDescriptor();
+
+        if (desc == null)
+            throw new IgniteSQLException("Row descriptor undefined for table '" + gridTbl.getName() + "'",
+                IgniteQueryErrorCode.NULL_TABLE_DESCRIPTOR);
+
+        if (fastUpdate != null)
+            return UpdatePlan.forFastUpdate(mode, gridTbl, fastUpdate);
+        else {
+            GridSqlSelect sel;
+
+            if (stmt instanceof GridSqlUpdate) {
+                boolean bin = desc.context().binaryMarshaller();
+
+                List<GridSqlColumn> updatedCols = ((GridSqlUpdate) stmt).cols();
+
+                int valColIdx = -1;
+
+                String[] colNames = new String[updatedCols.size()];
+
+                for (int i = 0; i < updatedCols.size(); i++) {
+                    colNames[i] = updatedCols.get(i).columnName();
+
+                    if (isValColumn(colNames[i], desc))
+                        valColIdx = i;
+                }
+
+                boolean hasNewVal = (valColIdx != -1);
+
+                // Statement updates distinct properties if it does not have _val in updated columns list
+                // or if its list of updated columns includes only _val, i.e. is single element.
+                boolean hasProps = !hasNewVal || updatedCols.size() > 1;
+
+                // Index of new _val in results of SELECT
+                if (hasNewVal)
+                    valColIdx += 2;
+
+                int newValColIdx;
+
+                if (!hasProps) // No distinct properties, only whole new value - let's take it
+                    newValColIdx = valColIdx;
+                else if (bin) // We update distinct columns in binary mode - let's choose correct index for the builder
+                    newValColIdx = (hasNewVal ? valColIdx : 1);
+                else // Distinct properties, non binary mode - let's instantiate.
+                    newValColIdx = -1;
+
+                // We want supplier to take present value only in case of binary mode as it will create
+                // whole new object as a result anyway, so we don't need to copy previous property values explicitly.
+                // Otherwise we always want it to instantiate new object whose properties we will later
+                // set to current values.
+                KeyValueSupplier newValSupplier = createSupplier(desc.context(), desc.type(), newValColIdx, hasProps, false);
+
+                sel = DmlAstUtils.selectForUpdate((GridSqlUpdate) stmt, errKeysPos);
+
+                return UpdatePlan.forUpdate(gridTbl, colNames, newValSupplier, valColIdx, sel.getSQL());
+            }
+            else {
+                sel = DmlAstUtils.selectForDelete((GridSqlDelete) stmt, errKeysPos);
+
+                return UpdatePlan.forDelete(gridTbl, sel.getSQL());
+            }
+        }
+    }
+
+    /**
+     * Detect appropriate method of instantiating key or value (take from param, create binary builder,
+     * invoke default ctor, or allocate).
+     *
+     * @param cctx Cache context.
+     * @param desc Table descriptor.
+     * @param colIdx Column index if key or value is present in columns list, {@code -1} if it's not.
+     * @param hasProps Whether column list affects individual properties of key or value.
+     * @param key Whether supplier should be created for key or for value.
+     * @return Closure returning key or value.
+     * @throws IgniteCheckedException
+     */
+    @SuppressWarnings({"ConstantConditions", "unchecked"})
+    private static KeyValueSupplier createSupplier(final GridCacheContext<?, ?> cctx, GridQueryTypeDescriptor desc,
+                                                   final int colIdx, boolean hasProps, final boolean key) throws IgniteCheckedException {
+        final String typeName = key ? desc.keyTypeName() : desc.valueTypeName();
+
+        //Try to find class for the key locally.
+        final Class<?> cls = key ? U.firstNotNull(U.classForName(desc.keyTypeName(), null), desc.keyClass())
+            : desc.valueClass();
+
+        boolean isSqlType = GridQueryProcessor.isSqlType(cls);
+
+        // If we don't need to construct anything from scratch, just return value from array.
+        if (isSqlType || !hasProps || !cctx.binaryMarshaller()) {
+            if (colIdx != -1)
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        return arg.get(colIdx);
+                    }
+                };
+            else if (isSqlType)
+                // Non constructable keys and values (SQL types) must be present in the query explicitly.
+                throw new IgniteCheckedException((key ? "Key" : "Value") + " is missing from query");
+        }
+
+        if (cctx.binaryMarshaller()) {
+            if (colIdx != -1) {
+                // If we have key or value explicitly present in query, create new builder upon them...
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        BinaryObject bin = cctx.grid().binary().toBinary(arg.get(colIdx));
+
+                        return cctx.grid().binary().builder(bin);
+                    }
+                };
+            }
+            else {
+                // ...and if we don't, just create a new builder.
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        return cctx.grid().binary().builder(typeName);
+                    }
+                };
+            }
+        }
+        else {
+            Constructor<?> ctor;
+
+            try {
+                ctor = cls.getDeclaredConstructor();
+                ctor.setAccessible(true);
+            }
+            catch (NoSuchMethodException | SecurityException ignored) {
+                ctor = null;
+            }
+
+            if (ctor != null) {
+                final Constructor<?> ctor0 = ctor;
+
+                // Use default ctor, if it's present...
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        try {
+                            return ctor0.newInstance();
+                        }
+                        catch (Exception e) {
+                            throw new IgniteCheckedException("Failed to invoke default ctor for " +
+                                (key ? "key" : "value"), e);
+                        }
+                    }
+                };
+            }
+            else {
+                // ...or allocate new instance with unsafe, if it's not
+                return new KeyValueSupplier() {
+                    /** {@inheritDoc} */
+                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                        try {
+                            return GridUnsafe.allocateInstance(cls);
+                        }
+                        catch (InstantiationException e) {
+                            throw new IgniteCheckedException("Failed to invoke default ctor for " +
+                                (key ? "key" : "value"), e);
+                        }
+                    }
+                };
+            }
+        }
+    }
+
+
+
+    /**
+     * @param target Expression to extract the table from.
+     * @return Back end table for this element.
+     */
+    private static GridSqlTable gridTableForElement(GridSqlElement target) {
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        DmlAstUtils.collectAllGridTablesInTarget(target, tbls);
+
+        if (tbls.size() != 1)
+            throw new IgniteSQLException("Failed to determine target table", IgniteQueryErrorCode.TABLE_NOT_FOUND);
+
+        return tbls.iterator().next();
+    }
+
+    /**
+     * Check that UPDATE statement affects no key columns.
+     *
+     * @param statement Statement.
+     */
+    private static void verifyUpdateColumns(GridSqlStatement statement) {
+        if (statement == null || !(statement instanceof GridSqlUpdate))
+            return;
+
+        GridSqlUpdate update = (GridSqlUpdate) statement;
+
+        GridSqlElement updTarget = update.target();
+
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        DmlAstUtils.collectAllGridTablesInTarget(updTarget, tbls);
+
+        if (tbls.size() != 1)
+            throw new IgniteSQLException("Failed to determine target table for UPDATE", IgniteQueryErrorCode.TABLE_NOT_FOUND);
+
+        GridSqlTable tbl = tbls.iterator().next();
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        if (updateAffectsKeyColumns(gridTbl, update.set().keySet()))
+            throw new IgniteSQLException("SQL UPDATE can't modify key or its fields directly",
+                IgniteQueryErrorCode.KEY_UPDATE);
+    }
+
+    /**
+     * Check if given set of modified columns intersects with the set of SQL properties of the key.
+     *
+     * @param gridTbl Table.
+     * @param affectedColNames Column names.
+     * @return {@code true} if any of given columns corresponds to the key or any of its properties.
+     */
+    private static boolean updateAffectsKeyColumns(GridH2Table gridTbl, Set<String> affectedColNames) {
+        GridH2RowDescriptor desc = gridTbl.rowDescriptor();
+
+        Column[] cols = gridTbl.getColumns();
+
+        // Check "_key" column itself - always has index of 0.
+        if (affectedColNames.contains(cols[0].getName()))
+            return true;
+
+        // Start off from i = 2 to skip indices of 0 an 1 corresponding to key and value respectively.
+        for (int i = 2; i < cols.length; i++)
+            if (affectedColNames.contains(cols[i].getName()) && desc.isColumnKeyProperty(i - 2))
+                return true;
+
+        return false;
+    }
+
+    /**
+     * Check that given column corresponds to the key with respect to case sensitivity, if needed (should be considered
+     * when the schema escapes all identifiers on table creation).
+     * @param colName Column name.
+     * @param desc Row descriptor.
+     * @return {@code true} if column name corresponds to _key with respect to case sensitivity depending on schema.
+     */
+    private static boolean isKeyColumn(String colName, GridH2RowDescriptor desc) {
+        if (desc.quoteAllIdentifiers())
+            return KEY_FIELD_NAME.equals(colName);
+        else
+            return KEY_FIELD_NAME.equalsIgnoreCase(colName);
+    }
+
+    /**
+     * Check that given column corresponds to the key with respect to case sensitivity, if needed (should be considered
+     * when the schema escapes all identifiers on table creation).
+     * @param colName Column name.
+     * @param desc Row descriptor.
+     * @return {@code true} if column name corresponds to _key with respect to case sensitivity depending on schema.
+     */
+    private static boolean isValColumn(String colName, GridH2RowDescriptor desc) {
+        if (desc.quoteAllIdentifiers())
+            return VAL_FIELD_NAME.equals(colName);
+        else
+            return VAL_FIELD_NAME.equalsIgnoreCase(colName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java
new file mode 100644
index 0000000..653f1be
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains routines for executing DML statements on top of H2.
+ */
+package org.apache.ignite.internal.processors.query.h2.dml;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
index f519c30..3465ed7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
@@ -107,6 +107,24 @@ public interface GridH2RowDescriptor extends GridOffHeapSmartPointerFactory<Grid
     public Object columnValue(Object key, Object val, int col);
 
     /**
+     * Gets column value by column index.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @param colVal Value to set to column.
+     * @param col Column index.
+     */
+    public void setColumnValue(Object key, Object val, Object colVal, int col);
+
+    /**
+     * Determine whether a column corresponds to a property of key or to one of value.
+     *
+     * @param col Column index.
+     * @return {@code true} if given column corresponds to a key property, {@code false} otherwise
+     */
+    public boolean isColumnKeyProperty(int col);
+
+    /**
      * @return Unsafe memory.
      */
     public GridUnsafeMemory memory();
@@ -145,4 +163,9 @@ public interface GridH2RowDescriptor extends GridOffHeapSmartPointerFactory<Grid
      * @return {@code True} if index should support snapshots.
      */
     public boolean snapshotableIndex();
+
+    /**
+     * @return Escape all identifiers.
+     */
+    public boolean quoteAllIdentifiers();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 33aaf7b..914e0da 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -68,7 +68,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         IndexColumn.mapColumns(cols, tbl);
 
         initBaseIndex(tbl, 0, name, cols,
-            pk ? IndexType.createUnique(false, false) : IndexType.createNonUnique(false, false, false));
+            pk ? IndexType.createPrimaryKey(false, false) : IndexType.createNonUnique(false, false, false));
 
         final GridH2RowDescriptor desc = tbl.rowDescriptor();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
new file mode 100644
index 0000000..39b1b74
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -0,0 +1,599 @@
+/*
+ * 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.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+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.dml.FastUpdateArguments;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.util.lang.IgnitePair;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.h2.expression.Expression;
+import org.h2.table.Column;
+import org.h2.table.Table;
+import org.h2.util.IntArray;
+import org.h2.value.DataType;
+import org.h2.value.Value;
+import org.h2.value.ValueDate;
+import org.h2.value.ValueInt;
+import org.h2.value.ValueString;
+import org.h2.value.ValueTime;
+import org.h2.value.ValueTimestamp;
+import org.h2.value.ValueTimestampUtc;
+import org.jetbrains.annotations.Nullable;
+
+import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArgument;
+
+/**
+ * AST utils for DML
+ */
+public final class DmlAstUtils {
+    /**
+     * Empty ctor to prevent initialization.
+     */
+    private DmlAstUtils() {
+        // No-op.
+    }
+
+    /**
+     * Create SELECT on which subsequent INSERT or MERGE will be based.
+     *
+     * @param cols Columns to insert values into.
+     * @param rows Rows to create pseudo-SELECT upon.
+     * @param subQry Subquery to use rather than rows.
+     * @param desc Row descriptor.
+     * @return Subquery or pseudo-SELECT to evaluate inserted expressions.
+     */
+    public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List<GridSqlElement[]> rows,
+        GridSqlQuery subQry, GridH2RowDescriptor desc) {
+        if (!F.isEmpty(rows)) {
+            assert !F.isEmpty(cols);
+
+            GridSqlSelect sel = new GridSqlSelect();
+
+            GridSqlFunction from = new GridSqlFunction(GridSqlFunctionType.TABLE);
+
+            sel.from(from);
+
+            GridSqlArray[] args = new GridSqlArray[cols.length];
+
+            for (int i = 0; i < cols.length; i++) {
+                GridSqlArray arr = new GridSqlArray(rows.size());
+
+                String colName = IgniteH2Indexing.escapeName(cols[i].columnName(), desc.quoteAllIdentifiers());
+
+                GridSqlAlias alias = new GridSqlAlias(colName, arr);
+
+                alias.resultType(cols[i].resultType());
+
+                from.addChild(alias);
+
+                args[i] = arr;
+
+                GridSqlColumn newCol = new GridSqlColumn(null, from, colName, "TABLE." + colName);
+
+                newCol.resultType(cols[i].resultType());
+
+                sel.addColumn(newCol, true);
+            }
+
+            for (GridSqlElement[] row : rows) {
+                assert cols.length == row.length;
+
+                for (int i = 0; i < row.length; i++)
+                    args[i].addChild(row[i]);
+            }
+
+            return sel;
+        }
+        else {
+            assert subQry != null;
+
+            return subQry;
+        }
+    }
+
+    /**
+     * Generate SQL SELECT based on DELETE's WHERE, LIMIT, etc.
+     *
+     * @param del Delete statement.
+     * @param keysParamIdx Index for .
+     * @return SELECT statement.
+     */
+    public static GridSqlSelect selectForDelete(GridSqlDelete del, @Nullable Integer keysParamIdx) {
+        GridSqlSelect mapQry = new GridSqlSelect();
+
+        mapQry.from(del.from());
+
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        collectAllGridTablesInTarget(del.from(), tbls);
+
+        assert tbls.size() == 1 : "Failed to determine target table for DELETE";
+
+        GridSqlTable tbl = tbls.iterator().next();
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        assert gridTbl != null : "Failed to determine target grid table for DELETE";
+
+        Column h2KeyCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.KEY_COL);
+
+        Column h2ValCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.VAL_COL);
+
+        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName(), h2KeyCol.getSQL());
+        keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
+
+        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName(), h2ValCol.getSQL());
+        valCol.resultType(GridSqlType.fromColumn(h2ValCol));
+
+        mapQry.addColumn(keyCol, true);
+        mapQry.addColumn(valCol, true);
+
+        GridSqlElement where = del.where();
+        if (keysParamIdx != null)
+            where = injectKeysFilterParam(where, keyCol, keysParamIdx);
+
+        mapQry.where(where);
+        mapQry.limit(del.limit());
+
+        return mapQry;
+    }
+
+    /**
+     * @param update UPDATE statement.
+     * @return {@code null} if given statement directly updates {@code _val} column with a literal or param value
+     * and filters by single non expression key (and, optionally,  by single non expression value).
+     */
+    public static FastUpdateArguments getFastUpdateArgs(GridSqlUpdate update) {
+        IgnitePair<GridSqlElement> filter = findKeyValueEqualityCondition(update.where());
+
+        if (filter == null)
+            return null;
+
+        if (update.cols().size() != 1 ||
+            !IgniteH2Indexing.VAL_FIELD_NAME.equalsIgnoreCase(update.cols().get(0).columnName()))
+            return null;
+
+        GridSqlElement set = update.set().get(update.cols().get(0).columnName());
+
+        if (!(set instanceof GridSqlConst || set instanceof GridSqlParameter))
+            return null;
+
+        return new FastUpdateArguments(operandForElement(filter.getKey()), operandForElement(filter.getValue()),
+            operandForElement(set));
+    }
+
+    /**
+     * Create operand based on exact type of SQL element.
+     *
+     * @param el element.
+     * @return Operand.
+     */
+    private static FastUpdateArgument operandForElement(GridSqlElement el) {
+        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
+
+        if (el == null)
+            return FastUpdateArguments.NULL_ARGUMENT;
+
+        if (el instanceof GridSqlConst)
+            return new ValueArgument(((GridSqlConst)el).value().getObject());
+        else
+            return new ParamArgument(((GridSqlParameter)el).index());
+    }
+
+    /**
+     * @param del DELETE statement.
+     * @return {@code true} if given statement filters by single non expression key.
+     */
+    public static FastUpdateArguments getFastDeleteArgs(GridSqlDelete del) {
+        IgnitePair<GridSqlElement> filter = findKeyValueEqualityCondition(del.where());
+
+        if (filter == null)
+            return null;
+
+        return new FastUpdateArguments(operandForElement(filter.getKey()), operandForElement(filter.getValue()),
+            FastUpdateArguments.NULL_ARGUMENT);
+    }
+
+    /**
+     * @param where Element to test.
+     * @return Whether given element corresponds to {@code WHERE _key = ?}, and key is a literal expressed
+     * in query or a query param.
+     */
+    private static IgnitePair<GridSqlElement> findKeyValueEqualityCondition(GridSqlElement where) {
+        if (where == null || !(where instanceof GridSqlOperation))
+            return null;
+
+        GridSqlOperation whereOp = (GridSqlOperation) where;
+
+        // Does this WHERE limit only by _key?
+        if (isKeyEqualityCondition(whereOp))
+            return new IgnitePair<>(whereOp.child(1), null);
+
+        // Or maybe it limits both by _key and _val?
+        if (whereOp.operationType() != GridSqlOperationType.AND)
+            return null;
+
+        GridSqlElement left = whereOp.child(0);
+
+        GridSqlElement right = whereOp.child(1);
+
+        if (!(left instanceof GridSqlOperation && right instanceof GridSqlOperation))
+            return null;
+
+        GridSqlOperation leftOp = (GridSqlOperation) left;
+
+        GridSqlOperation rightOp = (GridSqlOperation) right;
+
+        if (isKeyEqualityCondition(leftOp)) { // _key = ? and _val = ?
+            if (!isValueEqualityCondition(rightOp))
+                return null;
+
+            return new IgnitePair<>(leftOp.child(1), rightOp.child(1));
+        }
+        else if (isKeyEqualityCondition(rightOp)) { // _val = ? and _key = ?
+            if (!isValueEqualityCondition(leftOp))
+                return null;
+
+            return new IgnitePair<>(rightOp.child(1), leftOp.child(1));
+        }
+        else // Neither
+            return null;
+    }
+
+    /**
+     * @param op Operation.
+     * @param colName Column name to check.
+     * @return Whether this condition is of form {@code colName} = ?
+     */
+    private static boolean isEqualityCondition(GridSqlOperation op, String colName) {
+        if (op.operationType() != GridSqlOperationType.EQUAL)
+            return false;
+
+        GridSqlElement left = op.child(0);
+        GridSqlElement right = op.child(1);
+
+        return left instanceof GridSqlColumn &&
+            colName.equalsIgnoreCase(((GridSqlColumn) left).columnName()) &&
+            (right instanceof GridSqlConst || right instanceof GridSqlParameter);
+    }
+
+    /**
+     * @param op Operation.
+     * @return Whether this condition is of form _key = ?
+     */
+    private static boolean isKeyEqualityCondition(GridSqlOperation op) {
+        return isEqualityCondition(op, IgniteH2Indexing.KEY_FIELD_NAME);
+    }
+
+    /**
+     * @param op Operation.
+     * @return Whether this condition is of form _val = ?
+     */
+    private static boolean isValueEqualityCondition(GridSqlOperation op) {
+        return isEqualityCondition(op, IgniteH2Indexing.VAL_FIELD_NAME);
+    }
+
+
+    /**
+     * Generate SQL SELECT based on UPDATE's WHERE, LIMIT, etc.
+     *
+     * @param update Update statement.
+     * @param keysParamIdx Index of new param for the array of keys.
+     * @return SELECT statement.
+     */
+    public static GridSqlSelect selectForUpdate(GridSqlUpdate update, @Nullable Integer keysParamIdx) {
+        GridSqlSelect mapQry = new GridSqlSelect();
+
+        mapQry.from(update.target());
+
+        Set<GridSqlTable> tbls = new HashSet<>();
+
+        collectAllGridTablesInTarget(update.target(), tbls);
+
+        assert tbls.size() == 1 : "Failed to determine target table for UPDATE";
+
+        GridSqlTable tbl = tbls.iterator().next();
+
+        GridH2Table gridTbl = tbl.dataTable();
+
+        assert gridTbl != null : "Failed to determine target grid table for UPDATE";
+
+        Column h2KeyCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.KEY_COL);
+
+        Column h2ValCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.VAL_COL);
+
+        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName(), h2KeyCol.getSQL());
+        keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
+
+        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName(), h2ValCol.getSQL());
+        valCol.resultType(GridSqlType.fromColumn(h2ValCol));
+
+        mapQry.addColumn(keyCol, true);
+        mapQry.addColumn(valCol, true);
+
+        for (GridSqlColumn c : update.cols()) {
+            String newColName = "_upd_" + c.columnName();
+            // We have to use aliases to cover cases when the user
+            // wants to update _val field directly (if it's a literal)
+            GridSqlAlias alias = new GridSqlAlias(newColName, elementOrDefault(update.set().get(c.columnName()), c), true);
+            alias.resultType(c.resultType());
+            mapQry.addColumn(alias, true);
+        }
+
+        GridSqlElement where = update.where();
+        if (keysParamIdx != null)
+            where = injectKeysFilterParam(where, keyCol, keysParamIdx);
+
+        mapQry.where(where);
+        mapQry.limit(update.limit());
+
+        return mapQry;
+    }
+
+    /**
+     * Do what we can to compute default value for this column (mimics H2 behavior).
+     * @see Table#getDefaultValue
+     * @see Column#validateConvertUpdateSequence
+     * @param el SQL element.
+     * @param col Column.
+     * @return {@link GridSqlConst#NULL}, if {@code el} is null, or {@code el} if
+     * it's not {@link GridSqlKeyword#DEFAULT}, or computed default value.
+     */
+    private static GridSqlElement elementOrDefault(GridSqlElement el, GridSqlColumn col) {
+        if (el == null)
+            return GridSqlConst.NULL;
+
+        if (el != GridSqlKeyword.DEFAULT)
+            return el;
+
+        Column h2Col = col.column();
+
+        Expression dfltExpr = h2Col.getDefaultExpression();
+
+        Value dfltVal;
+
+        try {
+            dfltVal = dfltExpr != null ? dfltExpr.getValue(null) : null;
+        }
+        catch (Exception e) {
+            throw new IgniteSQLException("Failed to evaluate default value for a column " + col.columnName());
+        }
+
+        if (dfltVal != null)
+            return new GridSqlConst(dfltVal);
+
+        int type = h2Col.getType();
+
+        DataType dt = DataType.getDataType(type);
+
+        if (dt.decimal)
+            dfltVal = ValueInt.get(0).convertTo(type);
+        else if (dt.type == Value.TIMESTAMP)
+            dfltVal = ValueTimestamp.fromMillis(U.currentTimeMillis());
+        else if (dt.type == Value.TIMESTAMP_UTC)
+            dfltVal = ValueTimestampUtc.fromMillis(U.currentTimeMillis());
+        else if (dt.type == Value.TIME)
+            dfltVal = ValueTime.fromNanos(0);
+        else if (dt.type == Value.DATE)
+            dfltVal = ValueDate.fromMillis(U.currentTimeMillis());
+        else
+            dfltVal = ValueString.get("").convertTo(type);
+
+        return new GridSqlConst(dfltVal);
+    }
+
+    /**
+     * Append additional condition to WHERE for it to select only specific keys.
+     *
+     * @param where Initial condition.
+     * @param keyCol Column to base the new condition on.
+     * @return New condition.
+     */
+    private static GridSqlElement injectKeysFilterParam(GridSqlElement where, GridSqlColumn keyCol, int paramIdx) {
+        GridSqlElement e = new GridSqlOperation(GridSqlOperationType.IN, keyCol, new GridSqlParameter(paramIdx));
+
+        if (where == null)
+            return e;
+        else
+            return new GridSqlOperation(GridSqlOperationType.AND, where, e);
+    }
+
+    /**
+     * @param qry Select.
+     * @param params Parameters.
+     * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
+     * @return Extracted parameters list.
+     */
+    private static List<Object> findParams(GridSqlQuery qry, Object[] params, ArrayList<Object> target,
+                                           IntArray paramIdxs) {
+        if (qry instanceof GridSqlSelect)
+            return findParams((GridSqlSelect)qry, params, target, paramIdxs);
+
+        GridSqlUnion union = (GridSqlUnion)qry;
+
+        findParams(union.left(), params, target, paramIdxs);
+        findParams(union.right(), params, target, paramIdxs);
+
+        findParams(qry.limit(), params, target, paramIdxs);
+        findParams(qry.offset(), params, target, paramIdxs);
+
+        return target;
+    }
+
+    /**
+     * @param qry Select.
+     * @param params Parameters.
+     * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
+     * @return Extracted parameters list.
+     */
+    private static List<Object> findParams(GridSqlSelect qry, Object[] params, ArrayList<Object> target,
+                                           IntArray paramIdxs) {
+        if (params.length == 0)
+            return target;
+
+        for (GridSqlElement el : qry.columns(false))
+            findParams(el, params, target, paramIdxs);
+
+        findParams(qry.from(), params, target, paramIdxs);
+        findParams(qry.where(), params, target, paramIdxs);
+
+        // Don't search in GROUP BY and HAVING since they expected to be in select list.
+
+        findParams(qry.limit(), params, target, paramIdxs);
+        findParams(qry.offset(), params, target, paramIdxs);
+
+        return target;
+    }
+
+    /**
+     * @param el Element.
+     * @param params Parameters.
+     * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
+     */
+    private static void findParams(@Nullable GridSqlElement el, Object[] params, ArrayList<Object> target,
+                                   IntArray paramIdxs) {
+        if (el == null)
+            return;
+
+        if (el instanceof GridSqlParameter) {
+            // H2 Supports queries like "select ?5" but first 4 non-existing parameters are need to be set to any value.
+            // Here we will set them to NULL.
+            final int idx = ((GridSqlParameter)el).index();
+
+            while (target.size() < idx)
+                target.add(null);
+
+            if (params.length <= idx)
+                throw new IgniteException("Invalid number of query parameters. " +
+                    "Cannot find " + idx + " parameter.");
+
+            Object param = params[idx];
+
+            if (idx == target.size())
+                target.add(param);
+            else
+                target.set(idx, param);
+
+            paramIdxs.add(idx);
+        }
+        else if (el instanceof GridSqlSubquery)
+            findParams(((GridSqlSubquery)el).select(), params, target, paramIdxs);
+        else
+            for (GridSqlElement child : el)
+                findParams(child, params, target, paramIdxs);
+    }
+
+    /**
+     * Processes all the tables and subqueries using the given closure.
+     *
+     * @param from FROM element.
+     * @param c Closure each found table and subquery will be passed to. If returns {@code true} the we need to stop.
+     * @return {@code true} If we have found.
+     */
+    private static boolean findTablesInFrom(GridSqlElement from, IgnitePredicate<GridSqlElement> c) {
+        if (from == null)
+            return false;
+
+        if (from instanceof GridSqlTable || from instanceof GridSqlSubquery)
+            return c.apply(from);
+
+        if (from instanceof GridSqlJoin) {
+            // Left and right.
+            if (findTablesInFrom(from.child(0), c))
+                return true;
+
+            if (findTablesInFrom(from.child(1), c))
+                return true;
+
+            // We don't process ON condition because it is not a joining part of from here.
+            return false;
+        }
+        else if (from instanceof GridSqlAlias)
+            return findTablesInFrom(from.child(), c);
+        else if (from instanceof GridSqlFunction)
+            return false;
+
+        throw new IllegalStateException(from.getClass().getName() + " : " + from.getSQL());
+    }
+
+    /**
+     * @param from From element.
+     * @param tbls Tables.
+     */
+    public static void collectAllGridTablesInTarget(GridSqlElement from, final Set<GridSqlTable> tbls) {
+        findTablesInFrom(from, new IgnitePredicate<GridSqlElement>() {
+            @Override public boolean apply(GridSqlElement el) {
+                if (el instanceof GridSqlTable)
+                    tbls.add((GridSqlTable)el);
+
+                return false;
+            }
+        });
+    }
+
+    /** Simple constant value based operand. */
+    private final static class ValueArgument implements FastUpdateArgument {
+        /** Value to return. */
+        private final Object val;
+
+        /** */
+        private ValueArgument(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            return val;
+        }
+    }
+
+    /** Simple constant value based operand. */
+    private final static class ParamArgument implements FastUpdateArgument {
+        /** Value to return. */
+        private final int paramIdx;
+
+        /** */
+        private ParamArgument(int paramIdx) {
+            assert paramIdx >= 0;
+
+            this.paramIdx = paramIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
+            assert arg.length > paramIdx;
+
+            return arg[paramIdx];
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
index 69e98bf..caa2000 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import org.h2.util.StatementBuilder;
 
 /**
@@ -32,6 +33,13 @@ public class GridSqlArray extends GridSqlElement {
         super(size == 0 ? Collections.<GridSqlElement>emptyList() : new ArrayList<GridSqlElement>(size));
     }
 
+    /**
+     * @param children Initial child list.
+     */
+    public GridSqlArray(List<GridSqlElement> children) {
+        super(children);
+    }
+
     /** {@inheritDoc} */
     @Override public String getSQL() {
         if (size() == 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
index 976eb2c..bb7d338 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
@@ -18,14 +18,20 @@
 package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.Collections;
+import org.h2.expression.ValueExpression;
 import org.h2.value.Value;
 import org.h2.value.ValueBoolean;
+import org.h2.value.ValueNull;
 
 /**
  * Constant value.
  */
 public class GridSqlConst extends GridSqlElement implements GridSqlValue {
     /** */
+    public static final GridSqlElement NULL = new GridSqlConst(ValueNull.INSTANCE)
+        .resultType(GridSqlType.fromExpression(ValueExpression.getNull()));
+
+    /** */
     public static final GridSqlElement TRUE = new GridSqlConst(ValueBoolean.get(true))
         .resultType(GridSqlType.BOOLEAN);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.java
new file mode 100644
index 0000000..225de58
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlDelete.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.h2.sql;
+
+import org.h2.util.StatementBuilder;
+import org.h2.util.StringUtils;
+
+/** */
+public class GridSqlDelete extends GridSqlStatement {
+    /** */
+    private GridSqlElement from;
+
+    /** */
+    private GridSqlElement where;
+
+    /** */
+    public GridSqlDelete from(GridSqlElement from) {
+        this.from = from;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement from() {
+        return from;
+    }
+
+    /** */
+    public GridSqlDelete where(GridSqlElement where) {
+        this.where = where;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement where() {
+        return where;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("DELETE")
+            .append("\nFROM ")
+            .append(from.getSQL());
+
+        if (where != null)
+            buff.append("\nWHERE ").append(StringUtils.unEnclose(where.getSQL()));
+
+        if (limit != null)
+            buff.append("\nLIMIT (").append(StringUtils.unEnclose(limit.getSQL())).append(')');
+
+        return buff.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
index 7cd7a6b..4e93196 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
@@ -94,7 +94,10 @@ public class GridSqlFunction extends GridSqlElement {
         if (schema != null)
             buff.append(Parser.quoteIdentifier(schema)).append('.');
 
-        buff.append(Parser.quoteIdentifier(name));
+        // We don't need to quote identifier as long as H2 never does so with function names when generating plan SQL.
+        // On the other hand, quoting identifiers that also serve as keywords (like CURRENT_DATE() and CURRENT_DATE)
+        // turns CURRENT_DATE() into "CURRENT_DATE"(), which is not good.
+        buff.append(name);
 
         if (type == CASE) {
             buff.append(' ').append(child().getSQL());

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java
new file mode 100644
index 0000000..07b36e6
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlInsert.java
@@ -0,0 +1,149 @@
+/*
+ * 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.List;
+import org.h2.util.StatementBuilder;
+
+/** */
+public class GridSqlInsert extends GridSqlStatement {
+    /** */
+    private GridSqlElement into;
+
+    /** */
+    private GridSqlColumn[] cols;
+
+    /** */
+    private List<GridSqlElement[]> rows;
+
+    /** Insert subquery. */
+    private GridSqlQuery qry;
+
+    /**
+     * Not supported, introduced for clarity and correct SQL generation.
+     * @see org.h2.command.dml.Insert#insertFromSelect
+     */
+    private boolean direct;
+
+    /**
+     * Not supported, introduced for clarity and correct SQL generation.
+     * @see org.h2.command.dml.Insert#sortedInsertMode
+     */
+    private boolean sorted;
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("INSERT")
+            .append("\nINTO ")
+            .append(into.getSQL())
+            .append('(');
+
+        for (GridSqlColumn col : cols) {
+            buff.appendExceptFirst(", ");
+            buff.append('\n')
+                .append(col.getSQL());
+        }
+        buff.append("\n)\n");
+
+        if (direct)
+            buff.append("DIRECT ");
+
+        if (sorted)
+            buff.append("SORTED ");
+
+        if (!rows.isEmpty()) {
+            buff.append("VALUES\n");
+            StatementBuilder valuesBuff = new StatementBuilder();
+
+            for (GridSqlElement[] row : rows()) {
+                valuesBuff.appendExceptFirst(",\n");
+                StatementBuilder rowBuff = new StatementBuilder("(");
+                for (GridSqlElement e : row) {
+                    rowBuff.appendExceptFirst(", ");
+                    rowBuff.append(e != null ? e.getSQL() : "DEFAULT");
+                }
+                rowBuff.append(')');
+                valuesBuff.append(rowBuff.toString());
+            }
+            buff.append(valuesBuff.toString());
+        }
+        else
+            buff.append('\n')
+                .append(qry.getSQL());
+
+        return buff.toString();
+    }
+
+    /** */
+    public GridSqlElement into() {
+        return into;
+    }
+
+    /** */
+    public GridSqlInsert into(GridSqlElement from) {
+        this.into = from;
+        return this;
+    }
+
+    /** */
+    public List<GridSqlElement[]> rows() {
+        return rows;
+    }
+
+    /** */
+    public GridSqlInsert rows(List<GridSqlElement[]> rows) {
+        assert rows != null;
+        this.rows = rows;
+        return this;
+    }
+
+    /** */
+    public GridSqlQuery query() {
+        return qry;
+    }
+
+    /** */
+    public GridSqlInsert query(GridSqlQuery qry) {
+        this.qry = qry;
+        return this;
+    }
+
+    /** */
+    public GridSqlColumn[] columns() {
+        return cols;
+    }
+
+    /** */
+    public GridSqlInsert columns(GridSqlColumn[] cols) {
+        this.cols = cols;
+        return this;
+    }
+
+    /** */
+    public GridSqlInsert direct(boolean direct) {
+        this.direct = direct;
+        return this;
+    }
+
+    /** */
+    public GridSqlInsert sorted(boolean sorted) {
+        this.sorted = sorted;
+        return this;
+    }
+}