You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/02/15 10:44:34 UTC

[09/50] [abbrv] ignite git commit: IGNITE-4363: SQL: fixed inner property updates for DML operations.

IGNITE-4363: SQL: fixed inner property updates for DML operations.


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

Branch: refs/heads/ignite-3477-merge2.0
Commit: 70cd8e452b94b806a2fe6fe00b4b67ce80eb4373
Parents: 0726d32
Author: Alexander Paschenko <al...@gmail.com>
Authored: Tue Feb 7 11:41:08 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Feb 7 11:41:08 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  16 ++-
 .../processors/query/GridQueryProcessor.java    |  81 ++++++++++----
 .../processors/query/GridQueryProperty.java     |  21 ++--
 .../query/h2/DmlStatementsProcessor.java        |  69 ++++++------
 .../query/h2/dml/UpdatePlanBuilder.java         | 105 +++++++++++++------
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |   4 +
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   2 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  22 ++++
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  24 +++++
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  63 +++++++++--
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   5 +
 11 files changed, 301 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 0656dda..f0179ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -2358,9 +2358,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
                     prop.parent(parent);
 
-                    processAnnotation(key, sqlAnn, txtAnn, field.getType(), prop, type);
-
+                    // Add parent property before its possible nested properties so that
+                    // resulting parent column comes before columns corresponding to those
+                    // nested properties in the resulting table - that way nested
+                    // properties override will happen properly (first parent, then children).
                     type.addProperty(prop, key, true);
+
+                    processAnnotation(key, sqlAnn, txtAnn, field.getType(), prop, type);
                 }
             }
 
@@ -2380,9 +2384,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
                     prop.parent(parent);
 
-                    processAnnotation(key, sqlAnn, txtAnn, mtd.getReturnType(), prop, type);
-
+                    // Add parent property before its possible nested properties so that
+                    // resulting parent column comes before columns corresponding to those
+                    // nested properties in the resulting table - that way nested
+                    // properties override will happen properly (first parent, then children).
                     type.addProperty(prop, key, true);
+
+                    processAnnotation(key, sqlAnn, txtAnn, mtd.getReturnType(), prop, type);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index a239ee2..5bfdde8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryObjectEx;
+import org.apache.ignite.internal.binary.BinaryObjectExImpl;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
@@ -1948,7 +1949,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * Description of type property.
      */
-    private static class ClassProperty extends GridQueryProperty {
+    private static class ClassProperty implements GridQueryProperty {
         /** */
         private final PropertyAccessor accessor;
 
@@ -2041,12 +2042,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public String toString() {
             return S.toString(ClassProperty.class, this);
         }
+
+        /** {@inheritDoc} */
+        @Override public GridQueryProperty parent() {
+            return parent;
+        }
     }
 
     /**
      *
      */
-    private class BinaryProperty extends GridQueryProperty {
+    private class BinaryProperty implements GridQueryProperty {
         /** Property name. */
         private String propName;
 
@@ -2130,11 +2136,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 obj = isKeyProp0 == 1 ? key : val;
             }
 
-            assert obj instanceof BinaryObject;
-
-            BinaryObject obj0 = (BinaryObject)obj;
+            if (obj instanceof BinaryObject) {
+                BinaryObject obj0 = (BinaryObject) obj;
+                return fieldValue(obj0);
+            }
+            else if (obj instanceof BinaryObjectBuilder) {
+                BinaryObjectBuilder obj0 = (BinaryObjectBuilder)obj;
 
-            return fieldValue(obj0);
+                return obj0.getField(name());
+            }
+            else
+                throw new IgniteCheckedException("Unexpected binary object class [type=" + obj.getClass() + ']');
         }
 
         /** {@inheritDoc} */
@@ -2144,10 +2156,38 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (obj == null)
                 return;
 
+            Object srcObj = obj;
+
+            if (!(srcObj instanceof BinaryObjectBuilder))
+                throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+            if (parent != null)
+                obj = parent.value(key, val);
+
+            boolean needsBuild = false;
+
+            if (obj instanceof BinaryObjectExImpl) {
+                if (parent == null)
+                    throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+                needsBuild = true;
+
+                obj = ((BinaryObjectExImpl)obj).toBuilder();
+            }
+
             if (!(obj instanceof BinaryObjectBuilder))
                 throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
 
             setValue0((BinaryObjectBuilder) obj, propName, propVal, type());
+
+            if (needsBuild) {
+                obj = ((BinaryObjectBuilder) obj).build();
+
+                assert parent != null;
+
+                // And now let's set this newly constructed object to parent
+                setValue0((BinaryObjectBuilder) srcObj, parent.propName, obj, obj.getClass());
+            }
         }
 
         /**
@@ -2224,6 +2264,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             return isKeyProp0 == 1;
         }
+
+        /** {@inheritDoc} */
+        @Override public GridQueryProperty parent() {
+            return parent;
+        }
     }
 
     /**
@@ -2307,7 +2352,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public GridQueryProperty property(String name) {
-            return getProperty(name);
+            GridQueryProperty res = props.get(name);
+
+            if (res == null)
+                res = uppercaseProps.get(name.toUpperCase());
+
+            return res;
         }
 
         /** {@inheritDoc} */
@@ -2315,7 +2365,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
             assert field != null;
 
-            GridQueryProperty prop = getProperty(field);
+            GridQueryProperty prop = property(field);
 
             if (prop == null)
                 throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
@@ -2329,7 +2379,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throws IgniteCheckedException {
             assert field != null;
 
-            GridQueryProperty prop = getProperty(field);
+            GridQueryProperty prop = property(field);
 
             if (prop == null)
                 throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
@@ -2469,19 +2519,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             fields.put(name, prop.type());
         }
 
-        /**
-         * @param field Property name.
-         * @return Property with given field name.
-         */
-        private GridQueryProperty getProperty(String field) {
-            GridQueryProperty res = props.get(field);
-
-            if (res == null)
-                res = uppercaseProps.get(field.toUpperCase());
-
-            return res;
-        }
-
         /** {@inheritDoc} */
         @Override public boolean valueTextIndex() {
             return valTextIdx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
index 5d74a2e..fb4c037 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
@@ -22,11 +22,7 @@ import org.apache.ignite.IgniteCheckedException;
 /**
  * Description and access method for query entity field.
  */
-public abstract class GridQueryProperty {
-    /** */
-    public GridQueryProperty() {
-    }
-
+public interface GridQueryProperty {
     /**
      * Gets this property value from the given object.
      *
@@ -35,7 +31,7 @@ public abstract class GridQueryProperty {
      * @return Property value.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract Object value(Object key, Object val) throws IgniteCheckedException;
+    public Object value(Object key, Object val) throws IgniteCheckedException;
 
     /**
      * Sets this property value for the given object.
@@ -45,21 +41,26 @@ public abstract class GridQueryProperty {
      * @param propVal Property value.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException;
+    public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException;
 
     /**
      * @return Property name.
      */
-    public abstract String name();
+    public String name();
 
     /**
      * @return Class member type.
      */
-    public abstract Class<?> type();
+    public Class<?> type();
 
     /**
      * Property ownership flag.
      * @return {@code true} if this property belongs to key, {@code false} if it belongs to value.
      */
-    public abstract boolean key();
+    public boolean key();
+
+    /**
+     * @return Parent property or {@code null} if this property is not nested.
+     */
+    public GridQueryProperty parent();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 4030758..7995083 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -59,7 +59,6 @@ import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.dml.FastUpdateArguments;
-import org.apache.ignite.internal.processors.query.h2.dml.KeyValueSupplier;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan;
 import org.apache.ignite.internal.processors.query.h2.dml.UpdatePlanBuilder;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
@@ -481,7 +480,6 @@ public class DmlStatementsProcessor {
         while (it.hasNext()) {
             List<?> e = it.next();
             Object key = e.get(0);
-            Object val = (hasNewVal ? e.get(valColIdx) : e.get(1));
 
             Object newVal;
 
@@ -500,9 +498,6 @@ public class DmlStatementsProcessor {
             if (newVal == null)
                 throw new IgniteSQLException("New value for UPDATE must not be null", IgniteQueryErrorCode.NULL_VALUE);
 
-            if (bin && !(val instanceof BinaryObject))
-                val = cctx.grid().binary().toBinary(val);
-
             // Skip key and value - that's why we start off with 2nd column
             for (int i = 0; i < plan.tbl.getColumns().length - 2; i++) {
                 Column c = plan.tbl.getColumn(i + 2);
@@ -514,13 +509,10 @@ public class DmlStatementsProcessor {
 
                 boolean hasNewColVal = newColVals.containsKey(c.getName());
 
-                // Binary objects get old field values from the Builder, so we can skip what we're not updating
-                if (bin && !hasNewColVal)
+                if (!hasNewColVal)
                     continue;
 
-                // Column values that have been explicitly specified have priority over field values in old or new _val
-                // If no value given for the column, then we expect to find it in value, and not in key - hence null arg.
-                Object colVal = hasNewColVal ? newColVals.get(c.getName()) : prop.value(null, val);
+                Object colVal = newColVals.get(c.getName());
 
                 // UPDATE currently does not allow to modify key or its fields, so we must be safe to pass null as key.
                 desc.setColumnValue(null, newVal, colVal, i);
@@ -696,8 +688,8 @@ public class DmlStatementsProcessor {
 
         // If we have just one item to put, just do so
         if (plan.rowsNum == 1) {
-            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.colTypes, plan.keySupplier,
-                plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next(),
+                plan);
 
             cctx.cache().put(t.getKey(), t.getValue());
             return 1;
@@ -709,8 +701,7 @@ public class DmlStatementsProcessor {
             for (Iterator<List<?>> it = cursor.iterator(); it.hasNext();) {
                 List<?> row = it.next();
 
-                IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.colTypes, plan.keySupplier, plan.valSupplier,
-                    plan.keyColIdx, plan.valColIdx, desc);
+                IgniteBiTuple t = rowToKeyValue(cctx, row, plan);
 
                 rows.put(t.getKey(), t.getValue());
 
@@ -742,8 +733,7 @@ public class DmlStatementsProcessor {
 
         // If we have just one item to put, just do so
         if (plan.rowsNum == 1) {
-            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next().toArray(), plan.colNames, plan.colTypes,
-                plan.keySupplier, plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
+            IgniteBiTuple t = rowToKeyValue(cctx, cursor.iterator().next(), plan);
 
             if (cctx.cache().putIfAbsent(t.getKey(), t.getValue()))
                 return 1;
@@ -768,8 +758,7 @@ public class DmlStatementsProcessor {
             while (it.hasNext()) {
                 List<?> row = it.next();
 
-                final IgniteBiTuple t = rowToKeyValue(cctx, row.toArray(), plan.colNames, plan.colTypes, plan.keySupplier,
-                    plan.valSupplier, plan.keyColIdx, plan.valColIdx, desc);
+                final IgniteBiTuple t = rowToKeyValue(cctx, row, plan);
 
                 rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
 
@@ -837,21 +826,14 @@ public class DmlStatementsProcessor {
      * Convert row presented as an array of Objects into key-value pair to be inserted to cache.
      * @param cctx Cache context.
      * @param row Row to process.
-     * @param cols Query cols.
-     * @param colTypes Column types to convert data from {@code row} to.
-     * @param keySupplier Key instantiation method.
-     * @param valSupplier Key instantiation method.
-     * @param keyColIdx Key column index, or {@code -1} if no key column is mentioned in {@code cols}.
-     * @param valColIdx Value column index, or {@code -1} if no value column is mentioned in {@code cols}.
-     * @param rowDesc Row descriptor.
+     * @param plan Update plan.
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions", "ResultOfMethodCallIgnored"})
-    private IgniteBiTuple<?, ?> rowToKeyValue(GridCacheContext cctx, Object[] row, String[] cols,
-        int[] colTypes, KeyValueSupplier keySupplier, KeyValueSupplier valSupplier, int keyColIdx, int valColIdx,
-        GridH2RowDescriptor rowDesc) throws IgniteCheckedException {
-        Object key = keySupplier.apply(F.asList(row));
-        Object val = valSupplier.apply(F.asList(row));
+    private IgniteBiTuple<?, ?> rowToKeyValue(GridCacheContext cctx, List<?> row, UpdatePlan plan)
+        throws IgniteCheckedException {
+        Object key = plan.keySupplier.apply(row);
+        Object val = plan.valSupplier.apply(row);
 
         if (key == null)
             throw new IgniteSQLException("Key for INSERT or MERGE must not be null",  IgniteQueryErrorCode.NULL_KEY);
@@ -859,13 +841,32 @@ public class DmlStatementsProcessor {
         if (val == null)
             throw new IgniteSQLException("Value for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_VALUE);
 
-        GridQueryTypeDescriptor desc = rowDesc.type();
+        GridQueryTypeDescriptor desc = plan.tbl.rowDescriptor().type();
+
+        Map<String, Object> newColVals = new HashMap<>();
+
+        for (int i = 0; i < plan.colNames.length; i++) {
+            if (i == plan.keyColIdx || i == plan.valColIdx)
+                continue;
+
+            newColVals.put(plan.colNames[i], convert(row.get(i), plan.colNames[i],
+                plan.tbl.rowDescriptor(), plan.colTypes[i]));
+        }
+
+        // We update columns in the order specified by the table for a reason - table's
+        // column order preserves their precedence for correct update of nested properties.
+        Column[] cols = plan.tbl.getColumns();
 
-        for (int i = 0; i < cols.length; i++) {
-            if (i == keyColIdx || i == valColIdx)
+        // First 2 columns are _key and _val, skip 'em.
+        for (int i = 2; i < cols.length; i++) {
+            String colName = cols[i].getName();
+
+            if (!newColVals.containsKey(colName))
                 continue;
 
-            desc.setValue(cols[i], key, val, convert(row[i], cols[i], rowDesc, colTypes[i]));
+            Object colVal = newColVals.get(colName);
+
+            desc.setValue(colName, key, val, colVal);
         }
 
         if (cctx.binaryMarshaller()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index fdcd164..ce2971a 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
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUnion;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlUpdate;
 import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.command.Prepared;
 import org.h2.table.Column;
@@ -192,8 +193,8 @@ public final class UpdatePlanBuilder {
                 hasValProps = true;
         }
 
-        KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true);
-        KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false);
+        KeyValueSupplier keySupplier = createSupplier(cctx, desc.type(), keyColIdx, hasKeyProps, true, false);
+        KeyValueSupplier valSupplier = createSupplier(cctx, desc.type(), valColIdx, hasValProps, false, false);
 
         if (stmt instanceof GridSqlMerge)
             return UpdatePlan.forMerge(tbl.dataTable(), colNames, colTypes, keySupplier, valSupplier, keyColIdx,
@@ -253,8 +254,6 @@ public final class UpdatePlanBuilder {
             GridSqlSelect sel;
 
             if (stmt instanceof GridSqlUpdate) {
-                boolean bin = desc.context().binaryMarshaller();
-
                 List<GridSqlColumn> updatedCols = ((GridSqlUpdate) stmt).cols();
 
                 int valColIdx = -1;
@@ -282,20 +281,10 @@ public final class UpdatePlanBuilder {
                 if (hasNewVal)
                     valColIdx += 2;
 
-                int newValColIdx;
-
-                if (!hasProps) // No distinct properties, only whole new value - let's take it
-                    newValColIdx = valColIdx;
-                else if (bin) // We update distinct columns in binary mode - let's choose correct index for the builder
-                    newValColIdx = (hasNewVal ? valColIdx : 1);
-                else // Distinct properties, non binary mode - let's instantiate.
-                    newValColIdx = -1;
+                int newValColIdx = (hasNewVal ? valColIdx : 1);
 
-                // We want supplier to take present value only in case of binary mode as it will create
-                // whole new object as a result anyway, so we don't need to copy previous property values explicitly.
-                // Otherwise we always want it to instantiate new object whose properties we will later
-                // set to current values.
-                KeyValueSupplier newValSupplier = createSupplier(desc.context(), desc.type(), newValColIdx, hasProps, false);
+                KeyValueSupplier newValSupplier = createSupplier(desc.context(), desc.type(), newValColIdx, hasProps,
+                    false, true);
 
                 sel = DmlAstUtils.selectForUpdate((GridSqlUpdate) stmt, errKeysPos);
 
@@ -319,11 +308,11 @@ public final class UpdatePlanBuilder {
      * @param hasProps Whether column list affects individual properties of key or value.
      * @param key Whether supplier should be created for key or for value.
      * @return Closure returning key or value.
-     * @throws IgniteCheckedException
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
     private static KeyValueSupplier createSupplier(final GridCacheContext<?, ?> cctx, GridQueryTypeDescriptor desc,
-                                                   final int colIdx, boolean hasProps, final boolean key) throws IgniteCheckedException {
+        final int colIdx, boolean hasProps, final boolean key, boolean forUpdate) throws IgniteCheckedException {
         final String typeName = key ? desc.keyTypeName() : desc.valueTypeName();
 
         //Try to find class for the key locally.
@@ -332,15 +321,10 @@ public final class UpdatePlanBuilder {
 
         boolean isSqlType = GridQueryProcessor.isSqlType(cls);
 
-        // If we don't need to construct anything from scratch, just return value from array.
-        if (isSqlType || !hasProps || !cctx.binaryMarshaller()) {
+        // If we don't need to construct anything from scratch, just return value from given list.
+        if (isSqlType || !hasProps) {
             if (colIdx != -1)
-                return new KeyValueSupplier() {
-                    /** {@inheritDoc} */
-                    @Override public Object apply(List<?> arg) throws IgniteCheckedException {
-                        return arg.get(colIdx);
-                    }
-                };
+                return new PlainValueSupplier(colIdx);
             else if (isSqlType)
                 // Non constructable keys and values (SQL types) must be present in the query explicitly.
                 throw new IgniteCheckedException((key ? "Key" : "Value") + " is missing from query");
@@ -352,7 +336,12 @@ public final class UpdatePlanBuilder {
                 return new KeyValueSupplier() {
                     /** {@inheritDoc} */
                     @Override public Object apply(List<?> arg) throws IgniteCheckedException {
-                        BinaryObject bin = cctx.grid().binary().toBinary(arg.get(colIdx));
+                        Object obj = arg.get(colIdx);
+
+                        if (obj == null)
+                            return null;
+
+                        BinaryObject bin = cctx.grid().binary().toBinary(obj);
 
                         return cctx.grid().binary().builder(bin);
                     }
@@ -369,6 +358,26 @@ public final class UpdatePlanBuilder {
             }
         }
         else {
+            if (colIdx != -1) {
+                if (forUpdate && colIdx == 1) {
+                    // It's the case when the old value has to be taken as the basis for the new one on UPDATE,
+                    // so we have to clone it. And on UPDATE we don't expect any key supplier.
+                    assert !key;
+
+                    return new KeyValueSupplier() {
+                        /** {@inheritDoc} */
+                        @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+                            byte[] oldPropBytes = cctx.marshaller().marshal(arg.get(1));
+
+                            // colVal is another object now, we can mutate it
+                            return cctx.marshaller().unmarshal(oldPropBytes, U.resolveClassLoader(cctx.gridConfig()));
+                        }
+                    };
+                }
+                else // We either are not updating, or the new value is given explicitly, no cloning needed.
+                    return new PlainValueSupplier(colIdx);
+            }
+
             Constructor<?> ctor;
 
             try {
@@ -390,8 +399,12 @@ public final class UpdatePlanBuilder {
                             return ctor0.newInstance();
                         }
                         catch (Exception e) {
-                            throw new IgniteCheckedException("Failed to invoke default ctor for " +
-                                (key ? "key" : "value"), e);
+                            if (S.INCLUDE_SENSITIVE)
+                                throw new IgniteCheckedException("Failed to instantiate " +
+                                    (key ? "key" : "value") + " [type=" + typeName + ']', e);
+                            else
+                                throw new IgniteCheckedException("Failed to instantiate " +
+                                    (key ? "key" : "value") + '.', e);
                         }
                     }
                 };
@@ -405,8 +418,12 @@ public final class UpdatePlanBuilder {
                             return GridUnsafe.allocateInstance(cls);
                         }
                         catch (InstantiationException e) {
-                            throw new IgniteCheckedException("Failed to invoke default ctor for " +
-                                (key ? "key" : "value"), e);
+                            if (S.INCLUDE_SENSITIVE)
+                                throw new IgniteCheckedException("Failed to instantiate " +
+                                    (key ? "key" : "value") + " [type=" + typeName + ']', e);
+                            else
+                                throw new IgniteCheckedException("Failed to instantiate " +
+                                    (key ? "key" : "value") + '.', e);
                         }
                     }
                 };
@@ -414,8 +431,6 @@ public final class UpdatePlanBuilder {
         }
     }
 
-
-
     /**
      * @param target Expression to extract the table from.
      * @return Back end table for this element.
@@ -483,4 +498,26 @@ public final class UpdatePlanBuilder {
 
         return false;
     }
+
+    /**
+     * Simple supplier that just takes specified element of a given row.
+     */
+    private final static class PlainValueSupplier implements KeyValueSupplier {
+        /** Index of column to use. */
+        private final int colIdx;
+
+        /**
+         * Constructor.
+         *
+         * @param colIdx Column index.
+         */
+        private PlainValueSupplier(int colIdx) {
+            this.colIdx = colIdx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object apply(List<?> arg) throws IgniteCheckedException {
+            return arg.get(colIdx);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/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
index 86d01c7..626846b 100644
--- 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
@@ -46,6 +46,8 @@ 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;
 
+import static org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest.AllTypes;
+
 /**
  *
  */
@@ -126,6 +128,8 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             createCaches();
         else
             createBinaryCaches();
+
+        ignite(0).createCache(cacheConfig("I2AT", true, false, Integer.class, AllTypes.class));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/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
index 649012f..3c92fdf 100644
--- 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
@@ -65,7 +65,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
     /**
      * @return whether {@link #marsh} is an instance of {@link BinaryMarshaller} or not.
      */
-    private boolean isBinaryMarshaller() {
+    protected boolean isBinaryMarshaller() {
         return marsh instanceof BinaryMarshaller;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/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
index e9c21dc..f91f405 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
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.concurrent.Callable;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
@@ -202,4 +204,24 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
 
         assertEquals(createPerson(2, "Alex"), p.get(new Key4(2)));
     }
+
+    /**
+     *
+     */
+    public void testNestedFieldsHandling() {
+        IgniteCache<Integer, IgniteCacheUpdateSqlQuerySelfTest.AllTypes> p = ignite(0).cache("I2AT");
+
+        p.query(new SqlFieldsQuery("insert into AllTypes(_key, innerTypeCol, arrListCol, _val, innerStrCol) " +
+            "values (1, ?, ?, ?, 'sss')") .setArgs(new IgniteCacheUpdateSqlQuerySelfTest.AllTypes.InnerType(50L),
+            new ArrayList<>(Arrays.asList(3L, 2L, 1L)), new IgniteCacheUpdateSqlQuerySelfTest.AllTypes(1L)));
+
+        IgniteCacheUpdateSqlQuerySelfTest.AllTypes res = p.get(1);
+
+        IgniteCacheUpdateSqlQuerySelfTest.AllTypes.InnerType resInner = new IgniteCacheUpdateSqlQuerySelfTest.AllTypes.InnerType(50L);
+
+        resInner.innerStrCol = "sss";
+        resInner.arrListCol = new ArrayList<>(Arrays.asList(3L, 2L, 1L));
+
+        assertEquals(resInner, res.innerTypeCol);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/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
index 32b7a12..d9a2d9e 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,9 +17,13 @@
 
 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 static org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest.AllTypes;
+
 /**
  *
  */
@@ -149,4 +153,24 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
 
         assertEquals(createPerson(2, "Alex"), p.get(new Key4(2)));
     }
+
+    /**
+     *
+     */
+    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);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/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
index 575f617..c3d9d8e 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
@@ -44,6 +44,12 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         ignite(0).createCache(createAllTypesCacheConfig());
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+        ignite(0).cache("L2AT").clear();
+    }
+
     /**
      *
      */
@@ -182,17 +188,19 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         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("select \"primitiveIntsCol\" from \"AllTypes\"")).getAll();
-
-        cache.query(new SqlFieldsQuery("update \"AllTypes\" set \"doubleCol\" = CAST('50' as INT)," +
+        // 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(new AllTypes.InnerType(80L), new int[] {2, 3}, new Byte[] {4, 5, 6}));
+            .setArgs(5, new AllTypes.InnerType(80L), new int[] {2, 3}, new Byte[] {4, 5, 6}));
 
         AllTypes res = (AllTypes) cache.get(2L);
 
+        assertNotNull(res);
+
         assertEquals(new BigDecimal(301.0).doubleValue(), res.bigDecimalCol.doubleValue());
         assertEquals(50.0, res.doubleCol);
         assertEquals(2L, (long) res.longCol);
@@ -202,7 +210,11 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         assertTrue(Arrays.equals(new Byte[] {4, 5, 6}, res.bytesCol));
         assertTrue(Arrays.deepEquals(new Integer[] {0, 1}, res.intsCol));
         assertTrue(Arrays.equals(new int[] {2, 3}, res.primitiveIntsCol));
-        assertEquals(new AllTypes.InnerType(80L), res.innerTypeCol);
+
+        AllTypes.InnerType expInnerType = new AllTypes.InnerType(80L);
+        expInnerType.innerLongCol = 5L;
+
+        assertEquals(expInnerType, res.innerTypeCol);
         assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm:SS").parse("2016-11-30 12:00:00"), res.dateCol);
         assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm:SS").parse("2016-12-03 00:02:00"), res.tsCol);
         assertEquals(2, res.intCol);
@@ -213,6 +225,45 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         assertEquals(49, res.shortCol);
     }
 
+    /** */
+    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)));
+
+        assertFalse(cache.query(new SqlFieldsQuery("select * from \"AllTypes\"")).getAll().isEmpty());
+
+        cache.query(new SqlFieldsQuery("update \"AllTypes\" set \"innerLongCol\" = 5"));
+
+        AllTypes res = (AllTypes) cache.get(2L);
+
+        assertNotNull(res);
+
+        assertEquals(new BigDecimal(301.0).doubleValue(), res.bigDecimalCol.doubleValue());
+        assertEquals(3.01, res.doubleCol);
+        assertEquals(2L, (long) res.longCol);
+        assertFalse(res.booleanCol);
+
+        assertEquals("2", res.strCol);
+        assertTrue(Arrays.equals(new byte[] {0, 1}, res.primitiveBytesCol));
+        assertTrue(Arrays.deepEquals(new Byte[] {0, 1}, res.bytesCol));
+        assertTrue(Arrays.deepEquals(new Integer[] {0, 1}, res.intsCol));
+        assertTrue(Arrays.equals(new int[] {0, 1}, res.primitiveIntsCol));
+
+        AllTypes.InnerType expInnerType = new AllTypes.InnerType(2L);
+        expInnerType.innerLongCol = 5L;
+
+        assertEquals(expInnerType, res.innerTypeCol);
+        assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm:SS").parse("2016-11-30 12:00:00"), res.dateCol);
+        assertNull(res.tsCol);
+        assertEquals(2, res.intCol);
+        assertEquals(AllTypes.EnumType.ENUMTRUE, res.enumCol);
+        assertNull(res.sqlDateCol);
+
+        assertEquals(-23000, res.shortCol);
+    }
+
     /**
      *
      */
@@ -308,7 +359,7 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         InnerType innerTypeCol;
 
         /** */
-        static final class InnerType implements Serializable {
+        static class InnerType implements Serializable {
             /** */
             @QuerySqlField
             Long innerLongCol;

http://git-wip-us.apache.org/repos/asf/ignite/blob/70cd8e45/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 e412828..69285f1 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
@@ -578,6 +578,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
                 @Override public boolean key() {
                     return false;
                 }
+
+                /** */
+                @Override public GridQueryProperty parent() {
+                    return null;
+                }
             };
         }