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

[17/50] [abbrv] ignite git commit: IGNITE-4631: Added check to ensure QueryEntity 'keyFields' is subset of 'fields'. This closes #1484.

IGNITE-4631: Added check to ensure QueryEntity 'keyFields' is subset of 'fields'. This closes #1484.


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

Branch: refs/heads/ignite-3477-merge2.0
Commit: 46ff66efda93fa91a44d44ae68e7ade9add1c697
Parents: bb1ac0a
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Wed Feb 8 17:11:30 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Feb 8 17:11:30 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    |   9 ++
 .../cache/QueryEntityCaseMismatchTest.java      | 107 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 3 files changed, 118 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/46ff66ef/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..fd06828 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
@@ -1530,6 +1530,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         boolean isKeyClsSqlType = isSqlType(d.keyClass());
 
+        if (hasKeyFields && !isKeyClsSqlType) {
+            //ensure that 'keyFields' is case sensitive subset of 'fields'
+            for (String keyField : keyFields) {
+                if (!qryEntity.getFields().containsKey(keyField))
+                    throw new IgniteCheckedException("QueryEntity 'keyFields' property must be a subset of keys " +
+                        "from 'fields' property (case sensitive): " + keyField);
+            }
+        }
+
         for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
             Boolean isKeyField;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/46ff66ef/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryEntityCaseMismatchTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryEntityCaseMismatchTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryEntityCaseMismatchTest.java
new file mode 100644
index 0000000..4f8cfee
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryEntityCaseMismatchTest.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.concurrent.Callable;
+
+/**
+ * Test reveals issue of null values in SQL query resultset columns that correspond to compound key.
+ * That happens when QueryEntity.keyFields has wrong register compared to QueryEntity.fields.
+ * Issue only manifests for BinaryMarshaller case. Otherwise the keyFields aren't taken into account.
+ */
+public class QueryEntityCaseMismatchTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        cfg.setPeerClassLoadingEnabled(true);
+
+        TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+        ipFinder.setAddresses(Collections.singletonList("127.0.0.1:47500..47509"));
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        CacheConfiguration<Object, Integer> ccfg = new CacheConfiguration<>("");
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        QueryEntity queryEntity = new QueryEntity("KeyType", Integer.class.getName());
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("a", "TypeA");
+        fields.put("b", "TypeB");
+
+        queryEntity.setFields(fields);
+
+        //Specify key fields in upper register
+        HashSet<String> keyFields = new HashSet<>();
+
+        keyFields.add("a");
+        keyFields.add("B");
+
+        queryEntity.setKeyFields(keyFields);
+
+        ccfg.setQueryEntities(F.asList(queryEntity));
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * The cache must not initialize if QueryEntity.keyFields isn't subset of QueryEntity.fields
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testCacheInitializationFailure() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(1);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/46ff66ef/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index d85e111..386b8fd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -68,6 +68,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest;
+import org.apache.ignite.internal.processors.cache.QueryEntityCaseMismatchTest;
 import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
@@ -242,6 +243,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class);
         suite.addTestSuite(IgniteCacheQueryCacheDestroySelfTest.class);
         suite.addTestSuite(IgniteSqlEntryCacheModeAgnosticTest.class);
+        suite.addTestSuite(QueryEntityCaseMismatchTest.class);
 
         return suite;
     }