You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/07/09 03:47:18 UTC

[14/14] incubator-ignite git commit: IGNITE-950 - Adding tests WIP

IGNITE-950 - Adding tests WIP


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

Branch: refs/heads/ignite-950
Commit: 2105a48ef577322854f05d8aa0576696b6292c4c
Parents: b76014a
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jul 8 18:46:54 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jul 8 18:46:54 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheTypeMetadata.java  |   7 +-
 .../IgniteCacheObjectProcessorImpl.java         |   2 +-
 .../processors/query/GridQueryProcessor.java    |  11 +-
 .../OptimizedMarshallerIndexingHandler.java     |  60 +------
 .../tests/p2p/cache/IndexAnnotatedValue.java    | 104 +++++++++++
 .../ignite/tests/p2p/cache/IndexValue.java      |  99 +++++++++++
 ...IgniteCacheQueryNoServerClassesSelfTest.java | 174 +++++++++++++++++++
 7 files changed, 398 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
index f2b004e..997383d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
@@ -214,7 +214,12 @@ public class CacheTypeMetadata implements Serializable {
      * @param cls Value type class.
      */
     public void setValueType(Class<?> cls) {
-        setValueType(cls.getName());
+        if (valType != null)
+            throw new CacheException("Value type can be set only once.");
+
+        valType = cls.getName();
+
+        simpleValType = GridQueryProcessor.typeName(cls);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 46eefb6..7b556ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -388,7 +388,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     /** {@inheritDoc} */
     @Override public boolean isFieldsIndexingEnabled(Class<?> cls) {
         return indexingMgr != null && indexingMgr.isFieldsIndexingSupported() &&
-            indexingMgr.fieldsIndexingEnabledForClass(cls);
+            indexingMgr.enableFieldsIndexingForClass(cls);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/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 0be6055..22ff51e 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
@@ -441,10 +441,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             TypeId id;
 
-            boolean portableVal = ctx.cacheObjects().isPortableObject(val);
-            boolean indexedFieldsVal = val instanceof CacheIndexedObjectImpl;
+            boolean indexedFieldsVal = ctx.cacheObjects().isIndexedObject(val);
 
-            if (portableVal || indexedFieldsVal) {
+            if (indexedFieldsVal) {
                 int typeId = ctx.cacheObjects().typeId(val);
 
                 id = new TypeId(space, typeId);
@@ -460,7 +459,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (desc == null || !desc.registered())
                 return;
 
-            if (!portableVal && !indexedFieldsVal && !desc.valueClass().isAssignableFrom(valCls))
+            if (!indexedFieldsVal && !desc.valueClass().isAssignableFrom(valCls))
                 throw new IgniteCheckedException("Failed to update index due to class name conflict" +
                     "(multiple classes with same simple name are stored in the same cache) " +
                     "[expCls=" + desc.valueClass().getName() + ", actualCls=" + valCls.getName() + ']');
@@ -1616,7 +1615,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 obj = isKeyProp0 == 1 ? key : val;
             }
 
-            return ctx.cacheObjects().field(obj, propName);
+            Object res = ctx.cacheObjects().field(obj, propName);
+
+            return res;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerIndexingHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerIndexingHandler.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerIndexingHandler.java
index b48ce21..678e5b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerIndexingHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerIndexingHandler.java
@@ -30,10 +30,7 @@ import java.util.concurrent.*;
  */
 public class OptimizedMarshallerIndexingHandler {
     /** */
-    private final static OptimizedMarshallerIndexingHandler instance = new OptimizedMarshallerIndexingHandler();
-
-    /** */
-    private final static ConcurrentHashMap<Class<?>, Boolean> indexingEnabledCache = new ConcurrentHashMap<>();
+    private static final ConcurrentMap<Class<?>, Boolean> indexingEnabledCache = new ConcurrentHashMap<>();
 
     /** Class descriptors by class. */
     private ConcurrentMap<Class, OptimizedClassDescriptor> clsMap;
@@ -129,16 +126,18 @@ public class OptimizedMarshallerIndexingHandler {
      *
      * @param cls Class.
      * @return {@code true} if fields indexing is enabled.
-     * @throws IgniteCheckedException In case of error.
      */
-    public boolean enableFieldsIndexingForClass(Class<?> cls) throws IgniteCheckedException {
+    public boolean enableFieldsIndexingForClass(Class<?> cls) {
         if (!isFieldsIndexingSupported())
             return false;
 
         if (metaHandler == null)
             return false;
 
-        boolean res;
+        Boolean res = indexingEnabledCache.get(cls);
+
+        if (res != null)
+            return res;
 
         if (isFieldsIndexingExcludedForClass(cls))
             res = false;
@@ -164,54 +163,11 @@ public class OptimizedMarshallerIndexingHandler {
                     res = false;
 
             } catch (IOException e) {
-                throw new IgniteCheckedException("Failed to put meta for class: " + cls.getName(), e);
+                throw new IgniteException("Failed to put meta for class: " + cls.getName(), e);
             }
         }
 
-        synchronized (indexingEnabledCache) {
-            indexingEnabledCache.put(cls, res);
-        }
-
-        return res;
-    }
-
-    /**
-     * Checks whether fields indexing is enabled for objects of the given {@code cls}.
-     *
-     * @param cls Class.
-     * @return {@code true} if fields indexing is enabled.
-     */
-    public boolean fieldsIndexingEnabledForClass(Class<?> cls) {
-        if (!isFieldsIndexingSupported())
-            return false;
-
-        if (metaHandler == null)
-            return false;
-
-        Boolean res = indexingEnabledCache.get(cls);
-
-        if (res != null)
-            return res;
-
-        if (isFieldsIndexingExcludedForClass(cls))
-            res = false;
-        else if (OptimizedMarshalAware.class.isAssignableFrom(cls))
-            res = true;
-        else {
-            try {
-                OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper,
-                    this);
-
-                res = desc.fields() != null && desc.fields().fieldsIndexingSupported() && metaHandler != null &&
-                    metaHandler.metadata(desc.typeId()) != null;
-            } catch (IOException e) {
-                throw new IgniteException("Failed to load class description: " + cls);
-            }
-        }
-
-        synchronized (indexingEnabledCache) {
-            indexingEnabledCache.putIfAbsent(cls, res);
-        }
+        indexingEnabledCache.put(cls, res);
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexAnnotatedValue.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexAnnotatedValue.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexAnnotatedValue.java
new file mode 100644
index 0000000..a4cfc33
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexAnnotatedValue.java
@@ -0,0 +1,104 @@
+/*
+ * 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 org.apache.ignite.cache.query.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ *
+ */
+@SuppressWarnings({"ReturnOfDateField", "AssignmentToDateFieldFromParameter"})
+public class IndexAnnotatedValue implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    @QuerySqlField(index = true)
+    private int field1;
+
+    /** */
+    @QuerySqlField(index = true)
+    private String field2;
+
+    /** */
+    @QuerySqlField(index = true)
+    private Date field3;
+
+    /** */
+    @QuerySqlField
+    private Object field4;
+
+    /**
+     * @return Field1 value.
+     */
+    public int getField1() {
+        return field1;
+    }
+
+    /**
+     * @param field1 Field1 value.
+     */
+    public void setField1(int field1) {
+        this.field1 = field1;
+    }
+
+    /**
+     * @return Field2 value.
+     */
+    public String getField2() {
+        return field2;
+    }
+
+    /**
+     * @param field2 Field2 value.
+     */
+    public void setField2(String field2) {
+        this.field2 = field2;
+    }
+
+    /**
+     * @return Field3 value.
+     */
+    public Date getField3() {
+        return field3;
+    }
+
+    /**
+     * @param field3 Field3 value.
+     */
+    public void setField3(Date field3) {
+        this.field3 = field3;
+    }
+
+    /**
+     * @return Field4 value.
+     */
+    public Object getField4() {
+        return field4;
+    }
+
+    /**
+     * @param field4 Field4 value.
+     */
+    public void setField4(Object field4) {
+        this.field4 = field4;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexValue.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexValue.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexValue.java
new file mode 100644
index 0000000..b3da513
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/IndexValue.java
@@ -0,0 +1,99 @@
+/*
+ * 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.*;
+import java.util.*;
+
+/**
+ *
+ */
+@SuppressWarnings({"ReturnOfDateField", "AssignmentToDateFieldFromParameter"})
+public class IndexValue implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private int field1;
+
+    /** */
+    private String field2;
+
+    /** */
+    private Date field3;
+
+    /** */
+    private Object field4;
+
+    /**
+     * @return Field1 value.
+     */
+    public int getField1() {
+        return field1;
+    }
+
+    /**
+     * @param field1 Field1 value.
+     */
+    public void setField1(int field1) {
+        this.field1 = field1;
+    }
+
+    /**
+     * @return Field2 value.
+     */
+    public String getField2() {
+        return field2;
+    }
+
+    /**
+     * @param field2 Field2 value.
+     */
+    public void setField2(String field2) {
+        this.field2 = field2;
+    }
+
+    /**
+     * @return Field3 value.
+     */
+    public Date getField3() {
+        return field3;
+    }
+
+    /**
+     * @param field3 Field3 value.
+     */
+    public void setField3(Date field3) {
+        this.field3 = field3;
+    }
+
+    /**
+     * @return Field4 value.
+     */
+    public Object getField4() {
+        return field4;
+    }
+
+    /**
+     * @param field4 Field4 value.
+     */
+    public void setField4(Object field4) {
+        this.field4 = field4;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2105a48e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryNoServerClassesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryNoServerClassesSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryNoServerClassesSelfTest.java
new file mode 100644
index 0000000..4d9af70
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryNoServerClassesSelfTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.config.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import java.net.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteCacheQueryNoServerClassesSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final int NODES_CNT = 2;
+
+    /** */
+    private static final int CLIENT_IDX = NODES_CNT - 1;
+
+    /** */
+    private static final String ANNOTATED_CLASS_NAME = "org.apache.ignite.tests.p2p.cache.IndexAnnotatedValue";
+
+    /** */
+    private static final String PLAIN_CLASS_NAME = "org.apache.ignite.tests.p2p.cache.IndexValue";
+
+    /** */
+    private ClassLoader ldr;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        ldr = new GridTestExternalClassLoader(new URL[]{
+            new URL(GridTestProperties.getProperty("p2p.uri.cls"))});
+
+        startGrids(NODES_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids(true);
+
+        ldr = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(CLIENT_IDX).equals(gridName))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSqlQueryAnnotatedClass() throws Exception {
+        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>();
+
+        Class<?> valCls = ldr.loadClass(ANNOTATED_CLASS_NAME);
+
+        Thread.currentThread().setContextClassLoader(ldr);
+
+        cfg.setIndexedTypes(Integer.class, valCls);
+
+        IgniteCache<Object, Object> clientCache = ignite(CLIENT_IDX).createCache(cfg);
+
+        try {
+            for (int i = 0; i < 100; i++)
+                clientCache.put(i, value(valCls, i, "value" + i, new Date(i * 1000), "otherValue" + i));
+
+            // Check SQL query.
+            // TODO ignite-950.
+            List<Cache.Entry<Object, Object>> res = clientCache.query(new SqlQuery<>(valCls, "field1 >= 50")).getAll();
+
+            assertEquals(50, res.size());
+
+            // Check SQL fields query.
+            List<List<?>> rows = clientCache.query(new SqlFieldsQuery("select field1, field2 from IndexAnnotatedValue " +
+                "where field1 >= 50")).getAll();
+
+            assertEquals(50, rows.size());
+        }
+        finally {
+            ignite(CLIENT_IDX).destroyCache(null);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSqlQueryMetadata() throws Exception {
+        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>();
+
+        Class<?> valCls = ldr.loadClass(PLAIN_CLASS_NAME);
+
+        Thread.currentThread().setContextClassLoader(ldr);
+
+        CacheTypeMetadata meta = new CacheTypeMetadata();
+
+        meta.setKeyType(Integer.class);
+        meta.setValueType(valCls);
+
+        meta.setAscendingFields(F.asMap("field1", (Class<?>)Integer.class, "field2", String.class, "field3", Date.class));
+        meta.setQueryFields(F.<String, Class<?>>asMap("field4", String.class));
+
+        cfg.setTypeMetadata(F.asList(meta));
+
+        IgniteCache<Object, Object> clientCache = ignite(CLIENT_IDX).createCache(cfg);
+
+        try {
+            for (int i = 0; i < 100; i++)
+                clientCache.put(i, value(valCls, i, "value" + i, new Date(i * 1000), "otherValue" + i));
+
+            // Check SQL query.
+            // TODO ignite-950.
+//            List<Cache.Entry<Object, Object>> res = clientCache.query(new SqlQuery<>(valCls, "field1 >= 50")).getAll();
+//
+//            assertEquals(50, res.size());
+
+            // Check SQL fields query.
+            List<List<?>> rows = clientCache.query(new SqlFieldsQuery("select field1, field2 from IndexValue " +
+                "where field1 >= 50")).getAll();
+
+            assertEquals(50, rows.size());
+        }
+        finally {
+            ignite(CLIENT_IDX).destroyCache(null);
+        }
+    }
+
+    /**
+     * @param cls Class to instantiate.
+     * @param field1 Field 1 value.
+     * @param field2 Field 2 value.
+     * @param field3 Field 3 value.
+     * @param field4 Field 4 value.
+     * @return Filled object.
+     * @throws Exception If error occurred.
+     */
+    private Object value(Class<?> cls, int field1, String field2, Date field3, Object field4) throws Exception {
+        Object res = cls.newInstance();
+
+        GridTestUtils.setFieldValue(res, "field1", field1);
+        GridTestUtils.setFieldValue(res, "field2", field2);
+        GridTestUtils.setFieldValue(res, "field3", field3);
+        GridTestUtils.setFieldValue(res, "field4", field4);
+
+        return res;
+    }
+}