You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dm...@apache.org on 2015/12/14 10:01:36 UTC

[09/40] ignite git commit: IGNITE-2124 - Fixed key deserialization on server node. - Fixes #313.

IGNITE-2124 - Fixed key deserialization on server node. - Fixes #313.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/master
Commit: 9a02acbe5bc1deeefe1b79063c6938984a25baae
Parents: 91760b9
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Dec 11 16:11:32 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Dec 11 16:11:32 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/QueryEntity.java    |   3 +-
 .../CacheDataStructuresManager.java             |   5 +
 .../processors/query/GridQueryProcessor.java    |  21 +-
 .../apache/ignite/tests/p2p/cache/Person.java   |  77 ++++++
 .../ignite/tests/p2p/cache/PersonKey.java       |  74 ++++++
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  | 246 +++++++++++++++++++
 .../IgnitePortableCacheQueryTestSuite.java      |   2 +
 7 files changed, 426 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index cb84c47..7901bec 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.ignite.cache;
 
 import java.io.Serializable;
@@ -124,7 +125,7 @@ public class QueryEntity implements Serializable {
     /**
      * Sets mapping from full property name in dot notation to an alias that will be used as SQL column name.
      * Example: {"parent.name" -> "parentName"}.
-
+     *
      * @param aliases Aliases map.
      */
     public void setAliases(Map<String, String> aliases) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index 6ec29b4..6447194 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -314,6 +314,11 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      * @param keepPortable Keep portable flag.
      */
     public void onEntryUpdated(KeyCacheObject key, boolean rmv, boolean keepPortable) {
+        // No need to notify data structures manager for a user cache since all DS objects are stored
+        // in system caches.
+        if (cctx.userCache())
+            return;
+
         Object key0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepPortable, false);
 
         if (key0 instanceof SetItemKey)

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/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 64f2415..005f617 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
@@ -205,7 +205,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     Class<?> keyCls = U.classForName(qryEntity.getKeyType(), Object.class);
                     Class<?> valCls = U.classForName(qryEntity.getValueType(), null);
 
-                    String simpleValType = valCls == null ? qryEntity.getValueType() : typeName(valCls);
+                    String simpleValType = valCls == null ? typeName(qryEntity.getValueType()) : typeName(valCls);
 
                     desc.name(simpleValType);
 
@@ -982,6 +982,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Gets type name by class.
+     *
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    public static String typeName(String clsName) {
+        int packageEnd = clsName.lastIndexOf('.');
+
+        if (packageEnd >= 0 && packageEnd < clsName.length() - 1)
+            clsName = clsName.substring(packageEnd + 1);
+
+        if (clsName.endsWith("[]")) {
+            clsName = clsName.substring(0, clsName.length() - 2) + "_array";
+        }
+
+        return clsName;
+    }
+
+    /**
      * @param space Space.
      * @param clause Clause.
      * @param resType Result type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Person.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Person.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Person.java
index 3452cea..7cb7dd2 100644
--- a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Person.java
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Person.java
@@ -18,14 +18,35 @@
 package org.apache.ignite.tests.p2p.cache;
 
 import java.io.Serializable;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
 
 /**
  *
  */
 public class Person implements Serializable {
     /** */
+    @QuerySqlField
     private String name;
 
+    /** */
+    @QuerySqlField(index = true)
+    private int id;
+
+    /** */
+    @QuerySqlField
+    private String lastName;
+
+    /** */
+    @QuerySqlField
+    private double salary;
+
+    /**
+     *
+     */
+    public Person() {
+        // No-op.
+    }
+
     /**
      * @param name Name.
      */
@@ -39,4 +60,60 @@ public class Person implements Serializable {
     public String name() {
         return name;
     }
+
+    /**
+     * @return Name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @param name Name.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return ID.
+     */
+    public int getId() {
+        return id;
+    }
+
+    /**
+     * @param id ID.
+     */
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    /**
+     * @return Last name.
+     */
+    public String getLastName() {
+        return lastName;
+    }
+
+    /**
+     * @param lastName Last name.
+     */
+    public void setLastName(String lastName) {
+        this.lastName = lastName;
+    }
+
+    /**
+     * @return Salary.
+     */
+    public double getSalary() {
+        return salary;
+    }
+
+    /**
+     * @param salary Salary.
+     */
+    public void setSalary(double salary) {
+        this.salary = salary;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/PersonKey.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/PersonKey.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/PersonKey.java
new file mode 100644
index 0000000..3233f9b
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/PersonKey.java
@@ -0,0 +1,74 @@
+/*
+ * 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.tests.p2p.cache;
+
+import java.io.Serializable;
+
+/**
+ * Person key.
+ */
+public class PersonKey implements Serializable {
+    /** */
+    private int id;
+
+    /**
+     * Empty constructor for tests.
+     */
+    public PersonKey() {
+        // No-op.
+    }
+
+    /**
+     * @param id ID.
+     */
+    public PersonKey(int id) {
+        this.id = id;
+    }
+
+    /**
+     * @return ID.
+     */
+    public int id() {
+        return id;
+    }
+
+    /**
+     * @param id ID.
+     */
+    public void id(int id) {
+        this.id = id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof PersonKey))
+            return false;
+
+        PersonKey key = (PersonKey)o;
+
+        return id == key.id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectFieldsQuerySelfTest.java
new file mode 100644
index 0000000..3a08824
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectFieldsQuerySelfTest.java
@@ -0,0 +1,246 @@
+/*
+ * 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 javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests that server nodes do not need class definitions to execute queries.
+ */
+public class IgniteBinaryObjectFieldsQuerySelfTest extends GridCommonAbstractTest {
+    /** */
+    public static final String PERSON_KEY_CLS_NAME = "org.apache.ignite.tests.p2p.cache.PersonKey";
+
+    /** */
+    public static final String PERSON_CLS_NAME = "org.apache.ignite.tests.p2p.cache.Person";
+
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static ClassLoader extClassLoader;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        cfg.setMarshaller(null);
+
+        if (getTestGridName(3).equals(gridName)) {
+            cfg.setClientMode(true);
+            cfg.setClassLoader(extClassLoader);
+        }
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache.
+     */
+    protected CacheConfiguration cache(CacheMode cacheMode, CacheAtomicityMode atomicity) throws Exception {
+        CacheConfiguration cache = defaultCacheConfiguration();
+
+        cache.setName(null);
+        cache.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cache.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cache.setCacheMode(cacheMode);
+        cache.setAtomicityMode(atomicity);
+
+        cache.setIndexedTypes(extClassLoader.loadClass(PERSON_KEY_CLS_NAME), extClassLoader.loadClass(PERSON_CLS_NAME));
+
+        return cache;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        extClassLoader = getExternalClassLoader();
+
+        startGrids(4);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        extClassLoader = null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryPartitionedAtomic() throws Exception {
+        checkQuery(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryReplicatedAtomic() throws Exception {
+        checkQuery(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryPartitionedTransactional() throws Exception {
+        checkQuery(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryReplicatedTransactional() throws Exception {
+        checkQuery(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFieldsQueryPartitionedAtomic() throws Exception {
+        checkFieldsQuery(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFieldsQueryReplicatedAtomic() throws Exception {
+        checkFieldsQuery(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFieldsQueryPartitionedTransactional() throws Exception {
+        checkFieldsQuery(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFieldsQueryReplicatedTransactional() throws Exception {
+        checkFieldsQuery(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkFieldsQuery(CacheMode cacheMode, CacheAtomicityMode atomicity) throws Exception {
+        IgniteCache<Object, Object>cache = grid(3).getOrCreateCache(cache(cacheMode, atomicity));
+
+        try {
+            populate(cache);
+
+            QueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("select id, name, lastName, salary from " +
+                "Person order by id asc"));
+
+            List<List<?>> all = cur.getAll();
+
+            assertEquals(100, all.size());
+
+            for (int i = 0; i < 100; i++) {
+                List<?> row = all.get(i);
+
+                assertEquals(i, row.get(0));
+                assertEquals("person-" + i, row.get(1));
+                assertEquals("person-last-" + i, row.get(2));
+                assertEquals((double)(i * 25), row.get(3));
+            }
+        }
+        finally {
+            grid(3).destroyCache(null);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkQuery(CacheMode cacheMode, CacheAtomicityMode atomicity) throws Exception {
+        IgniteCache<Object, Object> cache = grid(3).getOrCreateCache(cache(cacheMode, atomicity));
+
+        try {
+            populate(cache);
+
+            QueryCursor<Cache.Entry<Object, Object>> cur = cache.query(new SqlQuery("Person", "order " +
+                "by id asc"));
+
+            List<Cache.Entry<Object, Object>> all = cur.getAll();
+
+            assertEquals(100, all.size());
+
+            for (int i = 0; i < 100; i++) {
+                Object person = all.get(i).getValue();
+
+                assertEquals(i, U.field(person, "id"));
+                assertEquals("person-" + i, U.field(person, "name"));
+                assertEquals("person-last-" + i, U.field(person, "lastName"));
+                assertEquals((double)(i * 25), U.field(person, "salary"));
+            }
+        }
+        finally {
+            grid(3).destroyCache(null);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void populate(IgniteCache<Object, Object> cache) throws Exception {
+        Class<?> keyCls = extClassLoader.loadClass(PERSON_KEY_CLS_NAME);
+        Class<?> cls = extClassLoader.loadClass(PERSON_CLS_NAME);
+
+        for (int i = 0; i < 100; i++) {
+            Object key = keyCls.newInstance();
+
+            GridTestUtils.setFieldValue(key, "id", i);
+
+            Object person = cls.newInstance();
+
+            GridTestUtils.setFieldValue(person, "id", i);
+            GridTestUtils.setFieldValue(person, "name", "person-" + i);
+            GridTestUtils.setFieldValue(person, "lastName", "person-last-" + i);
+            GridTestUtils.setFieldValue(person, "salary", (double)(i * 25));
+
+            cache.put(key, person);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a02acbe/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
index da34d75..3773f8c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLo
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
@@ -79,6 +80,7 @@ public class IgnitePortableCacheQueryTestSuite extends TestSuite {
 
         // Fields queries.
         suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
+        suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
 
         // Continuous queries.
         suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);