You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/12/28 06:40:23 UTC

[2/3] ignite git commit: IGNITE-2257: Fixed BinaryContext serialization.

IGNITE-2257: Fixed BinaryContext serialization.


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

Branch: refs/heads/ignite-843-rc2
Commit: 18c413c41b51994fb2eeaaebf4cf727049db0c8b
Parents: 49c2988
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Dec 25 09:48:28 2015 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Fri Dec 25 09:48:28 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryContext.java   | 44 +----------
 .../internal/binary/BinaryEnumObjectImpl.java   |  4 +-
 .../internal/binary/BinaryObjectImpl.java       | 27 ++++---
 .../internal/binary/GridBinaryMarshaller.java   | 82 +++++++++++++++++++-
 4 files changed, 94 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/18c413c4/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 5c63fbd..163532d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -31,10 +31,8 @@ import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey;
-import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
 import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -51,10 +49,6 @@ import org.jsr166.ConcurrentHashMap8;
 import java.io.Externalizable;
 import java.io.File;
 import java.io.IOException;
-import java.io.InvalidObjectException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.ObjectStreamException;
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.net.URISyntaxException;
@@ -81,10 +75,7 @@ import java.util.jar.JarFile;
 /**
  * Binary context.
  */
-public class BinaryContext implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
+public class BinaryContext {
     /** */
     private static final ClassLoader dfltLdr = U.gridClassLoader();
 
@@ -125,9 +116,6 @@ public class BinaryContext implements Externalizable {
     private MarshallerContext marshCtx;
 
     /** */
-    private String gridName;
-
-    /** */
     private IgniteConfiguration igniteCfg;
 
     /** Logger. */
@@ -162,8 +150,6 @@ public class BinaryContext implements Externalizable {
         this.igniteCfg = igniteCfg;
         this.log = log;
 
-        gridName = igniteCfg.getGridName();
-
         colTypes.put(ArrayList.class, GridBinaryMarshaller.ARR_LIST);
         colTypes.put(LinkedList.class, GridBinaryMarshaller.LINKED_LIST);
         colTypes.put(HashSet.class, GridBinaryMarshaller.HASH_SET);
@@ -727,34 +713,6 @@ public class BinaryContext implements Externalizable {
         return null;
     }
 
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeString(out, igniteCfg.getGridName());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        gridName = U.readString(in);
-    }
-
-    /**
-     * @return Binary context.
-     * @throws ObjectStreamException In case of error.
-     */
-    protected Object readResolve() throws ObjectStreamException {
-        try {
-            IgniteKernal g = IgnitionEx.gridx(gridName);
-
-            if (g == null)
-                throw new IllegalStateException("Failed to find grid for name: " + gridName);
-
-            return ((CacheObjectBinaryProcessorImpl)g.context().cacheObjects()).binaryContext();
-        }
-        catch (IllegalStateException e) {
-            throw U.withCause(new InvalidObjectException(e.getMessage()), e);
-        }
-    }
-
     /**
      * @param cls Class.
      * @param id Type ID.

http://git-wip-us.apache.org/repos/asf/ignite/blob/18c413c4/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 001d2d7..536c582 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
@@ -183,8 +183,6 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(ctx);
-
         out.writeInt(typeId);
         out.writeObject(clsName);
         out.writeInt(ord);
@@ -192,7 +190,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        ctx = (BinaryContext)in.readObject();
+        ctx = GridBinaryMarshaller.threadLocalContext();
 
         typeId = in.readInt();
         clsName = (String)in.readObject();

http://git-wip-us.apache.org/repos/asf/ignite/blob/18c413c4/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index fca8955..44c7a08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -17,16 +17,6 @@
 
 package org.apache.ignite.internal.binary;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -43,7 +33,18 @@ 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;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
+
+import static java.nio.charset.StandardCharsets.*;
 
 /**
  * Binary object implementation.
@@ -427,8 +428,6 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(ctx);
-
         if (detachAllowed) {
             int len = length();
 
@@ -445,7 +444,7 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        ctx = (BinaryContext)in.readObject();
+        ctx = GridBinaryMarshaller.threadLocalContext();
 
         arr = new byte[in.readInt()];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/18c413c4/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
index 71ca018..b4f4291 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
@@ -17,10 +17,15 @@
 
 package org.apache.ignite.internal.binary;
 
+import org.apache.ignite.IgniteIllegalStateException;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -34,6 +39,9 @@ public class GridBinaryMarshaller {
         }
     };
 
+    /** Binary context in TLS store. */
+    private static final ThreadLocal<BinaryContext> BINARY_CTX = new ThreadLocal<>();
+
     /** */
     static final byte OPTM_MARSH = -2;
 
@@ -237,7 +245,14 @@ public class GridBinaryMarshaller {
     @Nullable public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws BinaryObjectException {
         assert bytes != null;
 
-        return (T)BinaryUtils.unmarshal(BinaryHeapInputStream.create(bytes, 0), ctx, clsLdr);
+        BinaryContext oldCtx = pushContext(ctx);
+
+        try {
+            return (T) BinaryUtils.unmarshal(BinaryHeapInputStream.create(bytes, 0), ctx, clsLdr);
+        }
+        finally {
+            popContext(oldCtx);
+        }
     }
 
     /**
@@ -247,7 +262,14 @@ public class GridBinaryMarshaller {
      */
     @SuppressWarnings("unchecked")
     @Nullable public <T> T unmarshal(BinaryInputStream in) throws BinaryObjectException {
-        return (T)BinaryUtils.unmarshal(in, ctx, null);
+        BinaryContext oldCtx = pushContext(ctx);
+
+        try {
+            return (T)BinaryUtils.unmarshal(in, ctx, null);
+        }
+        finally {
+            popContext(oldCtx);
+        }
     }
 
     /**
@@ -264,7 +286,40 @@ public class GridBinaryMarshaller {
         if (arr[0] == NULL)
             return null;
 
-        return (T)new BinaryReaderExImpl(ctx, BinaryHeapInputStream.create(arr, 0), ldr).deserialize();
+        BinaryContext oldCtx = pushContext(ctx);
+
+        try {
+            return (T)new BinaryReaderExImpl(ctx, BinaryHeapInputStream.create(arr, 0), ldr).deserialize();
+        }
+        finally {
+            popContext(oldCtx);
+        }
+    }
+
+    /**
+     * Push binary context and return the old one.
+     *
+     * @param ctx Binary context.
+     * @return Old binary context.
+     */
+    @Nullable private static BinaryContext pushContext(BinaryContext ctx) {
+        BinaryContext old = BINARY_CTX.get();
+
+        BINARY_CTX.set(ctx);
+
+        return old;
+    }
+
+    /**
+     * Pop binary context and restore the old one.
+     *
+     * @param oldCtx Old binary context.
+     */
+    private static void popContext(@Nullable BinaryContext oldCtx) {
+        if (oldCtx == null)
+            BINARY_CTX.remove();
+        else
+            BINARY_CTX.set(oldCtx);
     }
 
     /**
@@ -293,4 +348,25 @@ public class GridBinaryMarshaller {
     public BinaryContext context() {
         return ctx;
     }
+
+    /**
+     * @return Thread-bound context.
+     */
+    public static BinaryContext threadLocalContext() {
+        BinaryContext ctx = GridBinaryMarshaller.BINARY_CTX.get();
+
+        if (ctx == null) {
+            IgniteKernal ignite = IgnitionEx.localIgnite();
+
+            IgniteCacheObjectProcessor proc = ignite.context().cacheObjects();
+
+            if (proc instanceof CacheObjectBinaryProcessorImpl)
+                return ((CacheObjectBinaryProcessorImpl)proc).binaryContext();
+            else
+                throw new IgniteIllegalStateException("Ignite instance must be started with " +
+                    BinaryMarshaller.class.getName() + " [name=" + ignite.name() + ']');
+        }
+
+        return ctx;
+    }
 }