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 2016/03/02 16:30:38 UTC

ignite git commit: ignite-2734 Binary enum type deserialization from offheap

Repository: ignite
Updated Branches:
  refs/heads/master 9200ec8bb -> 213ee5a63


ignite-2734 Binary enum type deserialization from offheap


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

Branch: refs/heads/master
Commit: 213ee5a63fa6328de804ada0321bedcedefa6bc1
Parents: 9200ec8
Author: agura <ag...@gridgain.com>
Authored: Mon Feb 29 18:39:45 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Mar 2 18:29:55 2016 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryEnumObjectImpl.java   | 41 ++++++++++++++--
 .../internal/processors/cache/CacheObject.java  |  3 ++
 .../binary/CacheObjectBinaryProcessorImpl.java  |  3 +-
 .../internal/binary/BinaryEnumsSelfTest.java    | 51 ++++++++++++++++++--
 .../cache/CacheEnumOperationsAbstractTest.java  | 32 ++++++++++++
 ...idCacheReplicatedPreloadOffHeapSelfTest.java |  3 +-
 .../GridCacheReplicatedPreloadSelfTest.java     | 16 ++++--
 7 files changed, 136 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index 2940828..e58edd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -36,6 +36,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 /**
  * Binary enum object.
  */
@@ -81,6 +83,40 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     }
 
     /**
+     * @param ctx Context.
+     * @param arr Array.
+     */
+    public BinaryEnumObjectImpl(BinaryContext ctx, byte[] arr) {
+        assert ctx != null;
+        assert arr != null;
+        assert arr[0] == GridBinaryMarshaller.ENUM;
+
+        this.ctx = ctx;
+
+        int off = 1;
+
+        this.typeId = BinaryPrimitives.readInt(arr, off);
+
+        off += 4;
+
+        if (this.typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) {
+            assert arr[off] == GridBinaryMarshaller.STRING;
+
+            int len = BinaryPrimitives.readInt(arr, ++off);
+
+            off += 4;
+
+            byte[] bytes = BinaryPrimitives.readByteArray(arr, off, len);
+
+            off += len;
+
+            this.clsName = new String(bytes, UTF_8);
+        }
+
+        this.ord = BinaryPrimitives.readInt(arr, off);
+    }
+
+    /**
      * @return Class name.
      */
     @Nullable public String className() {
@@ -168,9 +204,8 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
             type = null;
         }
 
-        if (type != null) {
+        if (type != null)
             return type.typeName() + "[ordinal=" + ord  + ']';
-        }
         else {
             if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID)
                 return "BinaryEnum[clsName=" + clsName + ", ordinal=" + ord + ']';
@@ -207,7 +242,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
 
     /** {@inheritDoc} */
     @Override public byte cacheObjectType() {
-        return TYPE_BINARY;
+        return TYPE_BINARY_ENUM;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
index 2385335..92c8676 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
@@ -34,6 +34,9 @@ public interface CacheObject extends Message {
     /** */
     public static final byte TYPE_BINARY = 100;
 
+    /** */
+    public static final byte TYPE_BINARY_ENUM = 101;
+
     /**
      * @param ctx Context.
      * @param cpy If {@code true} need to copy value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 624a453..18e2d09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -73,7 +73,6 @@ import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheUtils;
@@ -811,6 +810,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     @Override public CacheObject toCacheObject(CacheObjectContext ctx, byte type, byte[] bytes) {
         if (type == BinaryObjectImpl.TYPE_BINARY)
             return new BinaryObjectImpl(binaryContext(), bytes, 0);
+        else if (type == BinaryObjectImpl.TYPE_BINARY_ENUM)
+            return new BinaryEnumObjectImpl(binaryContext(), bytes);
 
         return super.toCacheObject(ctx, type, bytes);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
index 6bd9b65..fb7e618 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
@@ -17,19 +17,22 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.io.Serializable;
+import java.util.Arrays;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import java.io.Serializable;
-import java.util.Arrays;
-
 /**
  * Contains tests for binary enums.
  */
@@ -185,6 +188,46 @@ public class BinaryEnumsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testInstanceFromBytes() throws Exception {
+        startUp(true);
+
+        BinaryContext binCtx =
+            ((CacheObjectBinaryProcessorImpl)((IgniteKernal)node1).context().cacheObjects()).binaryContext();
+
+        int ord = EnumType.ONE.ordinal();
+
+        String clsName = EnumType.class.getName();
+
+        checkInstanceFromBytes(binCtx, ord, GridBinaryMarshaller.UNREGISTERED_TYPE_ID, clsName);
+
+        checkInstanceFromBytes(binCtx, ord, 42, null);
+    }
+
+    /**
+     * @param binCtx Binary context.
+     * @param ord Enum ordinal.
+     * @param typeId Type Id.
+     * @param clsName Class name.
+     */
+    private void checkInstanceFromBytes(BinaryContext binCtx, int ord, int typeId, String clsName)
+        throws IgniteCheckedException {
+
+        BinaryEnumObjectImpl srcBinEnum =new BinaryEnumObjectImpl(binCtx, typeId, clsName, ord);
+
+        Marshaller marsh = node1.configuration().getMarshaller();
+
+        byte[] bytes = marsh.marshal(srcBinEnum);
+
+        BinaryEnumObjectImpl binEnum = new BinaryEnumObjectImpl(binCtx, bytes);
+
+        assertEquals(clsName, binEnum.className());
+        assertEquals(typeId, binEnum.typeId());
+        assertEquals(ord, binEnum.enumOrdinal());
+    }
+
+    /**
      * Check simple serialization - deserialization.
      *
      * @param registered If type should be registered in advance.
@@ -439,7 +482,7 @@ public class BinaryEnumsSelfTest extends GridCommonAbstractTest {
     /**
      * Enumeration for tests.
      */
-    public static enum EnumType {
+    public enum EnumType {
         ONE,
         TWO
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
index 3957e46..87a9b91 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -20,12 +20,16 @@ package org.apache.ignite.internal.processors.cache;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.marshaller.Marshaller;
 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;
@@ -184,42 +188,52 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
         assertTrue(cache.putIfAbsent(key, TestEnum.VAL1));
 
         assertEquals(TestEnum.VAL1, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL1);
 
         assertFalse(cache.putIfAbsent(key, TestEnum.VAL2));
 
         assertEquals(TestEnum.VAL1, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL1);
 
         assertTrue(cache.replace(key, TestEnum.VAL2));
 
         assertEquals(TestEnum.VAL2, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL2);
 
         assertFalse(cache.replace(key, TestEnum.VAL1, TestEnum.VAL3));
 
         assertEquals(TestEnum.VAL2, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL2);
 
         assertTrue(cache.replace(key, TestEnum.VAL2, TestEnum.VAL3));
 
         assertEquals(TestEnum.VAL3, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL3);
 
         assertEquals(TestEnum.VAL3, cache.getAndPut(key, TestEnum.VAL1));
 
         assertEquals(TestEnum.VAL1, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL1);
 
         assertEquals(TestEnum.VAL1, cache.invoke(key, new EnumProcessor(TestEnum.VAL2, TestEnum.VAL1)));
 
         assertEquals(TestEnum.VAL2, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL2);
 
         assertEquals(TestEnum.VAL2, cache.getAndReplace(key, TestEnum.VAL3));
 
         assertEquals(TestEnum.VAL3, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL3);
 
         assertEquals(TestEnum.VAL3, cache.getAndPutIfAbsent(key, TestEnum.VAL1));
 
         assertEquals(TestEnum.VAL3, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL3);
 
         cache.put(key, TestEnum.VAL1);
 
         assertEquals(TestEnum.VAL1, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL1);
 
         assertEquals(TestEnum.VAL1, cache.getAndRemove(key));
 
@@ -230,6 +244,24 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
         assertNull(cache.getAndPutIfAbsent(key, TestEnum.VAL1));
 
         assertEquals(TestEnum.VAL1, cache.get(key));
+        assertBinaryEnum(cache, key, TestEnum.VAL1);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     * @param expVal Expected value.
+     */
+    private static void assertBinaryEnum(IgniteCache<Object, Object> cache, int key, TestEnum expVal) {
+        Marshaller marsh = ((IgniteCacheProxy)cache).context().marshaller();
+
+        if (marsh instanceof BinaryMarshaller) {
+            BinaryObject enumObj = (BinaryObject)cache.withKeepBinary().get(key);
+
+            assertEquals(expVal.ordinal(), enumObj.enumOrdinal());
+            assertTrue(enumObj.type().isEnum());
+            assertTrue(enumObj instanceof BinaryEnumObjectImpl);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadOffHeapSelfTest.java
index 8b0f710..bd3846e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadOffHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadOffHeapSelfTest.java
@@ -26,8 +26,7 @@ import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
  */
 public class GridCacheReplicatedPreloadOffHeapSelfTest extends GridCacheReplicatedPreloadSelfTest {
     /** {@inheritDoc} */
-    @Override
-    CacheConfiguration cacheConfiguration(String gridName) {
+    @Override CacheConfiguration cacheConfiguration(String gridName) {
         CacheConfiguration cacheCfg = super.cacheConfiguration(gridName);
 
         cacheCfg.setMemoryMode(OFFHEAP_VALUES);

http://git-wip-us.apache.org/repos/asf/ignite/blob/213ee5a6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index 7508e3d..ea2f27b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -34,6 +34,7 @@ import javax.cache.event.CacheEntryListener;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteMessaging;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
@@ -44,8 +45,8 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
-import org.apache.ignite.events.EventAdapter;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -58,6 +59,7 @@ 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.spi.eventstorage.memory.MemoryEventStorageSpi;
+import org.apache.ignite.testframework.config.GridTestProperties;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -93,6 +95,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     /** */
     private volatile boolean useExtClassLoader = false;
 
+    /** Disable p2p. */
     private volatile boolean disableP2p = false;
 
     /** */
@@ -171,9 +174,8 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
         cacheCfg.setRebalanceBatchSize(batchSize);
         cacheCfg.setRebalanceThreadPoolSize(poolSize);
 
-        if (extClassloadingAtCfg) {
+        if (extClassloadingAtCfg)
             loadExternalClassesToCfg(cacheCfg);
-        }
 
         return cacheCfg;
     }
@@ -328,6 +330,14 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
             Object e2 = cache2.get(2);
 
+            if (g1.configuration().getMarshaller() instanceof BinaryMarshaller) {
+                BinaryObject enumObj = (BinaryObject)cache2.withKeepBinary().get(2);
+
+                assertEquals(0, enumObj.enumOrdinal());
+                assertTrue(enumObj.type().isEnum());
+                assertTrue(enumObj instanceof BinaryEnumObjectImpl);
+            }
+
             assert e2 != null;
             assert e2.toString().equals(e1.toString());
             assert !e2.getClass().getClassLoader().equals(getClass().getClassLoader());