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/30 09:26:22 UTC

[ignite] branch master updated: IGNITE-10645: SQL: Avoid key/val ownership resolution of a field in runtime. This closes #5657.

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 f350ada  IGNITE-10645: SQL: Avoid key/val ownership resolution of a field in runtime. This closes #5657.
f350ada is described below

commit f350ada7f8cd921396032e4bdbf87343cac07b82
Author: Pavel Kuznetsov <pa...@gmail.com>
AuthorDate: Wed Jan 30 12:26:11 2019 +0300

    IGNITE-10645: SQL: Avoid key/val ownership resolution of a field in runtime. This closes #5657.
---
 .../internal/processors/query/QueryUtils.java      |  32 ++-
 .../query/property/QueryBinaryProperty.java        |  60 +----
 .../org/apache/ignite/client/FunctionalTest.java   |   5 +-
 .../cache/CacheBinaryKeyConcurrentQueryTest.java   |   3 +
 ...teCacheDistributedJoinCollocatedAndNotTest.java |   6 +-
 .../IgniteCacheJoinQueryWithAffinityKeyTest.java   |   2 +
 .../IgniteCacheSqlInsertValidationSelfTest.java    | 286 +++++++++++++++++++++
 .../processors/cache/index/BasicIndexTest.java     |   2 +
 ...achePartitionedAtomicColumnConstraintsTest.java |   3 +
 .../IgniteBinaryCacheQueryTestSuite.java           |   4 +-
 .../Cache/Query/CacheDmlQueriesTest.cs             |   6 +-
 modules/platforms/python/tests/test_binary.py      |   1 -
 12 files changed, 338 insertions(+), 72 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 0207eb5..852468b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -563,10 +563,6 @@ public class QueryUtils {
         Map<String, Integer> precision  = qryEntity.getFieldsPrecision();
         Map<String, Integer> scale = qryEntity.getFieldsScale();
 
-        // We have to distinguish between empty and null keyFields when the key is not of SQL type -
-        // when a key is not of SQL type, absence of a field in nonnull keyFields tell us that this field
-        // is a value field, and null keyFields tells us that current configuration
-        // does not tell us anything about this field's ownership.
         boolean hasKeyFields = (keyFields != null);
 
         boolean isKeyClsSqlType = isSqlType(d.keyClass());
@@ -580,23 +576,31 @@ public class QueryUtils {
             }
         }
 
+        // We are creating binary properties for all the fields, even if field is of sql type (keyFieldName or
+        // valueFieldName). In that case we rely on the fact, that binary property's methods value() and
+        // setValue() will never get called, because there is no value to extract, key/val object itself is a
+        // value.
         for (Map.Entry<String, String> entry : fields.entrySet()) {
-            Boolean isKeyField;
+            String fieldName = entry.getKey();
+            String fieldType = entry.getValue();
 
-            if (isKeyClsSqlType) // We don't care about keyFields in this case - it might be null, or empty, or anything
+            boolean isKeyField;
+
+            if (isKeyClsSqlType)
+                // Entire key is not field of itself, even if it is set in "keyFields".
                 isKeyField = false;
             else
-                isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
+                isKeyField = hasKeyFields && keyFields.contains(fieldName);
 
-            boolean notNull = notNulls != null && notNulls.contains(entry.getKey());
+            boolean notNull = notNulls != null && notNulls.contains(fieldName);
 
-            Object dfltVal = dlftVals != null ? dlftVals.get(entry.getKey()) : null;
+            Object dfltVal = dlftVals != null ? dlftVals.get(fieldName) : null;
 
-            QueryBinaryProperty prop = buildBinaryProperty(ctx, entry.getKey(),
-                U.classForName(entry.getValue(), Object.class, true),
+            QueryBinaryProperty prop = buildBinaryProperty(ctx, fieldName,
+                U.classForName(fieldType, Object.class, true),
                 d.aliases(), isKeyField, notNull, dfltVal,
-                precision == null ? -1 : precision.getOrDefault(entry.getKey(), -1),
-                scale == null ? -1 : scale.getOrDefault(entry.getKey(), -1));
+                precision == null ? -1 : precision.getOrDefault(fieldName, -1),
+                scale == null ? -1 : scale.getOrDefault(fieldName, -1));
 
             d.addProperty(prop, false);
         }
@@ -799,7 +803,7 @@ public class QueryUtils {
      * @return Binary property.
      */
     public static QueryBinaryProperty buildBinaryProperty(GridKernalContext ctx, String pathStr,
-        Class<?> resType, Map<String, String> aliases, @Nullable Boolean isKeyField, boolean notNull, Object dlftVal,
+        Class<?> resType, Map<String, String> aliases, boolean isKeyField, boolean notNull, Object dlftVal,
         int precision, int scale) {
         String[] path = pathStr.split("\\.");
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
index 0405057..6f7ac96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.property;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.binary.BinaryField;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -28,8 +27,6 @@ import org.apache.ignite.internal.binary.BinaryObjectEx;
 import org.apache.ignite.internal.binary.BinaryObjectExImpl;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Binary property.
@@ -38,9 +35,6 @@ public class QueryBinaryProperty implements GridQueryProperty {
     /** Kernal context. */
     private final GridKernalContext ctx;
 
-    /** Logger. */
-    private final IgniteLogger log;
-
     /** Property name. */
     private String propName;
 
@@ -53,8 +47,8 @@ public class QueryBinaryProperty implements GridQueryProperty {
     /** Result class. */
     private Class<?> type;
 
-    /** */
-    private volatile int isKeyProp;
+    /** Defines where value should be extracted from : cache entry's key or value. */
+    private final boolean isKeyProp;
 
     /** Binary field to speed-up deserialization. */
     private volatile BinaryField field;
@@ -62,9 +56,6 @@ public class QueryBinaryProperty implements GridQueryProperty {
     /** Flag indicating that we already tried to take a field. */
     private volatile boolean fieldTaken;
 
-    /** Whether user was warned about missing property. */
-    private volatile boolean warned;
-
     /** */
     private final boolean notNull;
 
@@ -84,7 +75,7 @@ public class QueryBinaryProperty implements GridQueryProperty {
      * @param propName Property name.
      * @param parent Parent property.
      * @param type Result type.
-     * @param key {@code true} if key property, {@code false} otherwise, {@code null}  if unknown.
+     * @param key {@code true} if key property, {@code false} otherwise.
      * @param alias Field alias.
      * @param notNull {@code true} if null value is not allowed.
      * @param defaultValue Default value.
@@ -92,21 +83,15 @@ public class QueryBinaryProperty implements GridQueryProperty {
      * @param scale Scale.
      */
     public QueryBinaryProperty(GridKernalContext ctx, String propName, QueryBinaryProperty parent,
-        Class<?> type, @Nullable Boolean key, String alias, boolean notNull, Object defaultValue,
+        Class<?> type, boolean key, String alias, boolean notNull, Object defaultValue,
         int precision, int scale) {
         this.ctx = ctx;
-
-        log = ctx.log(QueryBinaryProperty.class);
-
         this.propName = propName;
         this.alias = F.isEmpty(alias) ? propName : alias;
         this.parent = parent;
         this.type = type;
         this.notNull = notNull;
-
-        if (key != null)
-            this.isKeyProp = key ? 1 : -1;
-
+        this.isKeyProp = key;
         this.defaultValue = defaultValue;
         this.precision = precision;
         this.scale = scale;
@@ -126,33 +111,12 @@ public class QueryBinaryProperty implements GridQueryProperty {
                 throw new IgniteCheckedException("Non-binary object received as a result of property extraction " +
                     "[parent=" + parent + ", propName=" + propName + ", obj=" + obj + ']');
         }
-        else {
-            int isKeyProp0 = isKeyProp;
-
-            if (isKeyProp0 == 0) {
-                // Key is allowed to be a non-binary object here.
-                // We check value before key consistently with ClassProperty.
-                if (val instanceof BinaryObject && ((BinaryObject)val).hasField(propName))
-                    isKeyProp = isKeyProp0 = -1;
-                else if (key instanceof BinaryObject && ((BinaryObject)key).hasField(propName))
-                    isKeyProp = isKeyProp0 = 1;
-                else {
-                    if (!warned) {
-                        U.warn(log, "Neither key nor value have property \"" + propName + "\" " +
-                            "(is cache indexing configured correctly?)");
-
-                        warned = true;
-                    }
-
-                    return null;
-                }
-            }
-
-            obj = isKeyProp0 == 1 ? key : val;
-        }
+        else
+            obj = isKeyProp ? key : val;
 
         if (obj instanceof BinaryObject) {
             BinaryObject obj0 = (BinaryObject) obj;
+
             return fieldValue(obj0);
         }
         else if (obj instanceof BinaryObjectBuilder) {
@@ -273,13 +237,7 @@ public class QueryBinaryProperty implements GridQueryProperty {
 
     /** {@inheritDoc} */
     @Override public boolean key() {
-        int isKeyProp0 = isKeyProp;
-
-        if (isKeyProp0 == 0)
-            throw new IllegalStateException("Ownership flag not set for binary property. Have you set 'keyFields'" +
-                " property of QueryEntity in programmatic or XML configuration?");
-
-        return isKeyProp0 == 1;
+        return isKeyProp;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java b/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
index 1272287..078f3af 100644
--- a/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
@@ -169,7 +169,10 @@ public class FunctionalTest {
                             SimpleEntry::getKey, SimpleEntry::getValue, (a, b) -> a, LinkedHashMap::new
                         ))
                     )
-                    .setKeyFields(Collections.singleton("id"))
+                    // During query normalization null keyFields become empty set.
+                    // Set empty collection for comparator.
+                    .setKeyFields(Collections.emptySet())
+                    .setKeyFieldName("id")
                     .setNotNullFields(Collections.singleton("id"))
                     .setDefaultFieldValues(Collections.singletonMap("id", 0))
                     .setIndexes(Collections.singletonList(new QueryIndex("id", true, "IDX_EMPLOYEE_ID")))
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheBinaryKeyConcurrentQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheBinaryKeyConcurrentQueryTest.java
index cdf585f..9d8847c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheBinaryKeyConcurrentQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheBinaryKeyConcurrentQueryTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -229,6 +230,8 @@ public class CacheBinaryKeyConcurrentQueryTest extends GridCommonAbstractTest {
         qryEntity.addQueryField("id", Integer.class.getName(), null);
         qryEntity.addQueryField("val", Integer.class.getName(), null);
 
+        qryEntity.setKeyFields(Collections.singleton("id"));
+
         qryEntity.setIndexes(F.asList(new QueryIndex("id"), new QueryIndex("val")));
 
         ccfg.setQueryEntities(F.asList(qryEntity));
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinCollocatedAndNotTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinCollocatedAndNotTest.java
index 1618ce1..e34efbf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinCollocatedAndNotTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDistributedJoinCollocatedAndNotTest.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collections;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
@@ -80,7 +81,8 @@ public class IgniteCacheDistributedJoinCollocatedAndNotTest extends GridCommonAb
             entity.addQueryField("id", Integer.class.getName(), null);
             entity.addQueryField("affKey", Integer.class.getName(), null);
             entity.addQueryField("name", String.class.getName(), null);
-            entity.setKeyFields(Collections.singleton("affKey"));
+
+            entity.setKeyFields(new HashSet<>(Arrays.asList("id", "affKey")));
 
             ccfg.setQueryEntities(F.asList(entity));
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryWithAffinityKeyTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryWithAffinityKeyTest.java
index 05e71b0..961605e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryWithAffinityKeyTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheJoinQueryWithAffinityKeyTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.io.Serializable;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -360,6 +361,7 @@ public class IgniteCacheJoinQueryWithAffinityKeyTest extends GridCommonAbstractT
         QueryEntity person = new QueryEntity();
         person.setKeyType(personKeyType);
         person.setValueType(Person.class.getName());
+        person.setKeyFields(Collections.singleton("id"));
         person.addQueryField("orgId", Integer.class.getName(), null);
         person.addQueryField("id", Integer.class.getName(), null);
         person.addQueryField("name", String.class.getName(), null);
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheSqlInsertValidationSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheSqlInsertValidationSelfTest.java
new file mode 100644
index 0000000..1809d08
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheSqlInsertValidationSelfTest.java
@@ -0,0 +1,286 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Tests for validation of inserts sql queries.
+ */
+@RunWith(JUnit4.class)
+public class IgniteCacheSqlInsertValidationSelfTest extends GridCommonAbstractTest {
+    /** Entry point for sql api. Contains table configurations too. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** Default value for fk2 field of WITH_KEY_FLDS table. */
+    private static final Long DEFAULT_FK2_VAL = 42L;
+
+    /** Default value for fk2 field of WITH_KEY_FLDS table. */
+    private static final Long DEFAULT_FK1_VAL = null;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        Map<String, Object> defsFK2 = new HashMap<>();
+        defsFK2.put("fk2", DEFAULT_FK2_VAL);
+
+        cache = jcache(grid(0), defaultCacheConfiguration()
+                .setName("testCache")
+                .setQueryEntities(Arrays.asList(
+                    new QueryEntity(Key.class.getName(), Val.class.getName())
+                        .addQueryField("fk1", "java.lang.Long", null)
+                        .addQueryField("fk2", "java.lang.Long", null)
+                        .addQueryField("fv1", "java.lang.Long", null)
+                        .addQueryField("fv2", "java.lang.Long", null)
+                        .setTableName("FORGOTTEN_KEY_FLDS"),
+                    new QueryEntity(Key.class.getName(), Integer.class.getName())
+                        .addQueryField("fk1", "java.lang.Long", null)
+                        .addQueryField("fk2", "java.lang.Long", null)
+                        .setDefaultFieldValues(defsFK2)
+                        .setKeyFields(new HashSet<>(Arrays.asList("fk1", "fk2")))
+                        .setTableName("WITH_KEY_FLDS"),
+                    new QueryEntity(Integer.class.getName(), Val2.class.getName())
+                        .addQueryField("fv1", "java.lang.Long", null)
+                        .addQueryField("fv2", "java.lang.Long", null)
+                        .setTableName("INT_KEY_TAB"),
+                    new QueryEntity(SuperKey.class, String.class)
+                        .setTableName("SUPER_TAB")
+                ))
+            , "testCache");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (cache != null)
+            cache.destroy();
+    }
+
+    /**
+     * Check that if we cannot insert row using sql due to we don't have keyFields in the configuration, we are still
+     * able to put using cache api.
+     */
+    @Test
+    public void testCacheApiIsStillAllowed() {
+        cache.put(new Key(1, 2), new Val(3, 4));
+
+        assertNotNull("Expected cache to contain object ", cache.get(new Key(1, 2)));
+    }
+
+    /**
+     * Check that we are able to perform sql insert using special "_key" field. Even in case of non sql key.
+     */
+    @Test
+    public void testInsertDefaultKeyName() {
+        Object cnt = execute("INSERT INTO INT_KEY_TAB (_key, fv1, fv2) VALUES (1 , 2 , 3)").get(0).get(0);
+
+        assertEquals("Expected one row successfully inserted ", 1L, cnt);
+    }
+
+    /**
+     * Check forgotten key fields.
+     * If we've forgotten to specify key fields and we don't specify _key, then default key is inserted.
+     */
+    @Test
+    public void testIncorrectComplex() {
+        execute("INSERT INTO FORGOTTEN_KEY_FLDS(FK1, FK2, FV1, FV2) VALUES (2,3,4,5)");
+
+        GridTestUtils.assertThrows(log(),
+            () -> execute("INSERT INTO FORGOTTEN_KEY_FLDS(FK1, FK2, FV1, FV2) VALUES (8,9,10,11)"),
+            IgniteSQLException.class,
+            "Duplicate key during INSERT");
+    }
+
+    /**
+     * Check that we can specify only one pk column (out of two). Second one should be of default value for type;
+     */
+    @Test
+    public void testNotAllKeyColsComplex() {
+        execute("INSERT INTO WITH_KEY_FLDS(FK1, _val) VALUES (7, 1)"); // Missing FK2 -> (7, 42, 1)
+        execute("INSERT INTO WITH_KEY_FLDS(FK2, _val) VALUES (15, 2)"); // Missing FK1 -> (null, 15, 2)
+
+        Long fk2 = (Long)execute("SELECT FK2 FROM WITH_KEY_FLDS WHERE _val = 1").get(0).get(0);
+        Long fk1 = (Long)execute("SELECT FK1 FROM WITH_KEY_FLDS WHERE _val = 2").get(0).get(0);
+
+        assertEquals(DEFAULT_FK2_VAL, fk2);
+        assertEquals(DEFAULT_FK1_VAL, fk1);
+    }
+
+    /**
+     * Check that we can't perform insert without at least one key field specified.
+     */
+    @Test
+    public void testMixedPlaceholderWithOtherKeyFields() {
+        GridTestUtils.assertThrows(log(),
+            () -> execute("INSERT INTO WITH_KEY_FLDS(_key, FK1, _val) VALUES (?, ?, ?)",
+                new Key(1, 2), 42, 43),
+            IgniteSQLException.class,
+            "Column _KEY refers to entire key cache object.");
+    }
+
+    /**
+     * Check that key can contain nested field with its own fields. Check that we can insert mixing sql and non sql
+     * values.
+     */
+    @Test
+    public void testSuperKey() {
+        execute("INSERT INTO SUPER_TAB (SUPERKEYID, NESTEDKEY, _val) VALUES (?, ?, ?)",
+            123, new NestedKey("the name "), "the _val value");
+    }
+
+    /**
+     * Check that key can contain nested field with its own fields. Check that we can insert using _key placeholder.
+     */
+    @Test
+    public void testSuperKeyNative() {
+        execute("INSERT INTO SUPER_TAB (_key, _val) VALUES (?, ?)",
+            new SuperKey(1, new NestedKey("the name")),
+            "_val value");
+    }
+
+    /**
+     * Check we can amend fields list part.
+     */
+    @Test
+    public void testInsertImplicitAllFields() {
+        execute("CREATE TABLE PUBLIC.IMPLICIT_INS (id1 BIGINT, id2 BIGINT, val BIGINT, PRIMARY KEY(id1, id2))");
+
+        execute("INSERT INTO PUBLIC.IMPLICIT_INS VALUES (1,2,3)");
+    }
+
+    /**
+     * Execute native sql.
+     *
+     * @param sql query.
+     * @param args arguments of SqlFieldsQuery.
+     * @return {@link QueryCursor#getAll()} - result of the query.
+     */
+    private List<List<?>> execute(String sql, Object... args) {
+        return cache.query(new SqlFieldsQuery(sql).setArgs(args)).getAll();
+    }
+
+    /**
+     * @param qryEntity Query entity.
+     * @return Cache configuration.
+     */
+    protected CacheConfiguration cacheConfiguration(QueryEntity qryEntity) {
+        CacheConfiguration<?, ?> cache = defaultCacheConfiguration();
+
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setQueryEntities(Collections.singletonList(qryEntity));
+
+        return cache;
+    }
+
+    private static class Key {
+        private long fk1;
+
+        private long fk2;
+
+        public Key(long fk1, long fk2) {
+            this.fk1 = fk1;
+            this.fk2 = fk2;
+        }
+
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+            Key key = (Key)o;
+            return fk1 == key.fk1 &&
+                fk2 == key.fk2;
+        }
+
+        @Override public int hashCode() {
+            return Objects.hash(fk1, fk2);
+        }
+    }
+
+    private static class SuperKey {
+        @QuerySqlField
+        private long superKeyId;
+
+        @QuerySqlField
+        private NestedKey nestedKey;
+
+        public SuperKey(long superKeyId, NestedKey nestedKey) {
+            this.superKeyId = superKeyId;
+            this.nestedKey = nestedKey;
+        }
+    }
+
+    private static class NestedKey {
+        @QuerySqlField
+        private String name;
+
+        public NestedKey(String name) {
+            this.name = name;
+        }
+    }
+
+    private static class Val {
+        private long fv1;
+
+        private long fv2;
+
+        public Val(long fv1, long fv2) {
+            this.fv1 = fv1;
+            this.fv2 = fv2;
+        }
+    }
+
+    private static class Val2 {
+        private long fv1;
+
+        private long fv2;
+
+        public Val2(long fv1, long fv2) {
+            this.fv1 = fv1;
+            this.fv2 = fv2;
+        }
+    }
+}
\ No newline at end of file
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java
index 1e1c61c..249f128 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/BasicIndexTest.java
@@ -22,6 +22,7 @@ import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Objects;
@@ -86,6 +87,7 @@ public class BasicIndexTest extends AbstractIndexingCommonTest {
                     .setKeyType(Key.class.getName())
                     .setValueType(Val.class.getName())
                     .setFields(fields)
+                    .setKeyFields(new HashSet<>(Arrays.asList("keyStr", "keyLong", "keyPojo")))
                     .setIndexes(indexes)
             ))
             .setSqlIndexMaxInlineSize(inlineSize);
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/sql/IgniteCachePartitionedAtomicColumnConstraintsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/sql/IgniteCachePartitionedAtomicColumnConstraintsTest.java
index 9c5db97..1ee2a2e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/sql/IgniteCachePartitionedAtomicColumnConstraintsTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/sql/IgniteCachePartitionedAtomicColumnConstraintsTest.java
@@ -119,6 +119,7 @@ public class IgniteCachePartitionedAtomicColumnConstraintsTest extends AbstractI
         orgAddressPrecision.put("address", 5);
 
         jcache(grid(0), cacheConfiguration(new QueryEntity(Organization.class.getName(), Address.class.getName())
+            .setKeyFields(Collections.singleton("name"))
             .addQueryField("name", "java.lang.String", "name")
             .addQueryField("address", "java.lang.String", "address")
             .setFieldsPrecision(orgAddressPrecision)), OBJ_CACHE_NAME);
@@ -152,6 +153,7 @@ public class IgniteCachePartitionedAtomicColumnConstraintsTest extends AbstractI
         orgEmployeePrecision.put("salary", 4);
 
         jcache(grid(0), cacheConfiguration(new QueryEntity(DecOrganization.class.getName(), Employee.class.getName())
+            .setKeyFields(Collections.singleton("id"))
             .addQueryField("id", "java.math.BigDecimal", "id")
             .addQueryField("salary", "java.math.BigDecimal", "salary")
             .setFieldsPrecision(orgEmployeePrecision)), OBJ_CACHE_NAME_FOR_PREC);
@@ -193,6 +195,7 @@ public class IgniteCachePartitionedAtomicColumnConstraintsTest extends AbstractI
         orgEmployeeScale.put("salary", 2);
 
         jcache(grid(0), cacheConfiguration(new QueryEntity(DecOrganization.class.getName(), Employee.class.getName())
+            .setKeyFields(Collections.singleton("id"))
             .addQueryField("id", "java.math.BigDecimal", "id")
             .addQueryField("salary", "java.math.BigDecimal", "salary")
             .setFieldsScale(orgEmployeeScale)
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 e3547e6..5798ade 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.IgniteCacheSqlInsertValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheSqlDmlErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheUnionDuplicatesTest;
@@ -292,7 +293,7 @@ import org.junit.runners.Suite;
     GridH2IndexingInMemSelfTest.class,
     GridH2IndexingOffheapSelfTest.class,
 
-        // Parsing
+    // Parsing
     GridQueryParsingTest.class,
     IgniteCacheSqlQueryErrorSelfTest.class,
     IgniteCacheSqlDmlErrorSelfTest.class,
@@ -366,6 +367,7 @@ import org.junit.runners.Suite;
     IgniteCacheDeleteSqlQuerySelfTest.class,
     IgniteSqlSkipReducerOnUpdateDmlSelfTest.class,
     IgniteSqlSkipReducerOnUpdateDmlFlagSelfTest.class,
+    IgniteCacheSqlInsertValidationSelfTest.class,
 
     IgniteBinaryObjectQueryArgumentsTest.class,
     IgniteBinaryObjectLocalQueryArgumentsTest.class,
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
index 5c48284..ae57880 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
@@ -283,8 +283,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 ValueTypeName = "Foo",
                 Fields = new[]
                 {
+                    /// Next two fieleds belong to the <see cref="Key"/> object, so should have been marked with <see cref="QueryField.IsKeyField"/>
+                    // But if we forgot to do this - all fields are treated as value fields. Key fields have default values and second insert fails. 
                     new QueryField("Lo", typeof(int)),
                     new QueryField("Hi", typeof(int)),
+
                     new QueryField("Id", typeof(int)),
                     new QueryField("Name", typeof(string))
                 }
@@ -296,8 +299,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 () => cache.Query(new SqlFieldsQuery("insert into foo(lo, hi, id, name) " +
                                                            "values (1, 2, 3, 'John'), (4, 5, 6, 'Mary')")));
 
-            Assert.AreEqual("Ownership flag not set for binary property. Have you set 'keyFields' " +
-                            "property of QueryEntity in programmatic or XML configuration?", ex.Message);
+            StringAssert.StartsWith("Failed to INSERT some keys because they are already in cache", ex.Message);
         }
 
         /// <summary>
diff --git a/modules/platforms/python/tests/test_binary.py b/modules/platforms/python/tests/test_binary.py
index f6c48e9..29ccf68 100644
--- a/modules/platforms/python/tests/test_binary.py
+++ b/modules/platforms/python/tests/test_binary.py
@@ -120,7 +120,6 @@ def test_sql_write_as_binary(client):
                     {
                         'name': 'TEST_PK',
                         'type_name': 'java.lang.Integer',
-                        'is_key_field': True,
                         'is_notnull_constraint_field': True,
                     },
                     {