You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/12/20 08:04:51 UTC

[43/50] [abbrv] ignite git commit: IGNITE-4490: SQL: avoid querying H2 for INSERT and MERGE when it is not needed. This closes #1387.

IGNITE-4490: SQL: avoid querying H2 for INSERT and MERGE when it is not needed. This closes #1387.


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

Branch: refs/heads/ignite-zk
Commit: ec290234bf7b54c5549bd7aa0d99b36d3d0638ac
Parents: 8f327a1
Author: Alexander Paschenko <al...@gmail.com>
Authored: Tue Dec 19 11:11:11 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Dec 19 11:11:11 2017 +0300

----------------------------------------------------------------------
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |   2 +-
 .../ignite/jdbc/thin/JdbcThinBatchSelfTest.java |   5 +-
 .../query/h2/DmlStatementsProcessor.java        |  53 +++++-----
 .../processors/query/h2/dml/DmlArgument.java    |  31 ++++++
 .../processors/query/h2/dml/DmlArguments.java   | 104 +++++++++++++++++++
 .../processors/query/h2/dml/DmlAstUtils.java    |  14 ++-
 .../processors/query/h2/dml/DmlUtils.java       |  19 ++--
 .../processors/query/h2/dml/FastUpdate.java     |  91 ++--------------
 .../query/h2/dml/FastUpdateArgument.java        |  27 -----
 .../processors/query/h2/dml/UpdatePlan.java     |  80 +++++++++++---
 .../query/h2/dml/UpdatePlanBuilder.java         |  37 ++++++-
 11 files changed, 304 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index 952baa5..fb96f31 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -107,7 +107,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
     public void testDmlErrors() throws SQLException {
         checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, null)", "22004");
 
-        checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, 'zzz')", "50000");
+        checkErrorState("INSERT INTO \"test\".INTEGER(_key, _val) values(1, 'zzz')", "0700B");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
index 5e2e39e..8609615 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBatchSelfTest.java
@@ -260,7 +260,8 @@ public class JdbcThinBatchSelfTest extends JdbcThinAbstractDmlStatementSelfTest
             pstmt.executeBatch();
 
             fail("BatchUpdateException must be thrown");
-        } catch(BatchUpdateException e) {
+        }
+        catch(BatchUpdateException e) {
             int [] updCnts = e.getUpdateCounts();
 
             assertEquals("Invalid update counts size", BATCH_SIZE, updCnts.length);
@@ -268,7 +269,7 @@ public class JdbcThinBatchSelfTest extends JdbcThinAbstractDmlStatementSelfTest
             for (int i = 0; i < BATCH_SIZE; ++i)
                 assertEquals("Invalid update count",1, updCnts[i]);
 
-            if (!e.getMessage().contains("Failed to execute SQL query.")) {
+            if (!e.getMessage().contains("Value conversion failed")) {
                 log.error("Invalid exception: ", e);
 
                 fail();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/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 243d1dc..9a6b0af 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
@@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -51,7 +50,6 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlBatchSender;
 import org.apache.ignite.internal.processors.query.h2.dml.DmlDistributedPlanInfo;
-import org.apache.ignite.internal.processors.query.h2.dml.FastUpdate;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdateMode;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
@@ -167,7 +165,7 @@ public class DmlStatementsProcessor {
             UpdateResult r;
 
             try {
-                r = executeUpdateStatement(schemaName, cctx, conn, prepared, fieldsQry, loc, filters, cancel, errKeys);
+                r = executeUpdateStatement(schemaName, cctx, conn, prepared, fieldsQry, loc, filters, cancel);
             }
             finally {
                 cctx.operationContextPerCall(opCtx);
@@ -247,15 +245,13 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, Object[] args)
+    long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, final Object[] args)
         throws IgniteCheckedException {
-        args = U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY);
-
         Prepared p = GridSqlQueryParser.prepared(stmt);
 
         assert p != null;
 
-        UpdatePlan plan = UpdatePlanBuilder.planForStatement(p, true, idx, null, null, null);
+        final UpdatePlan plan = UpdatePlanBuilder.planForStatement(p, true, idx, null, null, null);
 
         if (!F.eq(streamer.cacheName(), plan.cacheContext().name()))
             throw new IgniteSQLException("Cross cache streaming is not supported, please specify cache explicitly" +
@@ -270,14 +266,22 @@ public class DmlStatementsProcessor {
 
             final ArrayList<List<?>> data = new ArrayList<>(plan.rowCount());
 
-            final GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()), plan.selectQuery(),
-                F.asList(args), null, false, 0, null);
-
             QueryCursorImpl<List<?>> stepCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
                 @Override public Iterator<List<?>> iterator() {
                     try {
-                        return new GridQueryCacheObjectsIterator(res.iterator(), idx.objectContext(),
-                            cctx.keepBinary());
+                        Iterator<List<?>> it;
+
+                        if (!F.isEmpty(plan.selectQuery())) {
+                            GridQueryFieldsResult res = idx.queryLocalSqlFields(idx.schema(cctx.name()),
+                                plan.selectQuery(), F.asList(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)),
+                                null, false, 0, null);
+
+                            it = res.iterator();
+                        }
+                        else
+                            it = plan.createRows(U.firstNotNull(args, X.EMPTY_OBJECT_ARRAY)).iterator();
+
+                        return new GridQueryCacheObjectsIterator(it, idx.objectContext(), cctx.keepBinary());
                     }
                     catch (IgniteCheckedException e) {
                         throw new IgniteException(e);
@@ -329,27 +333,23 @@ public class DmlStatementsProcessor {
      * @param loc Local query flag.
      * @param filters Cache name and key filter.
      * @param cancel Query cancel state holder.
-     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
      * @return Pair [number of successfully processed items; keys that have failed to be processed]
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
     private UpdateResult executeUpdateStatement(String schemaName, final GridCacheContext cctx, Connection c,
-        Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters,
-        GridQueryCancel cancel, Object[] failedKeys) throws IgniteCheckedException {
+        Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
         int mainCacheId = cctx.cacheId();
 
         Integer errKeysPos = null;
 
         UpdatePlan plan = getPlanForStatement(schemaName, c, prepared, fieldsQry, loc, errKeysPos);
 
-        FastUpdate fastUpdate = plan.fastUpdate();
-
-        if (fastUpdate != null) {
-            assert F.isEmpty(failedKeys) && errKeysPos == null;
+        UpdateResult fastUpdateRes = plan.processFast(fieldsQry.getArgs());
 
-            return fastUpdate.execute(plan.cacheContext().cache(), fieldsQry.getArgs());
-        }
+        if (fastUpdateRes != null)
+            return fastUpdateRes;
 
         if (plan.distributedPlan() != null) {
             UpdateResult result = doDistributedUpdate(schemaName, fieldsQry, plan, cancel);
@@ -359,13 +359,13 @@ public class DmlStatementsProcessor {
                 return result;
         }
 
-        assert !F.isEmpty(plan.selectQuery());
-
-        QueryCursorImpl<List<?>> cur;
+        Iterable<List<?>> cur;
 
         // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
         // sub-query and not some dummy stuff like "select 1, 2, 3;"
         if (!loc && !plan.isLocalSubquery()) {
+            assert !F.isEmpty(plan.selectQuery());
+
             SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQuery(), fieldsQry.isCollocated())
                 .setArgs(fieldsQry.getArgs())
                 .setDistributedJoins(fieldsQry.isDistributedJoins())
@@ -374,9 +374,10 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>)idx.queryDistributedSqlFields(schemaName, newFieldsQry, true,
-                cancel, mainCacheId, true).get(0);
+            cur = idx.queryDistributedSqlFields(schemaName, newFieldsQry, true, cancel, mainCacheId, true).get(0);
         }
+        else if (plan.hasRows())
+            cur = plan.createRows(fieldsQry.getArgs());
         else {
             final GridQueryFieldsResult res = idx.queryLocalSqlFields(schemaName, plan.selectQuery(),
                 F.asList(fieldsQry.getArgs()), filters, fieldsQry.isEnforceJoinOrder(), fieldsQry.getTimeout(), cancel);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java
new file mode 100644
index 0000000..b3c3dce
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArgument.java
@@ -0,0 +1,31 @@
+/*
+ * 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 argument
+ */
+public interface DmlArgument {
+    /**
+     * Get argument from parameter list.
+     *
+     * @param params Query input parameters.
+     * @return value.
+     */
+    Object get(Object[] params);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
new file mode 100644
index 0000000..f5aa332
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlArguments.java
@@ -0,0 +1,104 @@
+/*
+ * 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.sql.GridSqlConst;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DML arguments factory.
+ */
+public class DmlArguments {
+    /** Operand that always evaluates as {@code null}. */
+    private final static DmlArgument NULL_ARG = new ConstantArgument(null);
+
+    /**
+     * Create argument from AST element.
+     *
+     * @param el Element.
+     * @return DML argument.
+     */
+    public static DmlArgument create(@Nullable GridSqlElement el) {
+        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
+
+        if (el == null)
+            return NULL_ARG;
+
+        if (el instanceof GridSqlConst)
+            return new ConstantArgument(((GridSqlConst)el).value().getObject());
+        else
+            return new ParamArgument(((GridSqlParameter)el).index());
+    }
+
+    /**
+     * Private constructor.
+     */
+    private DmlArguments() {
+        // No-op.
+    }
+
+    /**
+     * Value argument.
+     */
+    private static class ConstantArgument implements DmlArgument {
+        /** Value to return. */
+        private final Object val;
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        private ConstantArgument(Object val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        public Object get(Object[] params) {
+            return val;
+        }
+    }
+
+    /**
+     * Parameter argument.
+     */
+    private static class ParamArgument implements DmlArgument {
+        /** Value to return. */
+        private final int paramIdx;
+
+        /**
+         * Constructor.
+         *
+         * @param paramIdx Parameter index.
+         */
+        private ParamArgument(int paramIdx) {
+            assert paramIdx >= 0;
+
+            this.paramIdx = paramIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get(Object[] params) {
+            assert params.length > paramIdx;
+
+            return params[paramIdx];
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
index 054e708..161ff4a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
@@ -83,7 +83,7 @@ public final class DmlAstUtils {
      * @param cols Columns to insert values into.
      * @param rows Rows to create pseudo-SELECT upon.
      * @param subQry Subquery to use rather than rows.
-     * @return Subquery or pseudo-SELECT to evaluate inserted expressions.
+     * @return Subquery or pseudo-SELECT to evaluate inserted expressions, or {@code null} no query needs to be run.
      */
     public static GridSqlQuery selectForInsertOrMerge(GridSqlColumn[] cols, List<GridSqlElement[]> rows,
         GridSqlQuery subQry) {
@@ -98,6 +98,8 @@ public final class DmlAstUtils {
 
             GridSqlArray[] args = new GridSqlArray[cols.length];
 
+            boolean noQry = true;
+
             for (int i = 0; i < cols.length; i++) {
                 GridSqlArray arr = new GridSqlArray(rows.size());
 
@@ -121,10 +123,18 @@ public final class DmlAstUtils {
             for (GridSqlElement[] row : rows) {
                 assert cols.length == row.length;
 
-                for (int i = 0; i < row.length; i++)
+                for (int i = 0; i < row.length; i++) {
+                    GridSqlElement el = row[i];
+
+                    noQry &= (el instanceof GridSqlConst || el instanceof GridSqlParameter);
+
                     args[i].addChild(row[i]);
+                }
             }
 
+            if (noQry)
+                return null;
+
             return sel;
         }
         else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
index 6621fc2..8d4861e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlUtils.java
@@ -17,6 +17,10 @@
 
 package org.apache.ignite.internal.processors.query.h2.dml;
 
+import java.lang.reflect.Array;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Date;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
@@ -30,11 +34,6 @@ import org.h2.value.ValueDate;
 import org.h2.value.ValueTime;
 import org.h2.value.ValueTimestamp;
 
-import java.lang.reflect.Array;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Date;
-
 /**
  * DML utility methods.
  */
@@ -101,7 +100,15 @@ public class DmlUtils {
                 return newArr;
             }
 
-            return H2Utils.convert(val, desc, type);
+            Object res = H2Utils.convert(val, desc, type);
+
+            if (res instanceof Date && res.getClass() != Date.class && expCls == Date.class) {
+                // We can get a Timestamp instead of Date when converting a String to Date
+                // without query - let's handle this
+                return new Date(((Date) res).getTime());
+            }
+
+            return res;
         }
         catch (Exception e) {
             throw new IgniteSQLException("Value conversion failed [from=" + currCls.getName() + ", to=" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
index e662245..dcceff3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
@@ -20,26 +20,21 @@ package org.apache.ignite.internal.processors.query.h2.dml;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.query.h2.UpdateResult;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlConst;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlParameter;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Arguments for fast, query-less UPDATE or DELETE - key and, optionally, value and new value.
  */
 public final class FastUpdate {
-    /** Operand that always evaluates as {@code null}. */
-    private final static FastUpdateArgument NULL_ARG = new ConstantArgument(null);
-
     /** Operand to compute key. */
-    private final FastUpdateArgument keyArg;
+    private final DmlArgument keyArg;
 
     /** Operand to compute value. */
-    private final FastUpdateArgument valArg;
+    private final DmlArgument valArg;
 
     /** Operand to compute new value. */
-    private final FastUpdateArgument newValArg;
+    private final DmlArgument newValArg;
 
     /**
      * Create fast update instance.
@@ -50,9 +45,9 @@ public final class FastUpdate {
      * @return Fast update.
      */
     public static FastUpdate create(GridSqlElement key, GridSqlElement val, @Nullable GridSqlElement newVal) {
-        FastUpdateArgument keyArg = argument(key);
-        FastUpdateArgument valArg = argument(val);
-        FastUpdateArgument newValArg = argument(newVal);
+        DmlArgument keyArg = DmlArguments.create(key);
+        DmlArgument valArg = DmlArguments.create(val);
+        DmlArgument newValArg = DmlArguments.create(newVal);
 
         return new FastUpdate(keyArg, valArg, newValArg);
     }
@@ -64,7 +59,7 @@ public final class FastUpdate {
      * @param valArg Value argument.
      * @param newValArg New value argument.
      */
-    private FastUpdate(FastUpdateArgument keyArg, FastUpdateArgument valArg, FastUpdateArgument newValArg) {
+    private FastUpdate(DmlArgument keyArg, DmlArgument valArg, DmlArgument newValArg) {
         this.keyArg = keyArg;
         this.valArg = valArg;
         this.newValArg = newValArg;
@@ -80,12 +75,12 @@ public final class FastUpdate {
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
     public UpdateResult execute(GridCacheAdapter cache, Object[] args) throws IgniteCheckedException {
-        Object key = keyArg.apply(args);
+        Object key = keyArg.get(args);
 
         assert key != null;
 
-        Object val = valArg.apply(args);
-        Object newVal = newValArg.apply(args);
+        Object val = valArg.get(args);
+        Object newVal = newValArg.get(args);
 
         boolean res;
 
@@ -106,70 +101,4 @@ public final class FastUpdate {
 
         return res ? UpdateResult.ONE : UpdateResult.ZERO;
     }
-
-    /**
-     * Create argument for AST element.
-     *
-     * @param el Element.
-     * @return Argument.
-     */
-    private static FastUpdateArgument argument(@Nullable GridSqlElement el) {
-        assert el == null ^ (el instanceof GridSqlConst || el instanceof GridSqlParameter);
-
-        if (el == null)
-            return NULL_ARG;
-
-        if (el instanceof GridSqlConst)
-            return new ConstantArgument(((GridSqlConst)el).value().getObject());
-        else
-            return new ParamArgument(((GridSqlParameter)el).index());
-    }
-
-    /**
-     * Value argument.
-     */
-    private static class ConstantArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final Object val;
-
-        /**
-         * Constructor.
-         *
-         * @param val Value.
-         */
-        private ConstantArgument(Object val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object apply(Object[] arg) throws IgniteCheckedException {
-            return val;
-        }
-    }
-
-    /**
-     * Parameter argument.
-     */
-    private static class ParamArgument implements FastUpdateArgument {
-        /** Value to return. */
-        private final int paramIdx;
-
-        /**
-         * Constructor.
-         *
-         * @param paramIdx Parameter index.
-         */
-        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/ec290234/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
deleted file mode 100644
index dc90fe9..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdateArgument.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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/ec290234/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
index 31dc52d..6a45c3c 100644
--- 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
@@ -17,6 +17,10 @@
 
 package org.apache.ignite.internal.processors.query.h2.dml;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -26,6 +30,7 @@ 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.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.UpdateResult;
 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.typedef.F;
@@ -34,10 +39,6 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.h2.table.Column;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
 
 /**
@@ -47,7 +48,7 @@ public final class UpdatePlan {
     /** Initial statement to drive the rest of the logic. */
     private final UpdateMode mode;
 
-    /** Target table to be affected by initial DML statement. */
+    /**  to be affected by initial DML statement. */
     private final GridH2Table tbl;
 
     /** Column names to set or update. */
@@ -74,6 +75,9 @@ public final class UpdatePlan {
     /** Subquery flag - {@code true} if {@link #selectQry} is an actual subquery that retrieves data from some cache. */
     private final boolean isLocSubqry;
 
+    /** Rows for query-less MERGE or INSERT. */
+    private final List<List<DmlArgument>> rows;
+
     /** Number of rows in rows based MERGE or INSERT. */
     private final int rowsNum;
 
@@ -96,6 +100,7 @@ public final class UpdatePlan {
      * @param valColIdx value column index.
      * @param selectQry Select query.
      * @param isLocSubqry Local subquery flag.
+     * @param rows Rows for query-less INSERT or MERGE.
      * @param rowsNum Rows number.
      * @param fastUpdate Fast update (if any).
      * @param distributed Distributed plan (if any)
@@ -111,12 +116,14 @@ public final class UpdatePlan {
         int valColIdx,
         String selectQry,
         boolean isLocSubqry,
+        List<List<DmlArgument>> rows,
         int rowsNum,
         @Nullable FastUpdate fastUpdate,
         @Nullable DmlDistributedPlanInfo distributed
     ) {
         this.colNames = colNames;
         this.colTypes = colTypes;
+        this.rows = rows;
         this.rowsNum = rowsNum;
 
         assert mode != null;
@@ -161,6 +168,7 @@ public final class UpdatePlan {
             -1,
             selectQry,
             false,
+            null,
             0,
             fastUpdate,
             distributed
@@ -339,6 +347,61 @@ public final class UpdatePlan {
     }
 
     /**
+     * Process fast DML operation if possible.
+     *
+     * @param args QUery arguments.
+     * @return Update result or {@code null} if fast update is not applicable for plan.
+     * @throws IgniteCheckedException If failed.
+     */
+    public UpdateResult processFast(Object[] args) throws IgniteCheckedException {
+        if (fastUpdate != null)
+            return fastUpdate.execute(cacheContext().cache(), args);
+
+        return null;
+    }
+
+    /**
+     * @return {@code True} if predefined rows exist.
+     */
+    public boolean hasRows() {
+        return !F.isEmpty(rows);
+    }
+
+    /**
+     * Extract rows from plan without performing any query.
+     * @param args Original query arguments.
+     * @return Rows from plan.
+     * @throws IgniteCheckedException if failed.
+     */
+    public List<List<?>> createRows(Object[] args) throws IgniteCheckedException {
+        assert rowsNum > 0 && !F.isEmpty(colNames);
+
+        List<List<?>> res = new ArrayList<>(rowsNum);
+
+        GridH2RowDescriptor desc = tbl.rowDescriptor();
+
+        for (List<DmlArgument> row : rows) {
+            List<Object> resRow = new ArrayList<>();
+
+            for (int j = 0; j < colNames.length; j++) {
+                Object colVal = row.get(j).get(args);
+
+                if (j == keyColIdx || j == valColIdx) {
+                    Class<?> colCls = j == keyColIdx ? desc.type().keyClass() : desc.type().valueClass();
+
+                    colVal = DmlUtils.convert(colVal, desc, colCls, colTypes[j]);
+                }
+
+                resRow.add(colVal);
+            }
+
+            res.add(resRow);
+        }
+
+        return res;
+    }
+
+    /**
      * @return Update mode.
      */
     public UpdateMode mode() {
@@ -379,11 +442,4 @@ public final class UpdatePlan {
     @Nullable public boolean isLocalSubquery() {
         return isLocSubqry;
     }
-
-    /**
-     * @return Fast update.
-     */
-    @Nullable public FastUpdate fastUpdate() {
-        return fastUpdate;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec290234/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
index a551639..5ffd264 100644
--- 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
@@ -21,6 +21,7 @@ import java.lang.reflect.Constructor;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -127,6 +128,8 @@ public final class UpdatePlanBuilder {
 
         GridH2RowDescriptor desc;
 
+        List<GridSqlElement[]> elRows = null;
+
         if (stmt instanceof GridSqlInsert) {
             GridSqlInsert ins = (GridSqlInsert) stmt;
             target = ins.into();
@@ -136,6 +139,10 @@ public final class UpdatePlanBuilder {
 
             cols = ins.columns();
             sel = DmlAstUtils.selectForInsertOrMerge(cols, ins.rows(), ins.query());
+
+            if (sel == null)
+                elRows = ins.rows();
+
             isTwoStepSubqry = (ins.query() != null);
             rowsNum = isTwoStepSubqry ? 0 : ins.rows().size();
         }
@@ -149,6 +156,10 @@ public final class UpdatePlanBuilder {
 
             cols = merge.columns();
             sel = DmlAstUtils.selectForInsertOrMerge(cols, merge.rows(), merge.query());
+
+            if (sel == null)
+                elRows = merge.rows();
+
             isTwoStepSubqry = (merge.query() != null);
             rowsNum = isTwoStepSubqry ? 0 : merge.rows().size();
         }
@@ -158,7 +169,7 @@ public final class UpdatePlanBuilder {
         }
 
         // Let's set the flag only for subqueries that have their FROM specified.
-        isTwoStepSubqry = (isTwoStepSubqry && (sel instanceof GridSqlUnion ||
+        isTwoStepSubqry &= (sel != null && (sel instanceof GridSqlUnion ||
             (sel instanceof GridSqlSelect && ((GridSqlSelect) sel).from() != null)));
 
         int keyColIdx = -1;
@@ -210,13 +221,33 @@ public final class UpdatePlanBuilder {
         KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true, false);
         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
 
-        String selectSql = sel.getSQL();
+        String selectSql = sel != null ? sel.getSQL() : null;
 
         DmlDistributedPlanInfo distributed = (rowsNum == 0 && !F.isEmpty(selectSql)) ?
             checkPlanCanBeDistributed(idx, conn, fieldsQuery, loc, selectSql, tbl.dataTable().cacheName()) : null;
 
         UpdateMode mode = stmt instanceof GridSqlMerge ? UpdateMode.MERGE : UpdateMode.INSERT;
 
+        List<List<DmlArgument>> rows = null;
+
+        if (elRows != null) {
+            assert sel == null;
+
+            rows = new ArrayList<>(elRows.size());
+
+            for (GridSqlElement[] elRow : elRows) {
+                List<DmlArgument> row = new ArrayList<>(cols.length);
+
+                for (GridSqlElement el : elRow) {
+                    DmlArgument arg = DmlArguments.create(el);
+
+                    row.add(arg);
+                }
+
+                rows.add(row);
+            }
+        }
+
         return new UpdatePlan(
             mode,
             tbl.dataTable(),
@@ -228,6 +259,7 @@ public final class UpdatePlanBuilder {
             valColIdx,
             selectSql,
             !isTwoStepSubqry,
+            rows,
             rowsNum,
             null,
             distributed
@@ -349,6 +381,7 @@ public final class UpdatePlanBuilder {
                     valColIdx,
                     selectSql,
                     false,
+                    null,
                     0,
                     null,
                     distributed