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 2015/11/03 14:42:11 UTC

[1/2] ignite git commit: IGNITE-1838: WIP on tests.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1838 4a19b5c92 -> 12e86e71a


IGNITE-1838: WIP on tests.


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

Branch: refs/heads/ignite-1838
Commit: 4fc3b861c1c432882b84cd6e45afa5ff252c13da
Parents: 4a19b5c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 3 16:22:54 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 3 16:22:54 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableFieldImpl.java    |  28 +-
 .../internal/portable/PortableObjectImpl.java   |   5 +-
 .../portable/PortableObjectOffheapImpl.java     |   5 +-
 .../internal/portable/PortableReaderExImpl.java |   2 +-
 .../internal/portable/PortableSchema.java       |   9 +-
 .../apache/ignite/portable/PortableField.java   |   7 +
 .../PortableFieldsAbstractSelfTest.java         | 337 +++++++++++++++++++
 .../portable/PortableFieldsOffheapSelfTest.java |  61 ++++
 .../IgnitePortableObjectsTestSuite.java         |   4 +
 9 files changed, 449 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableFieldImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableFieldImpl.java
index 12be55c..80b0a6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableFieldImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableFieldImpl.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.portable.PortableField;
 import org.apache.ignite.portable.PortableObject;
 
@@ -25,8 +27,12 @@ import org.apache.ignite.portable.PortableObject;
  */
 public class PortableFieldImpl implements PortableField {
     /** Well-known object schemas. */
+    @GridToStringExclude
     private final PortableSchemaRegistry schemas;
 
+    /** Field name. */
+    private final String fieldName;
+
     /** Pre-calculated field ID. */
     private final int fieldId;
 
@@ -34,18 +40,29 @@ public class PortableFieldImpl implements PortableField {
      * Constructor.
      *
      * @param schemas Schemas.
+     * @param fieldName Field name.
      * @param fieldId Field ID.
      */
-    public PortableFieldImpl(PortableSchemaRegistry schemas, int fieldId) {
+    public PortableFieldImpl(PortableSchemaRegistry schemas, String fieldName, int fieldId) {
+        assert schemas != null;
+        assert fieldName != null;
+        assert fieldId != 0;
+
         this.schemas = schemas;
+        this.fieldName = fieldName;
         this.fieldId = fieldId;
     }
 
     /** {@inheritDoc} */
+    @Override public String name() {
+        return fieldName;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean exists(PortableObject obj) {
         PortableObjectEx obj0 = (PortableObjectEx)obj;
 
-        return fieldOrder(obj0) != 0;
+        return fieldOrder(obj0) != PortableSchema.ORDER_NOT_FOUND;
     }
 
     /** {@inheritDoc} */
@@ -55,7 +72,7 @@ public class PortableFieldImpl implements PortableField {
 
         int order = fieldOrder(obj0);
 
-        return order != 0 ? (T)obj0.fieldByOrder(order) : null;
+        return order != PortableSchema.ORDER_NOT_FOUND ? (T)obj0.fieldByOrder(order) : null;
     }
 
     /**
@@ -79,4 +96,9 @@ public class PortableFieldImpl implements PortableField {
 
         return schema.order(fieldId);
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableFieldImpl.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
index 26076ed..bc75b6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -472,13 +473,15 @@ public final class PortableObjectImpl extends PortableObjectEx implements Extern
 
     /** {@inheritDoc} */
     @Override public PortableField fieldDescriptor(String fieldName) throws PortableException {
+        A.notNull(fieldName, "fieldName");
+
         int typeId = typeId();
 
         PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
 
         int fieldId = ctx.userTypeIdMapper(typeId).fieldId(typeId, fieldName);
 
-        return new PortableFieldImpl(schemaReg, fieldId);
+        return new PortableFieldImpl(schemaReg, fieldName, fieldId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
index c098834..bb79b74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -140,13 +141,15 @@ public class PortableObjectOffheapImpl extends PortableObjectEx implements Exter
 
     /** {@inheritDoc} */
     @Override public PortableField fieldDescriptor(String fieldName) throws PortableException {
+        A.notNull(fieldName, "fieldName");
+
         int typeId = typeId();
 
         PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId);
 
         int fieldId = ctx.userTypeIdMapper(typeId).fieldId(typeId, fieldName);
 
-        return new PortableFieldImpl(schemaReg, fieldId);
+        return new PortableFieldImpl(schemaReg, fieldName, fieldId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
index 061a9fd..eadfa94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
@@ -2633,7 +2633,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
 
             int order = schema.order(id);
 
-            if (order != 0) {
+            if (order != PortableSchema.ORDER_NOT_FOUND) {
                 int offsetPos = footerStart + order * (4 + offsetSize) + 4;
 
                 int pos = start + PortableUtils.fieldOffsetRelative(in, offsetPos, offsetSize);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
index 03be319..96a93f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
@@ -29,6 +29,9 @@ import java.util.Map;
  * - When there are more fields, we store them inside a hash map.
  */
 public class PortableSchema {
+    /** Order returned if field is not found. */
+    public static final int ORDER_NOT_FOUND = -1;
+
     /** Inline flag. */
     private final boolean inline;
 
@@ -179,12 +182,12 @@ public class PortableSchema {
             if (id == id7)
                 return 7;
 
-            return 0;
+            return ORDER_NOT_FOUND;
         }
         else {
-            Integer off = map.get(id);
+            Integer order = map.get(id);
 
-            return off != null ? off : 0;
+            return order != null ? order : ORDER_NOT_FOUND;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/main/java/org/apache/ignite/portable/PortableField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableField.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableField.java
index f8851ee..81a7424 100644
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableField.java
+++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableField.java
@@ -22,6 +22,13 @@ package org.apache.ignite.portable;
  */
 public interface PortableField {
     /**
+     * Get field's name.
+     *
+     * @return Name.
+     */
+    public String name();
+
+    /**
      * Check whether field exists in the object.
      *
      * @param obj Object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
new file mode 100644
index 0000000..bc0919b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
@@ -0,0 +1,337 @@
+/*
+ * 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.portable;
+
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.portable.PortableField;
+import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.Arrays;
+
+/**
+ * Contains tests for portable object fields.
+ */
+public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractTest {
+    /** Dummy metadata handler. */
+    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
+        @Override public void addMeta(int typeId, PortableMetadata meta) {
+            // No-op.
+        }
+
+        @Override public PortableMetadata metadata(int typeId) {
+            return null;
+        }
+    };
+
+    /** Marshaller. */
+    protected PortableMarshaller marsh;
+
+    /** Portable context. */
+    protected PortableContext ctx;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        ctx = new PortableContext(META_HND, null);
+
+        marsh = new PortableMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new PortableTypeConfiguration(TestObject.class.getName()),
+            new PortableTypeConfiguration(TestOuterObject.class.getName())
+        ));
+
+        marsh.setContext(new MarshallerContextTestImpl(null));
+
+        IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", ctx);
+    }
+
+    /**
+     * Test byte field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testByte() throws Exception {
+        check("fByte");
+    }
+
+    /**
+     * Test boolean field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBoolean() throws Exception {
+        check("fBool");
+    }
+
+    /**
+     * Test short field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testShort() throws Exception {
+        check("fShort");
+    }
+
+    /**
+     * Test char field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testChar() throws Exception {
+        check("fChar");
+    }
+    /**
+     * Test int field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testInt() throws Exception {
+        check("fInt");
+    }
+
+    /**
+     * Test long field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLong() throws Exception {
+        check("fLong");
+    }
+
+    /**
+     * Test int field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFloat() throws Exception {
+        check("fFloat");
+    }
+    /**
+     * Test int field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDouble() throws Exception {
+        check("fDouble");
+    }
+
+    /**
+     * Check field resolution in both normal and nested modes.
+     *
+     * @param fieldName Field name.
+     * @throws Exception If failed.
+     */
+    public void check(String fieldName) throws Exception {
+        checkNormal(fieldName);
+        checkNested(fieldName);
+    }
+
+    /**
+     * Check field.
+     *
+     * @param fieldName Field name.
+     * @throws Exception If failed.
+     */
+    private void checkNormal(String fieldName) throws Exception {
+        TestContext ctx = context(fieldName);
+
+        check0(fieldName, ctx);
+    }
+
+    /**
+     * Check nested field.
+     *
+     * @param fieldName Field name.
+     * @throws Exception If failed.
+     */
+    private void checkNested(String fieldName) throws Exception {
+        TestContext ctx = nestedContext(fieldName);
+
+        check0(fieldName, ctx);
+    }
+
+    /**
+     * Internal check routine.
+     *
+     * @param fieldName Field name.
+     * @param ctx Context.
+     * @throws Exception If failed.
+     */
+    private void check0(String fieldName, TestContext ctx) throws Exception {
+        Object expVal = U.field(ctx.obj, fieldName);
+
+        assertTrue(ctx.field.exists(ctx.portObj));
+
+        assertEquals(expVal, ctx.field.value(ctx.portObj));
+    }
+
+    /**
+     * Get test context.
+     *
+     * @param fieldName Field name.
+     * @return Test context.
+     * @throws Exception If failed.
+     */
+    private TestContext context(String fieldName) throws Exception {
+        TestObject obj = createObject();
+
+        PortableObjectEx portObj = toPortable(marsh, obj);
+
+        PortableField field = portObj.fieldDescriptor(fieldName);
+
+        return new TestContext(obj, portObj, field);
+    }
+
+    /**
+     * Get test context with nested test object.
+     *
+     * @param fieldName Field name.
+     * @return Test context.
+     * @throws Exception If failed.
+     */
+    private TestContext nestedContext(String fieldName) throws Exception {
+        TestObject obj = createObject();
+        TestOuterObject outObj = new TestOuterObject(obj);
+
+        PortableObjectEx portOutObj = toPortable(marsh, outObj);
+        PortableObjectEx portObj = portOutObj.field("fInner");
+
+        assert portObj != null;
+
+        PortableField field = portObj.fieldDescriptor(fieldName);
+
+        return new TestContext(obj, portObj, field);
+    }
+
+    /**
+     * Create test object.
+     *
+     * @return Test object.
+     */
+    private TestObject createObject() {
+        return new TestObject(0);
+    }
+
+    /**
+     * Convert object to portable object.
+     *
+     * @param marsh Marshaller.
+     * @param obj Object.
+     * @return Portable object.
+     * @throws Exception If failed.
+     */
+    protected abstract PortableObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception;
+
+    /**
+     * Outer test object.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public static class TestOuterObject {
+        /** Inner object. */
+        public TestObject fInner;
+
+        /**
+         * Default constructor.
+         */
+        public TestOuterObject() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param fInner Inner object.
+         */
+        public TestOuterObject(TestObject fInner) {
+            this.fInner = fInner;
+        }
+    }
+
+    /**
+     * Test object class, c
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public static class TestObject {
+        /** Primitive fields. */
+        public byte fByte;
+        public boolean fBool;
+        public short fShort;
+        public char fChar;
+        public int fInt;
+        public long fLong;
+        public float fFloat;
+        public double fDouble;
+
+        /**
+         * Default constructor.
+         */
+        public TestObject() {
+            // No-op.
+        }
+
+        /**
+         * Non-default constructor.
+         *
+         * @param ignore Ignored.
+         */
+        public TestObject(int ignore) {
+            fByte = 1;
+            fBool = true;
+            fShort = 2;
+            fChar = 3;
+            fInt = 4;
+            fLong = 5;
+            fFloat = 6.6f;
+            fDouble = 7.7;
+        }
+    }
+
+    /**
+     * Test context.
+     */
+    public static class TestContext {
+        /** Object. */
+        public final TestObject obj;
+
+        /** Portable object. */
+        public final PortableObjectEx portObj;
+
+        /** Field. */
+        public final PortableField field;
+
+        /**
+         * Constructor.
+         *
+         * @param obj Object.
+         * @param portObj Portable object.
+         * @param field Field.
+         */
+        public TestContext(TestObject obj, PortableObjectEx portObj, PortableField field) {
+            this.obj = obj;
+            this.portObj = portObj;
+            this.field = field;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsOffheapSelfTest.java
new file mode 100644
index 0000000..e421375
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsOffheapSelfTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.portable;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+import sun.misc.Unsafe;
+
+/**
+ * Field tests for heap-based portables.
+ */
+public class PortableFieldsOffheapSelfTest extends PortableFieldsAbstractSelfTest {
+    /** Unsafe instance. */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** Byte array offset for unsafe mechanics. */
+    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** Allocated unsafe pointer. */
+    private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        // Cleanup allocated objects.
+        for (Long ptr : ptrs)
+            UNSAFE.freeMemory(ptr);
+
+        ptrs.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected PortableObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
+        byte[] arr = marsh.marshal(obj);
+
+        long ptr = UNSAFE.allocateMemory(arr.length);
+
+        ptrs.add(ptr);
+
+        UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr, arr.length);
+
+        return new PortableObjectOffheapImpl(ctx, ptr, 0, arr.length);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc3b861/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
index ecd25e1..c7391a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@ -28,6 +28,8 @@ import org.apache.ignite.internal.portable.GridPortableMarshallerSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataDisabledSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataSelfTest;
 import org.apache.ignite.internal.portable.GridPortableWildcardsSelfTest;
+import org.apache.ignite.internal.portable.PortableFieldsHeapSelfTest;
+import org.apache.ignite.internal.portable.PortableFieldsOffheapSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataMultinodeTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStoreObjectsSelfTest;
@@ -60,6 +62,8 @@ public class IgnitePortableObjectsTestSuite extends TestSuite {
         suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
         suite.addTestSuite(GridPortableBuilderSelfTest.class);
         suite.addTestSuite(GridPortableBuilderStringAsCharsSelfTest.class);
+        suite.addTestSuite(PortableFieldsHeapSelfTest.class);
+        suite.addTestSuite(PortableFieldsOffheapSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataDisabledSelfTest.class);
         suite.addTestSuite(GridPortableAffinityKeySelfTest.class);


[2/2] ignite git commit: IGNITE-1838: WIP on tests.

Posted by vo...@apache.org.
IGNITE-1838: WIP on tests.


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

Branch: refs/heads/ignite-1838
Commit: 12e86e71aad7d2eed60843a10cf05141293fc299
Parents: 4fc3b86
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 3 16:42:53 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 3 16:42:53 2015 +0300

----------------------------------------------------------------------
 .../portable/PortableFIeldsHeapSelfTest.java    |  32 ++++
 .../PortableFieldsAbstractSelfTest.java         | 179 +++++++++++++++++--
 .../IgnitePortableObjectsTestSuite.java         |   4 +-
 3 files changed, 200 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/12e86e71/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFIeldsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFIeldsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFIeldsHeapSelfTest.java
new file mode 100644
index 0000000..aa88ecc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFIeldsHeapSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.portable;
+
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Field tests for heap-based portables.
+ */
+public class PortableFIeldsHeapSelfTest extends PortableFieldsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected PortableObjectEx toPortable(PortableMarshaller marsh, Object obj) throws Exception {
+        byte[] bytes = marsh.marshal(obj);
+
+        return new PortableObjectImpl(ctx, bytes, 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/12e86e71/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
index bc0919b..07b2745 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableFieldsAbstractSelfTest.java
@@ -23,10 +23,15 @@ import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.portable.PortableField;
 import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.portable.PortableObject;
 import org.apache.ignite.portable.PortableTypeConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.math.BigDecimal;
+import java.sql.Timestamp;
 import java.util.Arrays;
+import java.util.Date;
+import java.util.UUID;
 
 /**
  * Contains tests for portable object fields.
@@ -59,7 +64,8 @@ public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractT
 
         marsh.setTypeConfigurations(Arrays.asList(
             new PortableTypeConfiguration(TestObject.class.getName()),
-            new PortableTypeConfiguration(TestOuterObject.class.getName())
+            new PortableTypeConfiguration(TestOuterObject.class.getName()),
+            new PortableTypeConfiguration(TestInnerObject.class.getName())
         ));
 
         marsh.setContext(new MarshallerContextTestImpl(null));
@@ -121,15 +127,16 @@ public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractT
     }
 
     /**
-     * Test int field.
+     * Test float field.
      *
      * @throws Exception If failed.
      */
     public void testFloat() throws Exception {
         check("fFloat");
     }
+
     /**
-     * Test int field.
+     * Test double field.
      *
      * @throws Exception If failed.
      */
@@ -138,38 +145,115 @@ public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractT
     }
 
     /**
+     * Test string field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testString() throws Exception {
+        check("fString");
+    }
+
+    /**
+     * Test date field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDate() throws Exception {
+        check("fDate");
+    }
+
+    /**
+     * Test timestamp field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTimestamp() throws Exception {
+        check("fTimestamp");
+    }
+
+    /**
+     * Test UUID field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testUuid() throws Exception {
+        check("fUuid");
+    }
+
+    /**
+     * Test decimal field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDecimal() throws Exception {
+        check("fDecimal");
+    }
+
+    /**
+     * Test object field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testObject() throws Exception {
+        check("fObj");
+    }
+
+    /**
+     * Test null field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNull() throws Exception {
+        check("fNull");
+    }
+
+    /**
+     * Test missing field.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMissing() throws Exception {
+        String fieldName = "fMissing";
+
+        checkNormal(fieldName, false);
+        checkNested(fieldName, false);
+    }
+
+    /**
      * Check field resolution in both normal and nested modes.
      *
      * @param fieldName Field name.
      * @throws Exception If failed.
      */
     public void check(String fieldName) throws Exception {
-        checkNormal(fieldName);
-        checkNested(fieldName);
+        checkNormal(fieldName, true);
+        checkNested(fieldName, true);
     }
 
     /**
      * Check field.
      *
      * @param fieldName Field name.
+     * @param exists Whether field should exist.
      * @throws Exception If failed.
      */
-    private void checkNormal(String fieldName) throws Exception {
+    private void checkNormal(String fieldName, boolean exists) throws Exception {
         TestContext ctx = context(fieldName);
 
-        check0(fieldName, ctx);
+        check0(fieldName, ctx, exists);
     }
 
     /**
      * Check nested field.
      *
      * @param fieldName Field name.
+     * @param exists Whether field should exist.
      * @throws Exception If failed.
      */
-    private void checkNested(String fieldName) throws Exception {
+    private void checkNested(String fieldName, boolean exists) throws Exception {
         TestContext ctx = nestedContext(fieldName);
 
-        check0(fieldName, ctx);
+        check0(fieldName, ctx, exists);
     }
 
     /**
@@ -177,14 +261,27 @@ public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractT
      *
      * @param fieldName Field name.
      * @param ctx Context.
+     * @param exists Whether field should exist.
      * @throws Exception If failed.
      */
-    private void check0(String fieldName, TestContext ctx) throws Exception {
-        Object expVal = U.field(ctx.obj, fieldName);
+    private void check0(String fieldName, TestContext ctx, boolean exists) throws Exception {
+        Object val = ctx.field.value(ctx.portObj);
+
+        if (exists) {
+            assertTrue(ctx.field.exists(ctx.portObj));
 
-        assertTrue(ctx.field.exists(ctx.portObj));
+            Object expVal = U.field(ctx.obj, fieldName);
 
-        assertEquals(expVal, ctx.field.value(ctx.portObj));
+            if (val instanceof PortableObject)
+                val = ((PortableObject) val).deserialize();
+
+            assertEquals(expVal, val);
+        }
+        else {
+            assertFalse(ctx.field.exists(ctx.portObj));
+
+            assert val == null;
+        }
     }
 
     /**
@@ -284,6 +381,19 @@ public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractT
         public float fFloat;
         public double fDouble;
 
+        /** Special fields. */
+        public String fString;
+        public Date fDate;
+        public Timestamp fTimestamp;
+        public UUID fUuid;
+        public BigDecimal fDecimal;
+
+        /** Nested object. */
+        public TestInnerObject fObj;
+
+        /** Field which is always set to null. */
+        public Object fNull;
+
         /**
          * Default constructor.
          */
@@ -305,6 +415,49 @@ public abstract class PortableFieldsAbstractSelfTest extends GridCommonAbstractT
             fLong = 5;
             fFloat = 6.6f;
             fDouble = 7.7;
+
+            fString = "8";
+            fDate = new Date();
+            fTimestamp = new Timestamp(new Date().getTime());
+            fUuid = UUID.randomUUID();
+            fDecimal = new BigDecimal(9);
+
+            fObj = new TestInnerObject(10);
+        }
+    }
+
+    /**
+     * Inner test object.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public static class TestInnerObject {
+        /** Value. */
+        private int val;
+
+        /**
+         * Default constructor.
+         */
+        public TestInnerObject() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public TestInnerObject(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object other) {
+            return other != null && other instanceof TestInnerObject && val == ((TestInnerObject)(other)).val;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/12e86e71/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
index c7391a6..22f8b1e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@ -28,7 +28,7 @@ import org.apache.ignite.internal.portable.GridPortableMarshallerSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataDisabledSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataSelfTest;
 import org.apache.ignite.internal.portable.GridPortableWildcardsSelfTest;
-import org.apache.ignite.internal.portable.PortableFieldsHeapSelfTest;
+import org.apache.ignite.internal.portable.PortableFIeldsHeapSelfTest;
 import org.apache.ignite.internal.portable.PortableFieldsOffheapSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataMultinodeTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataTest;
@@ -62,7 +62,7 @@ public class IgnitePortableObjectsTestSuite extends TestSuite {
         suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
         suite.addTestSuite(GridPortableBuilderSelfTest.class);
         suite.addTestSuite(GridPortableBuilderStringAsCharsSelfTest.class);
-        suite.addTestSuite(PortableFieldsHeapSelfTest.class);
+        suite.addTestSuite(PortableFIeldsHeapSelfTest.class);
         suite.addTestSuite(PortableFieldsOffheapSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataDisabledSelfTest.class);