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:07 UTC

[03/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/sql/GridSqlKeyword.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
new file mode 100644
index 0000000..8eacf02
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
@@ -0,0 +1,46 @@
+/*
+ * 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.Collections;
+import org.h2.command.Parser;
+import org.h2.expression.ValueExpression;
+
+/** Keyword (like DEFAULT). */
+public final class GridSqlKeyword extends GridSqlElement {
+    /**
+     * Default update value - analogous to H2.
+     * @see ValueExpression#getDefault()
+     * @see Parser#parseUpdate()
+     */
+    public final static GridSqlKeyword DEFAULT = new GridSqlKeyword("DEFAULT");
+
+    /** Keyword to return as SQL. */
+    private final String keyword;
+
+    /** */
+    private GridSqlKeyword(String keyword) {
+        super(Collections.<GridSqlElement>emptyList());
+        this.keyword = keyword;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return keyword;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
new file mode 100644
index 0000000..4cdb314
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlMerge.java
@@ -0,0 +1,143 @@
+/*
+ * 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 GridSqlMerge extends GridSqlStatement {
+    /** */
+    private GridSqlElement into;
+
+    /** */
+    private GridSqlColumn[] cols;
+
+    /** */
+    private GridSqlColumn[] keys;
+
+    /** */
+    private List<GridSqlElement[]> rows;
+
+    /** Insert subquery. */
+    private GridSqlQuery qry;
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("MERGE INTO ")
+            .append(into.getSQL())
+            .append("(");
+
+        for (GridSqlColumn col : cols) {
+            buff.appendExceptFirst(", ");
+            buff.append('\n')
+                .append(col.getSQL());
+        }
+        buff.append("\n)\n");
+
+        if (keys != null) {
+            buff.append("KEY(\n");
+            buff.resetCount();
+            for (GridSqlColumn c : keys) {
+                buff.appendExceptFirst(", ");
+                buff.append(c.getSQL())
+                    .append('\n');
+            }
+            buff.append(")\n");
+        }
+
+        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 GridSqlMerge into(GridSqlElement from) {
+        this.into = from;
+        return this;
+    }
+
+    /** */
+    public List<GridSqlElement[]> rows() {
+        return rows;
+    }
+
+    /** */
+    public GridSqlMerge rows(List<GridSqlElement[]> rows) {
+        assert rows != null;
+        this.rows = rows;
+        return this;
+    }
+
+    /** */
+    public GridSqlQuery query() {
+        return qry;
+    }
+
+    /** */
+    public GridSqlMerge query(GridSqlQuery qry) {
+        this.qry = qry;
+        return this;
+    }
+
+    /** */
+    public GridSqlColumn[] columns() {
+        return cols;
+    }
+
+    /** */
+    public GridSqlMerge columns(GridSqlColumn[] cols) {
+        this.cols = cols;
+        return this;
+    }
+
+    /** */
+    public GridSqlColumn[] keys() {
+        return keys;
+    }
+
+    /** */
+    public GridSqlMerge keys(GridSqlColumn[] keys) {
+        this.keys = keys;
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
index d9784c8..331aa65 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
@@ -25,7 +25,7 @@ import org.h2.util.StringUtils;
 /**
  * Select query.
  */
-public abstract class GridSqlQuery {
+public abstract class GridSqlQuery extends GridSqlStatement {
     /** */
     protected boolean distinct;
 
@@ -35,29 +35,6 @@ public abstract class GridSqlQuery {
     /** */
     protected GridSqlElement offset;
 
-    /** */
-    protected GridSqlElement limit;
-
-    /** */
-    private boolean explain;
-
-    /**
-     * @param explain Explain.
-     * @return {@code this}.
-     */
-    public GridSqlQuery explain(boolean explain) {
-        this.explain = explain;
-
-        return this;
-    }
-
-    /**
-     * @return {@code true} If explain.
-     */
-    public boolean explain() {
-        return explain;
-    }
-
     /**
      * @return Offset.
      */
@@ -73,20 +50,6 @@ public abstract class GridSqlQuery {
     }
 
     /**
-     * @param limit Limit.
-     */
-    public void limit(GridSqlElement limit) {
-        this.limit = limit;
-    }
-
-    /**
-     * @return Limit.
-     */
-    public GridSqlElement limit() {
-        return limit;
-    }
-
-    /**
      * @return Distinct.
      */
     public boolean distinct() {
@@ -101,11 +64,6 @@ public abstract class GridSqlQuery {
     }
 
     /**
-     * @return Generate sql.
-     */
-    public abstract String getSQL();
-
-    /**
      * @return Sort.
      */
     public List<GridSqlSortColumn> sort() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index a7451c1..d9c546c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -19,17 +19,27 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.lang.reflect.Field;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.h2.command.Command;
 import org.h2.command.CommandContainer;
 import org.h2.command.Prepared;
+import org.h2.command.dml.Delete;
 import org.h2.command.dml.Explain;
+import org.h2.command.dml.Insert;
+import org.h2.command.dml.Merge;
 import org.h2.command.dml.Query;
 import org.h2.command.dml.Select;
 import org.h2.command.dml.SelectUnion;
+import org.h2.command.dml.Update;
 import org.h2.engine.FunctionAlias;
 import org.h2.expression.Aggregate;
 import org.h2.expression.Alias;
@@ -224,6 +234,64 @@ public class GridSqlQueryParser {
     private static final Getter<Explain, Prepared> EXPLAIN_COMMAND = getter(Explain.class, "command");
 
     /** */
+    private static final Getter<Merge, Table> MERGE_TABLE = getter(Merge.class, "table");
+
+    /** */
+    private static final Getter<Merge, Column[]> MERGE_COLUMNS = getter(Merge.class, "columns");
+
+    /** */
+    private static final Getter<Merge, Column[]> MERGE_KEYS = getter(Merge.class, "keys");
+
+    /** */
+    private static final Getter<Merge, List<Expression[]>> MERGE_ROWS = getter(Merge.class, "list");
+
+    /** */
+    private static final Getter<Merge, Query> MERGE_QUERY = getter(Merge.class, "query");
+
+    /** */
+    private static final Getter<Insert, Table> INSERT_TABLE = getter(Insert.class, "table");
+
+    /** */
+    private static final Getter<Insert, Column[]> INSERT_COLUMNS = getter(Insert.class, "columns");
+
+    /** */
+    private static final Getter<Insert, List<Expression[]>> INSERT_ROWS = getter(Insert.class, "list");
+
+    /** */
+    private static final Getter<Insert, Query> INSERT_QUERY = getter(Insert.class, "query");
+
+    /** */
+    private static final Getter<Insert, Boolean> INSERT_DIRECT = getter(Insert.class, "insertFromSelect");
+
+    /** */
+    private static final Getter<Insert, Boolean> INSERT_SORTED = getter(Insert.class, "sortedInsertMode");
+
+    /** */
+    private static final Getter<Delete, TableFilter> DELETE_FROM = getter(Delete.class, "tableFilter");
+
+    /** */
+    private static final Getter<Delete, Expression> DELETE_WHERE = getter(Delete.class, "condition");
+
+    /** */
+    private static final Getter<Delete, Expression> DELETE_LIMIT = getter(Delete.class, "limitExpr");
+
+    /** */
+    private static final Getter<Update, TableFilter> UPDATE_TARGET = getter(Update.class, "tableFilter");
+
+    /** */
+    private static final Getter<Update, ArrayList<Column>> UPDATE_COLUMNS = getter(Update.class, "columns");
+
+    /** */
+    private static final Getter<Update, HashMap<Column, Expression>> UPDATE_SET = getter(Update.class,
+        "expressionMap");
+
+    /** */
+    private static final Getter<Update, Expression> UPDATE_WHERE = getter(Update.class, "condition");
+
+    /** */
+    private static final Getter<Update, Expression> UPDATE_LIMIT = getter(Update.class, "limitExpr");
+
+    /** */
     private static final Getter<Command, Prepared> PREPARED =
         GridSqlQueryParser.<Command, Prepared>getter(CommandContainer.class, "prepared");
 
@@ -249,14 +317,32 @@ public class GridSqlQueryParser {
         GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(filter);
 
         if (res == null) {
-            Table tbl = filter.getTable();
+            res = parseTable(filter.getTable(), filter.getIndex(),
+                filter.getSelect() != null ? filter.getSelect().getSQL() : null);
 
+            String alias = ALIAS.get(filter);
+
+            if (alias != null)
+                res = new GridSqlAlias(alias, res, false);
+
+            h2ObjToGridObj.put(filter, res);
+        }
+
+        return res;
+    }
+
+
+    /**
+     * @param tbl Table.
+     */
+    private GridSqlElement parseTable(Table tbl, @Nullable Index idx, String sql) {
+        GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(tbl);
+
+        if (res == null) {
             if (tbl instanceof TableBase)
                 res = new GridSqlTable(tbl);
             else if (tbl instanceof TableView) {
-                Query qry = VIEW_QUERY.get((TableView)tbl);
-
-                Index idx = filter.getIndex();
+                Query qry = VIEW_QUERY.get((TableView) tbl);
 
                 Query idxQry = idx instanceof ViewIndex ? ((ViewIndex)idx).getQuery() : null;
 
@@ -271,14 +357,10 @@ public class GridSqlQueryParser {
                 res.addChild(parseExpression(RANGE_MAX.get((RangeTable)tbl), false));
             }
             else
-                assert0(false, filter.getSelect().getSQL());
-
-            String alias = ALIAS.get(filter);
-
-            if (alias != null)
-                res = new GridSqlAlias(alias, res, false);
+                assert0(false, "Unexpected Table implementation [cls=" + tbl.getClass().getSimpleName() + ", " +
+                    "sql=" + sql + ']');
 
-            h2ObjToGridObj.put(filter, res);
+            h2ObjToGridObj.put(tbl, res);
         }
 
         return res;
@@ -348,6 +430,171 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param merge Merge.
+     * @see <a href="http://h2database.com/html/grammar.html#merge">H2 merge spec</a>
+     */
+    private GridSqlMerge parseMerge(Merge merge) {
+        GridSqlMerge res = (GridSqlMerge)h2ObjToGridObj.get(merge);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlMerge();
+        h2ObjToGridObj.put(merge, res);
+
+        Table srcTbl = MERGE_TABLE.get(merge);
+        GridSqlElement tbl = parseTable(srcTbl, null, merge.getSQL());
+
+        res.into(tbl);
+
+        Column[] srcCols = MERGE_COLUMNS.get(merge);
+
+        GridSqlColumn[] cols = new GridSqlColumn[srcCols.length];
+
+        for (int i = 0; i < srcCols.length; i++) {
+            cols[i] = new GridSqlColumn(srcCols[i], tbl, srcCols[i].getName(), srcCols[i].getSQL());
+
+            cols[i].resultType(fromColumn(srcCols[i]));
+        }
+
+        res.columns(cols);
+
+        Column[] srcKeys = MERGE_KEYS.get(merge);
+
+        GridSqlColumn[] keys = new GridSqlColumn[srcKeys.length];
+        for (int i = 0; i < srcKeys.length; i++)
+            keys[i] = new GridSqlColumn(srcKeys[i], tbl, srcKeys[i].getName(), srcKeys[i].getSQL());
+        res.keys(keys);
+
+        List<Expression[]> srcRows = MERGE_ROWS.get(merge);
+        if (!srcRows.isEmpty()) {
+            List<GridSqlElement[]> rows = new ArrayList<>(srcRows.size());
+            for (Expression[] srcRow : srcRows) {
+                GridSqlElement[] row = new GridSqlElement[srcRow.length];
+
+                for (int i = 0; i < srcRow.length; i++)
+                    row[i] = parseExpression(srcRow[i], false);
+
+                rows.add(row);
+            }
+            res.rows(rows);
+        }
+        else {
+            res.rows(Collections.<GridSqlElement[]>emptyList());
+            res.query(parse(MERGE_QUERY.get(merge), null));
+        }
+
+        return res;
+    }
+
+    /**
+     * @param insert Insert.
+     * @see <a href="http://h2database.com/html/grammar.html#insert">H2 insert spec</a>
+     */
+    private GridSqlInsert parseInsert(Insert insert) {
+        GridSqlInsert res = (GridSqlInsert)h2ObjToGridObj.get(insert);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlInsert();
+        h2ObjToGridObj.put(insert, res);
+
+        Table srcTbl = INSERT_TABLE.get(insert);
+        GridSqlElement tbl = parseTable(srcTbl, null, insert.getSQL());
+
+        res.into(tbl).
+            direct(INSERT_DIRECT.get(insert)).
+            sorted(INSERT_SORTED.get(insert));
+
+        Column[] srcCols = INSERT_COLUMNS.get(insert);
+        GridSqlColumn[] cols = new GridSqlColumn[srcCols.length];
+
+        for (int i = 0; i < srcCols.length; i++) {
+            cols[i] = new GridSqlColumn(srcCols[i], tbl, srcCols[i].getName(), srcCols[i].getSQL());
+
+            cols[i].resultType(fromColumn(srcCols[i]));
+        }
+
+        res.columns(cols);
+
+        List<Expression[]> srcRows = INSERT_ROWS.get(insert);
+        if (!srcRows.isEmpty()) {
+            List<GridSqlElement[]> rows = new ArrayList<>(srcRows.size());
+            for (Expression[] srcRow : srcRows) {
+                GridSqlElement[] row = new GridSqlElement[srcRow.length];
+
+                for (int i = 0; i < srcRow.length; i++)
+                    row[i] = parseExpression(srcRow[i], false);
+
+                rows.add(row);
+            }
+            res.rows(rows);
+        }
+        else {
+            res.rows(Collections.<GridSqlElement[]>emptyList());
+            res.query(parse(INSERT_QUERY.get(insert), null));
+        }
+
+        return res;
+    }
+
+    /**
+     * @param del Delete.
+     * @see <a href="http://h2database.com/html/grammar.html#delete">H2 delete spec</a>
+     */
+    private GridSqlDelete parseDelete(Delete del) {
+        GridSqlDelete res = (GridSqlDelete)h2ObjToGridObj.get(del);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlDelete();
+        h2ObjToGridObj.put(del, res);
+
+        GridSqlElement tbl = parseTable(DELETE_FROM.get(del));
+        GridSqlElement where = parseExpression(DELETE_WHERE.get(del), true);
+        GridSqlElement limit = parseExpression(DELETE_LIMIT.get(del), true);
+        res.from(tbl).where(where).limit(limit);
+        return res;
+    }
+
+    /**
+     * @param update Update.
+     * @see <a href="http://h2database.com/html/grammar.html#update">H2 update spec</a>
+     */
+    private GridSqlUpdate parseUpdate(Update update) {
+        GridSqlUpdate res = (GridSqlUpdate)h2ObjToGridObj.get(update);
+
+        if (res != null)
+            return res;
+
+        res = new GridSqlUpdate();
+        h2ObjToGridObj.put(update, res);
+
+        GridSqlElement tbl = parseTable(UPDATE_TARGET.get(update));
+
+        List<Column> srcCols = UPDATE_COLUMNS.get(update);
+        Map<Column, Expression> srcSet = UPDATE_SET.get(update);
+
+        ArrayList<GridSqlColumn> cols = new ArrayList<>(srcCols.size());
+        LinkedHashMap<String, GridSqlElement> set = new LinkedHashMap<>(srcSet.size());
+
+        for (Column c : srcCols) {
+            GridSqlColumn col = new GridSqlColumn(c, tbl, c.getName(), c.getSQL());
+            col.resultType(fromColumn(c));
+            cols.add(col);
+            set.put(col.columnName(), parseExpression(srcSet.get(c), true));
+        }
+
+        GridSqlElement where = parseExpression(UPDATE_WHERE.get(update), true);
+        GridSqlElement limit = parseExpression(UPDATE_LIMIT.get(update), true);
+
+        res.target(tbl).cols(cols).set(set).where(where).limit(limit);
+        return res;
+    }
+
+    /**
      * @param sortOrder Sort order.
      * @param qry Query.
      */
@@ -387,26 +634,56 @@ public class GridSqlQueryParser {
      * @param qry Prepared.
      * @return Query.
      */
-    public GridSqlQuery parse(Prepared qry) {
+    public GridSqlStatement parse(Prepared qry) {
         return parse(qry, null);
     }
 
     /**
      * @param qry Select.
      */
-    public GridSqlQuery parse(Prepared qry, @Nullable Query idxQry) {
+    public GridSqlStatement parse(Prepared qry, @Nullable Query idxQry) {
         assert qry != null;
 
+        if (qry instanceof Query)
+            return parse((Query)qry, idxQry);
+
+        if (qry instanceof Merge)
+            return parseMerge((Merge)qry);
+
+        if (qry instanceof Insert)
+            return parseInsert((Insert)qry);
+
+        if (qry instanceof Delete)
+            return parseDelete((Delete)qry);
+
+        if (qry instanceof Update)
+            return parseUpdate((Update)qry);
+
+        if (qry instanceof Explain) {
+            GridSqlStatement stmt = parse(EXPLAIN_COMMAND.get((Explain) qry));
+
+            if (!(stmt instanceof GridSqlQuery))
+                throw new IgniteSQLException("EXPLAIN is not supported for DML statement: " + qry,
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+            return stmt.explain(true);
+        }
+
+        throw new IgniteSQLException("Unsupported statement: " + qry, IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    /**
+     * @param qry Query.
+     * @return Grid SQL query.
+     */
+    private GridSqlQuery parse(Query qry, @Nullable Query idxQry) {
         if (qry instanceof Select)
             return parse((Select)qry, idxQry);
 
         if (qry instanceof SelectUnion)
             return parse((SelectUnion)qry);
 
-        if (qry instanceof Explain)
-            return parse(EXPLAIN_COMMAND.get((Explain)qry)).explain(true);
-
-        throw new CacheException("Unsupported query: " + qry);
+        throw new UnsupportedOperationException("Unknown query type: " + qry);
     }
 
     /**
@@ -420,8 +697,8 @@ public class GridSqlQueryParser {
 
         res = new GridSqlUnion();
 
-        res.right(parse(union.getRight()));
-        res.left(parse(union.getLeft()));
+        res.right(parse(union.getRight(), null));
+        res.left(parse(union.getLeft(), null));
 
         res.unionType(union.getUnionType());
 
@@ -478,7 +755,9 @@ public class GridSqlQueryParser {
                 parseExpression(expression.getNonAliasExpression(), calcTypes), true);
 
         if (expression instanceof ValueExpression)
-            return new GridSqlConst(expression.getValue(null));
+            // == comparison is legit, see ValueExpression#getSQL()
+            return expression == ValueExpression.getDefault() ? GridSqlKeyword.DEFAULT :
+                new GridSqlConst(expression.getValue(null));
 
         if (expression instanceof Operation) {
             Operation operation = (Operation)expression;
@@ -535,7 +814,7 @@ public class GridSqlQueryParser {
 
             assert0(qry instanceof Select, expression);
 
-            return new GridSqlSubquery(parse((Select)qry));
+            return new GridSqlSubquery(parse(qry, null));
         }
 
         if (expression instanceof ConditionIn) {
@@ -579,7 +858,7 @@ public class GridSqlQueryParser {
 
             assert0(qry instanceof Select, qry);
 
-            res.addChild(new GridSqlSubquery(parse((Select)qry)));
+            res.addChild(new GridSqlSubquery(parse(qry, null)));
 
             return res;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 7d43bf6..09952cf 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -165,7 +165,11 @@ public class GridSqlQuerySplitter {
 
         final Prepared prepared = prepared(stmt);
 
-        GridSqlQuery qry = new GridSqlQueryParser().parse(prepared);
+        GridSqlStatement gridStmt = new GridSqlQueryParser().parse(prepared);
+
+        assert gridStmt instanceof GridSqlQuery;
+
+        GridSqlQuery qry = (GridSqlQuery) gridStmt;
 
         qry = collectAllTables(qry, schemas, tbls);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
index a38ae68..f49a714 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
@@ -74,7 +74,8 @@ public class GridSqlSelect extends GridSqlQuery {
             buff.append(expression.getSQL());
         }
 
-        buff.append("\nFROM ").append(from.getSQL());
+        if (from != null)
+            buff.append("\nFROM ").append(from.getSQL());
 
         if (where != null)
             buff.append("\nWHERE ").append(StringUtils.unEnclose(where.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/GridSqlStatement.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java
new file mode 100644
index 0000000..6eda0d7
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlStatement.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+/**
+ * SQL statement to query or update grid caches.
+ */
+public abstract class GridSqlStatement {
+    /** */
+    protected GridSqlElement limit;
+    /** */
+    private boolean explain;
+
+    /**
+     * @return Generate sql.
+     */
+    public abstract String getSQL();
+
+    /**
+     * @param explain Explain.
+     * @return {@code this}.
+     */
+    public GridSqlStatement explain(boolean explain) {
+        this.explain = explain;
+
+        return this;
+    }
+
+    /**
+     * @return {@code true} If explain.
+     */
+    public boolean explain() {
+        return explain;
+    }
+
+    /**
+     * @param limit Limit.
+     */
+    public void limit(GridSqlElement limit) {
+        this.limit = limit;
+    }
+
+    /**
+     * @return Limit.
+     */
+    public GridSqlElement limit() {
+        return limit;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java
new file mode 100644
index 0000000..1623d3e
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUpdate.java
@@ -0,0 +1,105 @@
+/*
+ * 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.LinkedHashMap;
+import org.h2.util.StatementBuilder;
+import org.h2.util.StringUtils;
+
+/** */
+public class GridSqlUpdate extends GridSqlStatement {
+    /** */
+    private GridSqlElement target;
+
+    /** */
+    private ArrayList<GridSqlColumn> cols;
+
+    /** */
+    private LinkedHashMap<String, GridSqlElement> set;
+
+    /** */
+    private GridSqlElement where;
+
+    /** */
+    public GridSqlUpdate target(GridSqlElement target) {
+        this.target = target;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement target() {
+        return target;
+    }
+
+    /** */
+    public GridSqlUpdate cols(ArrayList<GridSqlColumn> cols) {
+        this.cols = cols;
+        return this;
+    }
+
+    /** */
+    public ArrayList<GridSqlColumn> cols() {
+        return cols;
+    }
+
+
+    /** */
+    public GridSqlUpdate set(LinkedHashMap<String, GridSqlElement> set) {
+        this.set = set;
+        return this;
+    }
+
+    /** */
+    public GridSqlUpdate where(GridSqlElement where) {
+        this.where = where;
+        return this;
+    }
+
+    /** */
+    public GridSqlElement where() {
+        return where;
+    }
+
+    /** */
+    public LinkedHashMap<String, GridSqlElement> set() {
+        return set;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN " : "");
+        buff.append("UPDATE ")
+            .append(target.getSQL())
+            .append("\nSET\n");
+
+        for (GridSqlColumn c : cols) {
+            GridSqlElement e = set.get(c.columnName());
+            buff.appendExceptFirst(",\n    ");
+            buff.append(c.columnName()).append(" = ").append(e != null ? e.getSQL() : "DEFAULT");
+        }
+
+        if (where != null)
+            buff.append("\nWHERE ").append(StringUtils.unEnclose(where.getSQL()));
+
+        if (limit != null)
+            buff.append("\nLIMIT ").append(StringUtils.unEnclose(limit.getSQL()));
+
+        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/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 48567da..0eaed12 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -65,7 +65,6 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
-import org.apache.ignite.internal.processors.query.h2.GridH2ResultSetIterator;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
@@ -715,7 +714,7 @@ public class GridReduceQueryExecutor {
                                 timeoutMillis,
                                 cancel);
 
-                            resIter = new Iter(res);
+                            resIter = new IgniteH2Indexing.FieldsIterator(res);
                         }
                         finally {
                             GridH2QueryContext.clearThreadLocal();
@@ -1377,31 +1376,6 @@ public class GridReduceQueryExecutor {
     /**
      *
      */
-    private static class Iter extends GridH2ResultSetIterator<List<?>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * @param data Data array.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected Iter(ResultSet data) throws IgniteCheckedException {
-            super(data, true, false);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected List<?> createRow() {
-            ArrayList<Object> res = new ArrayList<>(row.length);
-
-            Collections.addAll(res, row);
-
-            return res;
-        }
-    }
-
-    /**
-     *
-     */
     private class ExplicitPartitionsSpecializer implements IgniteBiClosure<ClusterNode,Message,Message> {
         /** */
         private final Map<ClusterNode,IntArray> partsMap;

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
new file mode 100644
index 0000000..2c3ab23
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
@@ -0,0 +1,567 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryAbstractIdentityResolver;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryFieldIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.IgniteTestResources;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected final Marshaller marsh;
+
+    /**
+     *
+     */
+    IgniteCacheAbstractInsertSqlQuerySelfTest() {
+        try {
+            marsh = IgniteTestResources.getMarshaller();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * @return whether {@link #marsh} is an instance of {@link BinaryMarshaller} or not.
+     */
+    boolean isBinaryMarshaller() {
+        return marsh instanceof BinaryMarshaller;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        binCfg.setTypeConfigurations(Arrays.asList(
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key.class.getName());
+
+                setIdentityResolver(BinaryArrayIdentityResolver.instance());
+            }},
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key2.class.getName());
+
+                setIdentityResolver(BinaryArrayIdentityResolver.instance());
+            }},
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key3.class.getName());
+
+                setIdentityResolver(new BinaryFieldIdentityResolver().setFieldNames("key"));
+            }},
+            new BinaryTypeConfiguration() {{
+                setTypeName(Key4.class.getName());
+
+                setIdentityResolver(new Key4Id());
+            }}
+        ));
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3, false);
+
+        if (!isBinaryMarshaller())
+            createCaches();
+        else
+            createBinaryCaches();
+    }
+
+    /**
+     *
+     */
+    protected void createCaches() {
+        ignite(0).createCache(cacheConfig("S2P", true, false, String.class, Person.class, String.class, String.class));
+        ignite(0).createCache(cacheConfig("I2P", true, false, Integer.class, Person.class));
+        ignite(0).createCache(cacheConfig("K2P", true, false, Key.class, Person.class));
+        ignite(0).createCache(cacheConfig("K22P", true, true, Key2.class, Person2.class));
+        ignite(0).createCache(cacheConfig("I2I", true, false, Integer.class, Integer.class));
+    }
+
+    /**
+     *
+     */
+    final void createBinaryCaches() {
+        {
+            CacheConfiguration s2pCcfg = cacheConfig("S2P", true, false);
+
+            QueryEntity s2p = new QueryEntity(String.class.getName(), "Person");
+
+            s2p.setKeyFields(Collections.<String>emptySet());
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            s2p.setFields(flds);
+
+            s2p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            QueryEntity s2s = new QueryEntity(String.class.getName(), String.class.getName());
+
+            s2s.setKeyFields(Collections.<String>emptySet());
+
+            s2pCcfg.setQueryEntities(Arrays.asList(s2p, s2s));
+
+            ignite(0).createCache(s2pCcfg);
+        }
+
+        {
+            CacheConfiguration i2pCcfg = cacheConfig("I2P", true, false);
+
+            QueryEntity i2p = new QueryEntity(Integer.class.getName(), "Person");
+
+            i2p.setKeyFields(Collections.<String>emptySet());
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            i2p.setFields(flds);
+
+            i2p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            i2pCcfg.setQueryEntities(Collections.singletonList(i2p));
+
+            ignite(0).createCache(i2pCcfg);
+        }
+
+        {
+            CacheConfiguration k2pCcfg = cacheConfig("K2P", true, false);
+
+            QueryEntity k2p = new QueryEntity(Key.class.getName(), "Person");
+
+            k2p.setKeyFields(Collections.singleton("key"));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("key", Integer.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            k2p.setFields(flds);
+
+            k2p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k2pCcfg.setQueryEntities(Collections.singletonList(k2p));
+
+            ignite(0).createCache(k2pCcfg);
+        }
+
+        {
+            CacheConfiguration k22pCcfg = cacheConfig("K22P", true, true);
+
+            QueryEntity k22p = new QueryEntity(Key2.class.getName(), "Person2");
+
+            k22p.setKeyFields(Collections.singleton("Id"));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("Id", Integer.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+            flds.put("_Val", Integer.class.getName());
+
+            k22p.setFields(flds);
+
+            k22p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k22pCcfg.setQueryEntities(Collections.singletonList(k22p));
+
+            ignite(0).createCache(k22pCcfg);
+        }
+
+        {
+            CacheConfiguration k32pCcfg = cacheConfig("K32P", true, false);
+
+            QueryEntity k32p = new QueryEntity(Key3.class.getName(), "Person");
+
+            k32p.setKeyFields(new HashSet<>(Arrays.asList("key", "strKey")));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("key", Integer.class.getName());
+            flds.put("strKey", String.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            k32p.setFields(flds);
+
+            k32p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k32pCcfg.setQueryEntities(Collections.singletonList(k32p));
+
+            ignite(0).createCache(k32pCcfg);
+        }
+
+        {
+            CacheConfiguration k42pCcfg = cacheConfig("K42P", true, false);
+
+            QueryEntity k42p = new QueryEntity(Key4.class.getName(), "Person");
+
+            k42p.setKeyFields(new HashSet<>(Arrays.asList("key", "strKey")));
+
+            LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+            flds.put("key", Integer.class.getName());
+            flds.put("strKey", String.class.getName());
+            flds.put("id", Integer.class.getName());
+            flds.put("name", String.class.getName());
+
+            k42p.setFields(flds);
+
+            k42p.setIndexes(Collections.<QueryIndex>emptyList());
+
+            k42pCcfg.setQueryEntities(Collections.singletonList(k42p));
+
+            ignite(0).createCache(k42pCcfg);
+        }
+
+        {
+            CacheConfiguration i2iCcfg = cacheConfig("I2I", true, false);
+
+            QueryEntity i2i = new QueryEntity(Integer.class.getName(), Integer.class.getName());
+
+            i2i.setKeyFields(Collections.<String>emptySet());
+
+            i2i.setFields(new LinkedHashMap<String, String>());
+
+            i2i.setIndexes(Collections.<QueryIndex>emptyList());
+
+            i2iCcfg.setQueryEntities(Collections.singletonList(i2i));
+
+            ignite(0).createCache(i2iCcfg);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        ignite(0).cache("S2P").clear();
+        ignite(0).cache("I2P").clear();
+        ignite(0).cache("K2P").clear();
+        ignite(0).cache("K22P").clear();
+        ignite(0).cache("I2I").clear();
+
+        if (isBinaryMarshaller()) {
+            ignite(0).cache("K32P").clear();
+            ignite(0).cache("K42P").clear();
+        }
+
+        super.afterTest();
+    }
+
+    /**
+     *
+     */
+    Object createPerson(int id, String name) {
+        if (!isBinaryMarshaller()) {
+            Person p = new Person(id);
+            p.name = name;
+
+            return p;
+        }
+        else {
+            BinaryObjectBuilder o = grid(0).binary().builder("Person");
+            o.setField("id", id);
+            o.setField("name", name);
+
+            return o.build();
+        }
+    }
+
+    /**
+     *
+     */
+    Object createPerson2(int id, String name, int valFld) {
+        if (!isBinaryMarshaller()) {
+            Person2 p = new Person2(id);
+            p.name = name;
+            p._Val = valFld;
+
+            return p;
+        }
+        else {
+            BinaryObjectBuilder o = grid(0).binary().builder("Person2");
+            o.setField("id", id);
+            o.setField("name", name);
+            o.setField("_Val", valFld);
+
+            return o.build();
+        }
+    }
+
+    /**
+     * @param name Cache name.
+     * @param partitioned Partition or replicated cache.
+     * @param escapeSql whether identifiers should be quoted - see {@link CacheConfiguration#setSqlEscapeAll}
+     * @param idxTypes Indexed types.
+     * @return Cache configuration.
+     */
+    private static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql, Class<?>... idxTypes) {
+        return new CacheConfiguration()
+            .setName(name)
+            .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED)
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setBackups(1)
+            .setSqlEscapeAll(escapeSql)
+            .setIndexedTypes(idxTypes);
+    }
+
+    /**
+     *
+     */
+    protected final static class Key implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key(int key) {
+            this.key = key;
+        }
+
+        /** */
+        @QuerySqlField
+        public final int key;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Key key1 = (Key) o;
+
+            return key == key1.key;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key;
+        }
+    }
+
+    /**
+     *
+     */
+    protected final static class Key2 implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key2(int Id) {
+            this.Id = Id;
+        }
+
+        /** */
+        @QuerySqlField
+        public final int Id;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Key2 key1 = (Key2) o;
+
+            return Id == key1.Id;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Id;
+        }
+    }
+
+    /**
+     *
+     */
+    final static class Key3 implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key3(int key) {
+            this.key = key;
+            this.strKey = Integer.toString(key);
+        }
+
+        /** */
+        @QuerySqlField
+        public final int key;
+
+        /** */
+        @QuerySqlField
+        public final String strKey;
+    }
+
+    /**
+     *
+     */
+    final static class Key4 implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        public Key4(int key) {
+            this.key = key;
+            this.strKey = Integer.toString(key);
+        }
+
+        /** */
+        @QuerySqlField
+        public final int key;
+
+        /** */
+        @QuerySqlField
+        public final String strKey;
+    }
+
+    /**
+     *
+     */
+    final static class Key4Id extends BinaryAbstractIdentityResolver {
+        /** {@inheritDoc} */
+        @Override protected int hashCode0(BinaryObject obj) {
+            return (int) obj.field("key") * 100;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) {
+            return (int) o1.field("key") == (int) o2.field("key");
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Person implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        @SuppressWarnings("unused")
+        private Person() {
+            // No-op.
+        }
+
+        /** */
+        public Person(int id) {
+            this.id = id;
+        }
+
+        /** */
+        @QuerySqlField
+        protected int id;
+
+        /** */
+        @QuerySqlField
+        protected String name;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Person person = (Person) o;
+
+            if (id != person.id) return false;
+            return name != null ? name.equals(person.name) : person.name == null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = id;
+            result = 31 * result + (name != null ? name.hashCode() : 0);
+            return result;
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Person2 extends Person {
+        /** */
+        @SuppressWarnings("unused")
+        private Person2() {
+            // No-op.
+        }
+
+        /** */
+        public Person2(int id) {
+            super(id);
+        }
+
+        /** */
+        @QuerySqlField
+        public int _Val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
new file mode 100644
index 0000000..2dbf1b4
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.IgniteTestResources;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected final Marshaller marsh;
+
+    /**
+     *
+     */
+    IgniteCacheAbstractSqlDmlQuerySelfTest() {
+        try {
+            marsh = IgniteTestResources.getMarshaller();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * @return whether {@link #marsh} is an instance of {@link BinaryMarshaller} or not.
+     */
+    private boolean isBinaryMarshaller() {
+        return marsh instanceof BinaryMarshaller;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3, true);
+
+        ignite(0).createCache(cacheConfig());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+        ignite(0).cache("S2P").put("FirstKey", createPerson(1, "John", "White"));
+        ignite(0).cache("S2P").put("SecondKey", createPerson(2, "Joe", "Black"));
+        ignite(0).cache("S2P").put("k3", createPerson(3, "Sylvia", "Green"));
+        ignite(0).cache("S2P").put("f0u4thk3y", createPerson(4, "Jane", "Silver"));
+    }
+
+    /** */
+    Object createPerson(int id, String name, String secondName) {
+        if (!isBinaryMarshaller())
+            return new Person(id, name, secondName);
+        else {
+            BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
+
+            bldr.setField("id", id);
+            bldr.setField("name", name);
+            bldr.setField("secondName", secondName);
+
+            return bldr.build();
+        }
+
+    }
+
+    /** */
+    protected IgniteCache<?, ?> cache() {
+        return ignite(0).cache("S2P").withKeepBinary();
+    }
+
+    /** */
+    protected CacheConfiguration cacheConfig() {
+        if (!isBinaryMarshaller())
+            return cacheConfig("S2P", true, false).setIndexedTypes(String.class, Person.class);
+        else
+            return createBinCacheConfig();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @param name Cache name.
+     * @param partitioned Partition or replicated cache.
+     * @param escapeSql whether identifiers should be quoted - see {@link CacheConfiguration#setSqlEscapeAll}
+     * @return Cache configuration.
+     */
+    private static CacheConfiguration cacheConfig(String name, boolean partitioned, boolean escapeSql) {
+        return new CacheConfiguration()
+            .setName(name)
+            .setCacheMode(partitioned ? CacheMode.PARTITIONED : CacheMode.REPLICATED)
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setBackups(1)
+            .setSqlEscapeAll(escapeSql);
+    }
+
+    /**
+     *
+     */
+    private static CacheConfiguration createBinCacheConfig() {
+        CacheConfiguration ccfg = cacheConfig("S2P", true, false);
+
+        QueryEntity e = new QueryEntity(String.class.getName(), "Person");
+
+        e.setKeyFields(Collections.<String>emptySet());
+
+        LinkedHashMap<String, String> flds = new LinkedHashMap<>();
+
+        flds.put("id", Integer.class.getName());
+        flds.put("name", String.class.getName());
+        flds.put("secondName", String.class.getName());
+
+        e.setFields(flds);
+
+        e.setIndexes(Collections.<QueryIndex>emptyList());
+
+        ccfg.setQueryEntities(Collections.singletonList(e));
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    private static class Person implements Serializable {
+        /** */
+        public Person(int id, String name, String secondName) {
+            this.id = id;
+            this.name = name;
+            this.secondName = secondName;
+        }
+
+        /** */
+        @QuerySqlField
+        protected int id;
+
+        /** */
+        @QuerySqlField
+        protected final String name;
+
+        /** */
+        @QuerySqlField
+        final String secondName;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Person person = (Person) o;
+
+            return id == person.id && name.equals(person.name) && secondName.equals(person.secondName);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = id;
+            res = 31 * res + name.hashCode();
+            res = 31 * res + secondName.hashCode();
+            return res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
new file mode 100644
index 0000000..8b0a033
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDeleteSqlQuerySelfTest.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheDeleteSqlQuerySelfTest extends IgniteCacheAbstractSqlDmlQuerySelfTest {
+    /**
+     *
+     */
+    public void testDeleteSimple() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("delete from Person p where length(p._key) = 2 " +
+            "or p.secondName like '%ite'"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(2, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(1));
+    }
+
+    /**
+     *
+     */
+    public void testDeleteSingle() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("delete from Person where _key = ?")
+            .setArgs("FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id, _key"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(3, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
new file mode 100644
index 0000000..4f87740
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsertSqlQuerySelfTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    public void testInsertWithExplicitKey() {
+        IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("insert into Person (_key, id, name) values ('s', ?, ?), " +
+            "('a', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get("s"));
+
+        assertEquals(createPerson(2, "Alex"), p.get("a"));
+    }
+
+    /**
+     *
+     */
+    public void testInsertFromSubquery() {
+        IgniteCache p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("insert into String (_key, _val) values ('s', ?), " +
+            "('a', ?)").setArgs("Sergi", "Alex").setLocal(true));
+
+        assertEquals("Sergi", p.get("s"));
+        assertEquals("Alex", p.get("a"));
+
+        p.query(new SqlFieldsQuery("insert into Person(_key, id, name) " +
+            "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
+
+        assertEquals(createPerson(5, "Sergi"), p.get("se"));
+
+        assertEquals(createPerson(4, "Alex"), p.get("al"));
+    }
+
+    /**
+     *
+     */
+    public void testInsertWithExplicitPrimitiveKey() {
+        IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (_key, id, name) values (cast('1' as int), ?, ?), (2, (5 - 3), 'Alex')")
+            .setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(1));
+
+        assertEquals(createPerson(2, "Alex"), p.get(2));
+    }
+
+    /**
+     *
+     */
+    public void testInsertWithDynamicKeyInstantiation() {
+        IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key(2)));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsCaseSensitivity() {
+        IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"name\", \"_Val\") values (1, ?, ?, 5), " +
+            "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
+
+        assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
+
+        assertEquals(createPerson2(3, "Alex", 6), p.get(new Key2(2)));
+    }
+
+    /**
+     *
+     */
+    public void testPrimitives() {
+        IgniteCache<Integer, Integer> p = ignite(0).cache("I2I");
+
+        p.query(new SqlFieldsQuery("insert into Integer(_key, _val) values (1, ?), " +
+            "(?, 4)").setArgs(2, 3));
+
+        assertEquals(2, (int)p.get(1));
+
+        assertEquals(4, (int)p.get(3));
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDuplicateKeysException() {
+        final IgniteCache<Integer, Integer> p = ignite(0).cache("I2I");
+
+        p.clear();
+
+        p.put(3, 5);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            /** {@inheritDoc} */
+            @Override public Void call() throws Exception {
+                p.query(new SqlFieldsQuery("insert into Integer(_key, _val) values (1, ?), " +
+                    "(?, 4), (5, 6)").setArgs(2, 3));
+
+                return null;
+            }
+        }, CacheException.class, "Failed to INSERT some keys because they are already in cache [keys=[3]]");
+
+        assertEquals(2, (int)p.get(1));
+        assertEquals(5, (int)p.get(3));
+        assertEquals(6, (int)p.get(5));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsListIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key3(2)));
+    }
+
+    /**
+     *
+     */
+    public void testCustomIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key4(2)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
new file mode 100644
index 0000000..e487564
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertSqlQuerySelfTest {
+    /**
+     *
+     */
+    public void testMergeWithExplicitKey() {
+        IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into Person (_key, id, name) values ('s', ?, ?), " +
+            "('a', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get("s"));
+
+        assertEquals(createPerson(2, "Alex"), p.get("a"));
+    }
+
+    /**
+     *
+     */
+    public void testMergeFromSubquery() {
+        IgniteCache p = ignite(0).cache("S2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into String (_key, _val) values ('s', ?), " +
+            "('a', ?)").setArgs("Sergi", "Alex").setLocal(true));
+
+        assertEquals("Sergi", p.get("s"));
+        assertEquals("Alex", p.get("a"));
+
+        p.query(new SqlFieldsQuery("merge into Person(_key, id, name) " +
+            "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
+
+        assertEquals(createPerson(5, "Sergi"), p.get("se"));
+
+        assertEquals(createPerson(4, "Alex"), p.get("al"));
+    }
+
+    /**
+     *
+     */
+    public void testMergeWithExplicitPrimitiveKey() {
+        IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (_key, id, name) values (cast(? as int), ?, ?), (2, (5 - 3), 'Alex')")
+            .setArgs("1", 1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(1));
+
+        assertEquals(createPerson(2, "Alex"), p.get(2));
+    }
+
+    /**
+     *
+     */
+    public void testMergeWithDynamicKeyInstantiation() {
+        IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key(2)));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsCaseSensitivity() {
+        IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"name\", \"_Val\") values (1, ?, ?, 5), " +
+            "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
+
+        assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
+
+        assertEquals(createPerson2(3, "Alex", 6), p.get(new Key2(2)));
+    }
+
+    /**
+     *
+     */
+    public void testPrimitives() {
+        IgniteCache<Integer, Integer> p = ignite(0).cache("I2I").withKeepBinary();
+
+        p.query(new SqlFieldsQuery("merge into Integer(_key, _val) values (1, ?), " +
+            "(?, 4)").setArgs(2, 3));
+
+        assertEquals(2, (int)p.get(1));
+
+        assertEquals(4, (int)p.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testFieldsListIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key3(2)));
+    }
+
+    /**
+     *
+     */
+    public void testCustomIdentity() {
+        if (!isBinaryMarshaller())
+            return;
+
+        IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
+
+        p.query(new SqlFieldsQuery(
+            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+
+        assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
+
+        assertEquals(createPerson(2, "Alex"), p.get(new Key4(2)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
new file mode 100644
index 0000000..538141f
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDmlQuerySelfTest {
+    /**
+     *
+     */
+    public void testUpdateSimple() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set p.id = p.id * 2, p.name = " +
+            "substring(p.name, 0, 2) where length(p._key) = ? or p.secondName like ?").setArgs(2, "%ite"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(2, "Jo", "White"), 2, "Jo", "White"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(6, "Sy", "Green"), 6, "Sy", "Green"),
+            leftovers.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testUpdateSingle() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set _val = ? where _key = ?")
+            .setArgs(createPerson(2, "Jo", "White"), "FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by id, _key"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(2, "Jo", "White"), 2, "Jo", "White"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testUpdateValueAndFields() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set id = ?, _val = ? where _key = ?")
+            .setArgs(44, createPerson(2, "Jo", "Woo"), "FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(44, "Jo", "Woo"), 44, "Jo", "Woo"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(3));
+    }
+
+    /**
+     *
+     */
+    public void testDefault() {
+        IgniteCache p = cache();
+
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set id = DEFAULT, _val = ? where _key = ?")
+            .setArgs(createPerson(2, "Jo", "Woo"), "FirstKey"));
+
+        c.iterator();
+
+        c = p.query(new SqlFieldsQuery("select * from Person order by _key, id"));
+
+        List<List<?>> leftovers = c.getAll();
+
+        assertEquals(4, leftovers.size());
+
+        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(0, "Jo", "Woo"), 0, "Jo", "Woo"),
+            leftovers.get(0));
+
+        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
+            leftovers.get(1));
+
+        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
+            leftovers.get(2));
+
+        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
+            leftovers.get(3));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
index d9c4ac2..ef111e3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheAtomicFieldsQuerySelfTest.java
@@ -17,12 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
-import java.util.List;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.util.typedef.X;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 
@@ -39,21 +35,4 @@ public class IgniteCacheAtomicFieldsQuerySelfTest extends IgniteCachePartitioned
     @Override protected NearCacheConfiguration nearConfiguration() {
         return null;
     }
-
-    /**
-     *
-     */
-    public void testUnsupportedOperations() {
-        try {
-            QueryCursor<List<?>> qry = grid(0).cache(null).query(new SqlFieldsQuery(
-                "update Person set name = ?").setArgs("Mary Poppins"));
-
-            qry.getAll();
-
-            fail("We don't support updates.");
-        }
-        catch (Exception e) {
-            X.println("___ " + e.getMessage());
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index b4abbf6..512001f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -542,7 +542,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         /**
          * @return Space name.
          */
-        public String space() {
+        String space() {
             return space;
         }
 
@@ -554,18 +554,30 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         /** {@inheritDoc} */
         @Override public GridQueryProperty property(final String name) {
             return new GridQueryProperty() {
+                /** */
                 @Override public Object value(Object key, Object val) throws IgniteCheckedException {
                     return TypeDesc.this.value(name, key, val);
                 }
 
+                /** */
+                @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+                    throw new UnsupportedOperationException();
+                }
+
+                /** */
                 @Override public String name() {
                     return name;
                 }
 
-                @Override
-                public Class<?> type() {
+                /** */
+                @Override public Class<?> type() {
                     return Object.class;
                 }
+
+                /** */
+                @Override public boolean key() {
+                    return false;
+                }
             };
         }
 
@@ -584,6 +596,18 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             return null;
         }
 
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void setValue(String field, Object key, Object val, Object propVal) throws IgniteCheckedException {
+            assert !F.isEmpty(field);
+
+            assert key instanceof Integer;
+
+            Map<String, Object> m = (Map<String, Object>)val;
+
+            m.put(field, propVal);
+        }
+
         /** */
         @Override public Map<String, GridQueryIndexDescriptor> indexes() {
             return textIdx == null ? Collections.<String, GridQueryIndexDescriptor>emptyMap() :
@@ -601,6 +625,16 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** */
+        @Override public String keyTypeName() {
+            return null;
+        }
+
+        /** */
+        @Override public String valueTypeName() {
+            return null;
+        }
+
+        /** */
         @Override public boolean valueTextIndex() {
             return textIdx == null;
         }