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 2019/01/25 13:55:24 UTC

[ignite] branch master updated: IGNITE-10824: SQL: Restricted mixed use of _KEY and key fields or _VAL and value fields in the same DML statement. This closes #5918.

This is an automated email from the ASF dual-hosted git repository.

vozerov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 58241fb  IGNITE-10824: SQL: Restricted mixed use of _KEY and key fields or _VAL and value fields in the same DML statement. This closes #5918.
58241fb is described below

commit 58241fb6b1552f4b2227bc409765003096dc7f47
Author: Pavel Kuznetsov <pa...@gmail.com>
AuthorDate: Fri Jan 25 16:54:53 2019 +0300

    IGNITE-10824: SQL: Restricted mixed use of _KEY and key fields or _VAL and value fields in the same DML statement. This closes #5918.
---
 .../processors/query/h2/dml/UpdatePlanBuilder.java |  87 ++++++
 .../cache/IgniteCacheInsertSqlQuerySelfTest.java   |  38 ++-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java    |  25 --
 .../cache/IgniteCacheSqlDmlErrorSelfTest.java      | 341 +++++++++++++++++++++
 .../cache/IgniteCacheUpdateSqlQuerySelfTest.java   |  89 ++++--
 .../IgniteBinaryCacheQueryTestSuite.java           |   4 +-
 6 files changed, 515 insertions(+), 69 deletions(-)

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 efccb6c..0c55f54 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
@@ -22,6 +22,7 @@ import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -67,6 +68,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteClosure;
 import org.h2.command.Prepared;
 import org.h2.table.Column;
 import org.jetbrains.annotations.Nullable;
@@ -77,6 +79,10 @@ import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueR
  * Logic for building update plans performed by {@link DmlStatementsProcessor}.
  */
 public final class UpdatePlanBuilder {
+    /** Converter from GridSqlColumn to Column. */
+    private static final IgniteClosure<GridSqlColumn, Column> TO_H2_COL =
+        (IgniteClosure<GridSqlColumn, Column>)GridSqlColumn::column;
+
     /**
      * Constructor.
      */
@@ -310,6 +316,8 @@ public final class UpdatePlanBuilder {
                 hasValProps = true;
         }
 
+        verifyDmlColumns(tbl.dataTable(), F.viewReadOnly(Arrays.asList(cols), TO_H2_COL));
+
         KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true, false);
         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
 
@@ -552,6 +560,8 @@ public final class UpdatePlanBuilder {
 
         String[] colNames = new String[cols.size()];
 
+        Column[] h2Cols = new Column[cols.size()];
+
         int[] colTypes = new int[cols.size()];
 
         int keyColIdx = -1;
@@ -567,6 +577,8 @@ public final class UpdatePlanBuilder {
 
             Column h2Col = tbl.getColumn(colName);
 
+            h2Cols[i] = h2Col;
+
             colTypes[i] = h2Col.getType();
             int colId = h2Col.getColumnId();
 
@@ -590,6 +602,8 @@ public final class UpdatePlanBuilder {
                 hasValProps = true;
         }
 
+        verifyDmlColumns(tbl, Arrays.asList(h2Cols));
+
         KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps,
             true, false);
         KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps,
@@ -783,6 +797,9 @@ public final class UpdatePlanBuilder {
         if (gridTbl != null && updateAffectsKeyColumns(gridTbl, update.set().keySet()))
             throw new IgniteSQLException("SQL UPDATE can't modify key or its fields directly",
                 IgniteQueryErrorCode.KEY_UPDATE);
+
+        if (gridTbl != null)
+            verifyDmlColumns(gridTbl, F.viewReadOnly(update.cols(), TO_H2_COL));
     }
 
     /**
@@ -811,6 +828,76 @@ public final class UpdatePlanBuilder {
         return false;
     }
 
+
+    /**
+     * Checks that DML query (insert, merge, update, bulk load aka copy) columns: <br/>
+     * 1) doesn't contain both entire key (_key or alias) and columns referring to part of the key; <br/>
+     * 2) doesn't contain both entire value (_val or alias) and columns referring to part of the value. <br/>
+     *
+     * @param tab - updated table.
+     * @param affectedCols - table's column names affected by dml query. Their order should be the same as in the
+     * dml statement only to have the same columns order in the error message.
+     * @throws IgniteSQLException if check failed.
+     */
+    private static void verifyDmlColumns(GridH2Table tab, Collection<Column> affectedCols) {
+        GridH2RowDescriptor desc = tab.rowDescriptor();
+
+        // _key (_val) or it alias exist in the update columns.
+        String keyColName = null;
+        String valColName = null;
+
+        // Whether fields that are part of the key (value) exist in the updated columns.
+        boolean hasKeyProps = false;
+        boolean hasValProps = false;
+
+        for (Column col : affectedCols) {
+            int colId = col.getColumnId();
+
+            // At first, let's define whether column refers to entire key, entire value or one of key/val fields.
+            // Checking that it's not specified both _key(_val) and its alias by the way.
+            if (desc.isKeyColumn(colId)) {
+                if (keyColName == null)
+                    keyColName = col.getName();
+                else
+                    throw new IgniteSQLException(
+                        "Columns " + keyColName + " and " + col + " both refer to entire cache key object.",
+                        IgniteQueryErrorCode.PARSING);
+            }
+            else if (desc.isValueColumn(colId)) {
+                if (valColName == null)
+                    valColName = col.getName();
+                else
+                    throw new IgniteSQLException(
+                        "Columns " + valColName + " and " + col + " both refer to entire cache value object.",
+                        IgniteQueryErrorCode.PARSING);
+
+            }
+            else {
+                // Column ids 0..2 are _key, _val, _ver
+                assert colId >= DEFAULT_COLUMNS_COUNT : "Unexpected column [name=" + col + ", id=" + colId + "].";
+
+                if (desc.isColumnKeyProperty(colId - DEFAULT_COLUMNS_COUNT))
+                    hasKeyProps = true;
+                else
+                    hasValProps = true;
+            }
+
+            // And check invariants for the fast fail.
+            boolean hasEntireKeyCol = keyColName != null;
+            boolean hasEntireValcol = valColName != null;
+
+            if (hasEntireKeyCol && hasKeyProps)
+                throw new IgniteSQLException("Column " + keyColName + " refers to entire key cache object. " +
+                    "It must not be mixed with other columns that refer to parts of key.",
+                    IgniteQueryErrorCode.PARSING);
+
+            if (hasEntireValcol && hasValProps)
+                throw new IgniteSQLException("Column " + valColName + " refers to entire value cache object. " +
+                    "It must not be mixed with other columns that refer to parts of value.",
+                    IgniteQueryErrorCode.PARSING);
+        }
+    }
+
     /**
      * Checks whether the given update plan can be distributed and returns additional info.
      *
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
index e14a025..18d1f92 100644
--- 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
@@ -204,22 +204,30 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     }
 
     /**
-     *
+     * Test that nested fields could be updated using sql UPDATE just by nested field name.
      */
     @Test
-    public void testNestedFieldsHandling() {
+    public void testNestedFieldsHandling1() {
         IgniteCache<Integer, AllTypes> p = ignite(0).cache("I2AT");
 
+        final int ROOT_KEY = 1;
+
+        // Create 1st level value
+        AllTypes rootVal = new AllTypes(1L);
+
+        // With random inner field
+        rootVal.innerTypeCol = new AllTypes.InnerType(42L);
+
+
         p.query(new SqlFieldsQuery(
-            "insert into AllTypes(_key, innerTypeCol, arrListCol, _val, innerStrCol) values (1, ?, ?, ?, 'sss')").
-            setArgs(
-                new AllTypes.InnerType(50L),
-                new ArrayList<>(Arrays.asList(3L, 2L, 1L)),
-                new AllTypes(1L)
-            )
-        );
+            "INSERT INTO AllTypes(_key,_val) VALUES (?, ?)").setArgs(ROOT_KEY, rootVal)
+        ).getAll();
+
+        // Update inner fields just by their names
+        p.query(new SqlFieldsQuery("UPDATE AllTypes SET innerLongCol = ?, innerStrCol = ?, arrListCol = ?;")
+            .setArgs(50L, "sss", new ArrayList<>(Arrays.asList(3L, 2L, 1L)))).getAll();
 
-        AllTypes res = p.get(1);
+        AllTypes res = p.get(ROOT_KEY);
 
         AllTypes.InnerType resInner = new AllTypes.InnerType(50L);
 
@@ -235,12 +243,14 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     @Test
     public void testCacheRestartHandling() {
         for (int i = 0; i < 4; i++) {
-            IgniteCache<Integer, IgniteCacheUpdateSqlQuerySelfTest.AllTypes> p =
+            IgniteCache<Integer, AllTypes> p =
                 ignite(0).getOrCreateCache(cacheConfig("I2AT", true, false, Integer.class,
-                    IgniteCacheUpdateSqlQuerySelfTest.AllTypes.class));
+                    AllTypes.class));
+
+            p.query(new SqlFieldsQuery("INSERT INTO AllTypes(_key, _val) VALUES (1, ?)")
+                .setArgs(new AllTypes(1L))).getAll();
 
-            p.query(new SqlFieldsQuery("insert into AllTypes(_key, _val, dateCol) values (1, ?, null)")
-                .setArgs(new IgniteCacheUpdateSqlQuerySelfTest.AllTypes(1L)));
+            p.query(new SqlFieldsQuery("UPDATE AllTypes SET dateCol = null;")).getAll();
 
             p.destroy();
         }
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
index cab0119..d89c60c 100644
--- 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
@@ -17,16 +17,12 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.ArrayList;
-import java.util.Arrays;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-import static org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest.AllTypes;
-
 /**
  *
  */
@@ -129,25 +125,4 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
 
         assertEquals(4, (int)p.get(3));
     }
-
-    /**
-     *
-     */
-    @Test
-    public void testNestedFieldsHandling() {
-        IgniteCache<Integer, AllTypes> p = ignite(0).cache("I2AT");
-
-        p.query(new SqlFieldsQuery("merge into AllTypes(_key, innerTypeCol, arrListCol, _val, innerStrCol) " +
-            "values (1, ?, ?, ?, 'sss')") .setArgs(new AllTypes.InnerType(50L),
-            new ArrayList<>(Arrays.asList(3L, 2L, 1L)), new AllTypes(1L)));
-
-        AllTypes res = p.get(1);
-
-        AllTypes.InnerType resInner = new AllTypes.InnerType(50L);
-
-        resInner.innerStrCol = "sss";
-        resInner.arrListCol = new ArrayList<>(Arrays.asList(3L, 2L, 1L));
-
-        assertEquals(resInner, res.innerTypeCol);
-    }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheSqlDmlErrorSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheSqlDmlErrorSelfTest.java
new file mode 100644
index 0000000..480350a
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheSqlDmlErrorSelfTest.java
@@ -0,0 +1,341 @@
+/*
+ * 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.List;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Negative java API tests for dml queries (insert, merge, update).
+ */
+@RunWith(JUnit4.class)
+public class IgniteCacheSqlDmlErrorSelfTest extends GridCommonAbstractTest {
+    /** Dummy cache, just cache api entry point. */
+    private static IgniteCache<?, ?> cache;
+
+    /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        startGrids(1);
+
+        cache = grid(0).createCache(DEFAULT_CACHE_NAME);
+    }
+
+    /**
+     * Create test tables.
+     */
+    @Before
+    public void dropAdnCreateTables() {
+        execute("DROP TABLE IF EXISTS COMPOSITE;");
+        execute("DROP TABLE IF EXISTS SIMPLE");
+        execute("DROP TABLE IF EXISTS SIMPLE_WRAPPED");
+
+        execute("CREATE TABLE COMPOSITE (id1 INT, id2 INT, name1 VARCHAR, name2 VARCHAR, PRIMARY KEY(id1, id2)) " +
+            "WITH \"key_type=" + CompositeKey.class.getName() + ", value_type=" + CompositeValue.class.getName() + "\"");
+        execute("CREATE TABLE SIMPLE (id INT PRIMARY KEY, name VARCHAR) WITH \"wrap_value=false, wrap_key=false\"");
+        execute("CREATE TABLE SIMPLE_WRAPPED (id INT PRIMARY KEY, name VARCHAR) WITH \"wrap_value=true, wrap_key=true\"");
+
+        execute("INSERT INTO COMPOSITE (_key, _val) VALUES (?, ?)", new CompositeKey(), new CompositeValue());
+        execute("INSERT INTO SIMPLE VALUES (146, 'default name')");
+        execute("INSERT INTO SIMPLE_WRAPPED VALUES (147, 'default name')");
+    }
+
+    /**
+     * Check it's forbidden to specify any two of _key, _key alias or key field (column that belongs to key) together in
+     * the insert/merge dml statement. Same constraints are right for (_val, _val alias, val fields).
+     */
+    @Test
+    public void testInsertMixingPlaceholderAndFields() {
+        assertThrows(() ->
+                execute("INSERT INTO COMPOSITE (_key, id2, name1, name2) VALUES (?, ?, ?, ?)",
+                    new CompositeKey(), 42, "name#1", "name#2"),
+            "Column _KEY refers to entire key cache object.");
+
+        assertThrows(() ->
+                execute("INSERT INTO COMPOSITE (id1, id2, _val, name2) VALUES (?, ?, ?, ?)",
+                    1, 2, new CompositeValue(), "name#2"),
+            "Column _VAL refers to entire value cache object.");
+
+        assertThrows(() ->
+                execute("INSERT INTO SIMPLE (_key, id, name) VALUES (?, ?, ?)", 42, 43, "some name"),
+            "Columns _KEY and ID both refer to entire cache key object.");
+
+        assertThrows(() ->
+                execute("INSERT INTO SIMPLE (_key, _val, name) VALUES (?, ?, ?)", 42, "name#1", "name#2"),
+            "Columns _VAL and NAME both refer to entire cache value object.");
+
+        // And the same asserts for the MERGE:
+        assertThrows(() ->
+                execute("MERGE INTO COMPOSITE (_key, id2, name1, name2) VALUES (?, ?, ?, ?)",
+                    new CompositeKey(), 42, "name#1", "name#2"),
+            "Column _KEY refers to entire key cache object.");
+
+        assertThrows(() ->
+                execute("MERGE INTO COMPOSITE (id1, id2, _val, name2) VALUES (?, ?, ?, ?)",
+                    1, 2, new CompositeValue(), "name#2"),
+            "Column _VAL refers to entire value cache object.");
+
+        assertThrows(() ->
+                execute("MERGE INTO SIMPLE (_key, id, name) VALUES (?, ?, ?)", 42, 43, "some name"),
+            "Columns _KEY and ID both refer to entire cache key object.");
+
+        assertThrows(() ->
+                execute("MERGE INTO SIMPLE (_key, _val, name) VALUES (?, ?, ?)", 42, "name#1", "name#2"),
+            "Columns _VAL and NAME both refer to entire cache value object.");
+    }
+
+    /**
+     * Check it's forbidden to specify any two of _key, _key alias or key field (column that belongs to key) together in
+     * the COPY (aka bulk load) sql statement. Same constraints are right for (_val, _val alias, val fields).
+     */
+    @Test
+    public void testCopyMixingPlaceholderAndFields() {
+        assertThrows(() ->
+                execute("COPY FROM \'stub/file/path\' " +
+                    "INTO SIMPLE (_key, id, name) FORMAT CSV"),
+            "Columns _KEY and ID both refer to entire cache key object.");
+
+        assertThrows(() ->
+                execute("COPY FROM \'stub/file/path\' " +
+                    "INTO SIMPLE_WRAPPED (_key, id, name) FORMAT CSV"),
+            "Column _KEY refers to entire key cache object.");
+
+        assertThrows(() ->
+                execute("COPY FROM \'stub/file/path\' " +
+                    "INTO SIMPLE (id, _val, name) FORMAT CSV"),
+            "Columns _VAL and NAME both refer to entire cache value object.");
+
+        assertThrows(() ->
+                execute("COPY FROM \'stub/file/path\' " +
+                    "INTO SIMPLE_WRAPPED (id, _val, name) FORMAT CSV"),
+            "Column _VAL refers to entire value cache object.");
+
+    }
+
+    /**
+     * Check update statements that modify any two of _val, _val alias or val field (column that belongs to cache value
+     * object) are forbidden.
+     */
+    @Test
+    public void testUpdateMixingValueAndValueFields() {
+        assertThrows(() ->
+                execute("UPDATE COMPOSITE SET _val = ?, name2 = ?",
+                    new CompositeValue(), "name#2"),
+            "Column _VAL refers to entire value cache object.");
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET _val = ?, name = ?",
+                    "name#1", "name#2"),
+            "Columns _VAL and NAME both refer to entire cache value object.");
+    }
+
+    /**
+     * Check that null values for entire key or value are disallowed.
+     */
+    @Test
+    public void testInsertNullKeyValue() {
+        assertThrows(() ->
+                execute("INSERT INTO COMPOSITE (_key, _val) VALUES (?, ?)", null, new CompositeKey()),
+            "Key for INSERT, COPY, or MERGE must not be null");
+
+        assertThrows(() ->
+                execute("INSERT INTO COMPOSITE (_key, _val) VALUES (?, ?)", new CompositeKey(), null),
+            "Value for INSERT, COPY, MERGE, or UPDATE must not be null");
+
+        assertThrows(() ->
+                execute("INSERT INTO SIMPLE (_key, _val) VALUES(?, ?)", null, "name#1"),
+            "Null value is not allowed for column 'ID'");
+
+        assertThrows(() ->
+                execute("INSERT INTO SIMPLE (id, _val) VALUES(?, ?)", null, "name#1"),
+            "Null value is not allowed for column 'ID'");
+
+        assertThrows(() ->
+                execute("INSERT INTO SIMPLE (_key, _val) VALUES(?, ?)", 42, null),
+            "Null value is not allowed for column 'NAME'");
+
+        assertThrows(() ->
+                execute("INSERT INTO SIMPLE (_key, name) VALUES(?, ?)", 42, null),
+            "Null value is not allowed for column 'NAME'");
+
+        // And the same checks for the MERGE:
+        assertThrows(() ->
+                execute("MERGE INTO COMPOSITE (_key, _val) VALUES (?, ?)", null, new CompositeKey()),
+            "Key for INSERT, COPY, or MERGE must not be null");
+
+        assertThrows(() ->
+                execute("MERGE INTO COMPOSITE (_key, _val) VALUES (?, ?)", new CompositeKey(), null),
+            "Value for INSERT, COPY, MERGE, or UPDATE must not be null");
+
+        assertThrows(() ->
+                execute("MERGE INTO SIMPLE (_key, _val) VALUES(?, ?)", null, "name#1"),
+            "Null value is not allowed for column 'ID'");
+
+        assertThrows(() ->
+                execute("MERGE INTO SIMPLE (id, _val) VALUES(?, ?)", null, "name#1"),
+            "Null value is not allowed for column 'ID'");
+
+        assertThrows(() ->
+                execute("MERGE INTO SIMPLE (_key, _val) VALUES(?, ?)", 42, null),
+            "Null value is not allowed for column 'NAME'");
+
+        assertThrows(() ->
+                execute("MERGE INTO SIMPLE (_key, name) VALUES(?, ?)", 42, null),
+            "Null value is not allowed for column 'NAME'");
+    }
+
+    /**
+     * Check that updates of key or key fields are disallowed.
+     */
+    @Test
+    public void testUpdateKey() {
+        assertThrows(() ->
+                execute("UPDATE COMPOSITE SET _key = ?, _val = ?", new CompositeKey(), new CompositeValue()),
+            "SQL UPDATE can't modify key or its fields directly");
+        assertThrows(() ->
+                execute("UPDATE COMPOSITE SET id1 = ?, _val = ?", 42, new CompositeValue()),
+            "SQL UPDATE can't modify key or its fields directly");
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET _key = ?, _val = ?", 42, "simple name"),
+            "SQL UPDATE can't modify key or its fields directly");
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET id = ?, _val = ?", 42, "simple name"),
+            "SQL UPDATE can't modify key or its fields directly");
+    }
+
+    /**
+     * Check that setting entire cache key to {@code null} via sql is forbidden.
+     */
+    @Test
+    public void testUpdateKeyToNull() {
+        // It's ok to assert just fact of failure if we update key to null.
+        // Both reasons (the fact of updating key and setting _key to null) are correct.
+        // Empty string is contained by any exception message.
+        final String ANY_MESSAGE = "";
+
+        assertThrows(() ->
+                execute("UPDATE COMPOSITE SET _key = ?, _val = ?", null, new CompositeValue()),
+            ANY_MESSAGE);
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET id = ?, _val = ?", null, "simple name"),
+            ANY_MESSAGE);
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET id = ?, _val = ?", null, "simple name"),
+            ANY_MESSAGE);
+    }
+
+    /**
+     * Check that setting entire cache value to {@code null} via sql is forbidden.
+     */
+    @Test
+    public void testUpdateValToNull() {
+        assertThrows(() ->
+                execute("UPDATE COMPOSITE SET _val = ?", (Object)null),
+            "New value for UPDATE must not be null");
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET _val = ?", (Object)null),
+            "New value for UPDATE must not be null");
+
+        assertThrows(() ->
+                execute("UPDATE SIMPLE SET name = ?", (Object)null),
+            "New value for UPDATE must not be null");
+    }
+
+    /**
+     * Execute sql query with PUBLIC schema and specified positional arguments of sql query.
+     *
+     * @param sql query.
+     * @param args positional arguments if sql query got ones.
+     * @return fetched result set.
+     */
+    private static List<List<?>> execute(String sql, Object... args) {
+        SqlFieldsQuery qry = new SqlFieldsQuery(sql).setSchema("PUBLIC");
+
+        if (!F.isEmpty(args))
+            qry.setArgs(args);
+
+        return cache.query(qry).getAll();
+    }
+
+    /**
+     * Check that query execution using cache api results to {@code IgniteSqlException} with message, containing
+     * provided expected message.
+     *
+     * @param qryClos closure that performs the query.
+     * @param expErrMsg expected message.
+     */
+    private void assertThrows(Callable<?> qryClos, String expErrMsg) {
+        GridTestUtils.assertThrows(
+            log(),
+            qryClos,
+            IgniteSQLException.class,
+            expErrMsg);
+    }
+
+    /**
+     * Class which instance can be (de)serialized to(from) key object.
+     */
+    private static class CompositeKey {
+        /** First key field. */
+        int id1;
+
+        /** Second key field. */
+        int id2;
+
+        /** Constructs key with random fields. */
+        public CompositeKey() {
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            id1 = rnd.nextInt();
+            id2 = rnd.nextInt();
+        }
+    }
+
+    /**
+     * Class which instance can be (de)serialized to(from) value object.
+     */
+    private static class CompositeValue {
+        /** First value field. */
+        String name1;
+
+        /** Second value field. */
+        String name2;
+
+        /** Creates value with random fields. */
+        public CompositeValue() {
+            name1 = UUID.randomUUID().toString();
+            name2 = UUID.randomUUID().toString();
+        }
+    }
+}
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
index d943f9d..ed217b7 100644
--- 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
@@ -128,36 +128,40 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
     }
 
     /**
-     *
+     * Test that nested fields could be updated using sql UPDATE just by nested field name.
      */
     @Test
-    public void testUpdateValueAndFields() {
-        IgniteCache p = cache();
+    public void testNestedFieldsUpdate() {
+        IgniteCache<Long, AllTypes> p = ignite(0).cache("L2AT");
 
-        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set id = ?, _val = ? where _key = ?")
-            .setArgs(44, createPerson(2, "Jo", "Woo"), "FirstKey"));
+        final long ROOT_KEY = 1;
 
-        c.iterator();
+        // Create 1st level value
+        AllTypes rootVal = new AllTypes(1L);
 
-        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by _key, id"));
+        // With random inner field
+        rootVal.innerTypeCol = new AllTypes.InnerType(42L);
 
-        List<List<?>> leftovers = c.getAll();
+        p.query(new SqlFieldsQuery(
+            "INSERT INTO \"AllTypes\"(_key,_val) VALUES (?, ?)").setArgs(ROOT_KEY, rootVal)
+        ).getAll();
 
-        assertEquals(4, leftovers.size());
+        // Update inner fields just by their names
+        p.query(new SqlFieldsQuery("UPDATE \"AllTypes\" " +
+            "SET \"innerLongCol\" = ?, \"innerStrCol\" = ?, \"arrListCol\" = ?;"
+        ).setArgs(50L, "sss", new ArrayList<>(Arrays.asList(3L, 2L, 1L)))).getAll();
 
-        assertEqualsCollections(Arrays.asList("FirstKey", createPerson(44, "Jo", "Woo"), 44, "Jo", "Woo"),
-            leftovers.get(0));
+        AllTypes res = p.get(ROOT_KEY);
 
-        assertEqualsCollections(Arrays.asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
-            leftovers.get(1));
+        AllTypes.InnerType resInner = new AllTypes.InnerType(50L);
 
-        assertEqualsCollections(Arrays.asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
-            leftovers.get(2));
+        resInner.innerStrCol = "sss";
+        resInner.arrListCol = new ArrayList<>(Arrays.asList(3L, 2L, 1L));
 
-        assertEqualsCollections(Arrays.asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
-            leftovers.get(3));
+        assertEquals(resInner, res.innerTypeCol);
     }
 
+
     /**
      *
      */
@@ -165,8 +169,9 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
     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"));
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery(
+            "UPDATE Person p SET id = DEFAULT, firstName = ?, secondName = ? WHERE _key = ?"
+        ).setArgs( "Jo", "Woo", "FirstKey"));
 
         c.iterator();
 
@@ -194,17 +199,35 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
     public void testTypeConversions() throws ParseException {
         IgniteCache cache = ignite(0).cache("L2AT");
 
-        cache.query(new SqlFieldsQuery("insert into \"AllTypes\"(_key, _val, \"dateCol\", \"booleanCol\"," +
-            "\"tsCol\") values(2, ?, '2016-11-30 12:00:00', false, DATE '2016-12-01')").setArgs(new AllTypes(2L)));
+        cache.query(new SqlFieldsQuery("INSERT INTO \"AllTypes\" (_key, _val) VALUES(2, ?)")
+            .setArgs(new AllTypes(2L))).getAll();
+
+        cache.query (new SqlFieldsQuery(
+            "UPDATE \"AllTypes\" " +
+                "SET " +
+                "\"dateCol\" = '2016-11-30 12:00:00', " +
+                "\"booleanCol\" = false, " +
+                "\"tsCol\" = DATE '2016-12-01' " +
+                "WHERE _key = 2")
+        );
 
         // Look ma, no hands: first we set value of inner object column (innerTypeCol), then update only one of its
         // fields (innerLongCol), while leaving another inner property (innerStrCol) as specified by innerTypeCol.
-        cache.query(new SqlFieldsQuery("update \"AllTypes\" set \"innerLongCol\" = ?, \"doubleCol\" = CAST('50' as INT)," +
-            " \"booleanCol\" = 80, \"innerTypeCol\" = ?, \"strCol\" = PI(), \"shortCol\" = " +
-            "CAST(WEEK(PARSEDATETIME('2016-11-30', 'yyyy-MM-dd')) as VARCHAR), " +
-            "\"sqlDateCol\"=TIMESTAMP '2016-12-02 13:47:00', \"tsCol\"=TIMESTAMPADD('MI', 2, " +
-            "DATEADD('DAY', 2, \"tsCol\")), \"primitiveIntsCol\" = ?, \"bytesCol\" = ?")
-            .setArgs(5, new AllTypes.InnerType(80L), new int[] {2, 3}, new Byte[] {4, 5, 6}));
+        cache.query(new SqlFieldsQuery(
+                "UPDATE \"AllTypes\" " +
+                    "SET " +
+                    "\"innerLongCol\" = ?, " +  // (1)
+                    "\"doubleCol\" = CAST('50' as INT), " +
+                    "\"booleanCol\" = 80, " +
+                    "\"innerTypeCol\" = ?, " + // (2)
+                    "\"strCol\" = PI(), " +
+                    "\"shortCol\" = CAST(WEEK(PARSEDATETIME('2016-11-30', 'yyyy-MM-dd')) as VARCHAR), " +
+                    "\"sqlDateCol\"=TIMESTAMP '2016-12-02 13:47:00', " +
+                    "\"tsCol\"=TIMESTAMPADD('MI', 2, DATEADD('DAY', 2, \"tsCol\")), " +
+                    "\"primitiveIntsCol\" = ?, " +  //(3)
+                    "\"bytesCol\" = ?" // (4)
+            ).setArgs(5, new AllTypes.InnerType(80L), new int[] {2, 3}, new Byte[] {4, 5, 6})
+        ).getAll();
 
         AllTypes res = (AllTypes) cache.get(2L);
 
@@ -239,8 +262,16 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
     public void testSingleInnerFieldUpdate() throws ParseException {
         IgniteCache cache = ignite(0).cache("L2AT");
 
-        cache.query(new SqlFieldsQuery("insert into \"AllTypes\"(_key, _val, \"dateCol\", \"booleanCol\") values(2, ?," +
-            "'2016-11-30 12:00:00', false)").setArgs(new AllTypes(2L)));
+        cache.query(new SqlFieldsQuery("insert into \"AllTypes\" (_key, _val) values(2, ?)")
+            .setArgs(new AllTypes(2L))).getAll();
+
+        cache.query(new SqlFieldsQuery(
+            "UPDATE \"AllTypes\" " +
+                "SET " +
+                "\"dateCol\" = '2016-11-30 12:00:00', " +
+                "\"booleanCol\" = false " +
+                "WHERE _key = 2")
+        ).getAll();
 
         assertFalse(cache.query(new SqlFieldsQuery("select * from \"AllTypes\"")).getAll().isEmpty());
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 0584c8d..e3547e6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -82,6 +82,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCachePrimitiveFieldsQue
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryH2IndexingLeakTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheSqlDmlErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheUnionDuplicatesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest;
@@ -291,9 +292,10 @@ import org.junit.runners.Suite;
     GridH2IndexingInMemSelfTest.class,
     GridH2IndexingOffheapSelfTest.class,
 
-    // Parsing
+        // Parsing
     GridQueryParsingTest.class,
     IgniteCacheSqlQueryErrorSelfTest.class,
+    IgniteCacheSqlDmlErrorSelfTest.class,
 
     // Config.
     IgniteCacheDuplicateEntityConfigurationSelfTest.class,