You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2015/10/09 02:37:37 UTC

[19/50] [abbrv] incubator-geode git commit: GEODE-324: defer deserialization of old value in PR put

GEODE-324: defer deserialization of old value in PR put

When a PR put returns the old value our internal messages were deserializing
the values before we get back to the user thread that initiated the operation.
If a thread local class loader is used for the class we need to load during
deserialization then the early deserialization will fail.

Now the deserialization is delayed until the serialized old value gets through
our messaging layer and back to the thread that performed the pr put.

Added junit tests that verify that both PutReplyMessage classes no longer
deserialize their old value.


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

Branch: refs/heads/feature/GEODE-11
Commit: d8570c980d10827a82fc94aaeaaddb47d7c6d242
Parents: 27eceb1
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Sep 10 11:43:26 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Sep 30 11:05:42 2015 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/DataSerializer.java    |  11 +-
 .../internal/InternalDataSerializer.java        |  33 ++--
 .../cache/DistributedCacheOperation.java        |   8 +-
 .../internal/cache/RemotePutMessage.java        |  56 ++++---
 .../internal/cache/partitioned/PutMessage.java  |  21 ++-
 .../offheap/OffHeapCachedDeserializable.java    |  20 +++
 .../offheap/SimpleMemoryAllocatorImpl.java      |  36 ++++
 .../gemfire/internal/offheap/StoredObject.java  |  26 ++-
 .../cache/OldValueImporterTestBase.java         | 165 +++++++++++++++++++
 .../cache/RemotePutReplyMessageJUnitTest.java   |  37 +++++
 .../PutPutReplyMessageJUnitTest.java            |  38 +++++
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java | 101 ++++++++++++
 .../gemfire/pdx/ByteSourceJUnitTest.java        |   4 +-
 .../sanctionedDataSerializables.txt             |   4 +-
 14 files changed, 511 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
index 8f977d8..685f546 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
@@ -1475,9 +1475,16 @@ public abstract class DataSerializer {
     throws IOException {
     Object object = obj;
     if (obj instanceof CachedDeserializable) {
-     // if( ( !(obj instanceof ByteSource) || ((StoredObject) obj).isSerialized())) {
+      if (obj instanceof StoredObject) {
+        StoredObject so = (StoredObject)obj;
+        if (logger.isTraceEnabled(LogMarker.SERIALIZER)) {
+          logger.trace(LogMarker.SERIALIZER, "writeObjectAsByteArray StoredObject");
+        }
+        so.sendAsByteArray(out);
+        return;
+      } else {
         object = ((CachedDeserializable) obj).getSerializedValue();
-     // }
+      }
     }
     if (logger.isTraceEnabled(LogMarker.SERIALIZER)) {
       if (object == null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
index 87ab28d..09f9280 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
@@ -1361,16 +1361,30 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
 
   ///////////////// START DataSerializer Implementation Methods ///////////
 
+  // Writes just the header of a DataSerializableFixedID to out.
+  public static final void writeDSFIDHeader(int dsfid, DataOutput out) throws IOException {
+    if (dsfid == DataSerializableFixedID.ILLEGAL) {
+      throw new IllegalStateException(LocalizedStrings.InternalDataSerializer_ATTEMPTED_TO_SERIALIZE_ILLEGAL_DSFID.toLocalizedString());
+    }
+   if (dsfid <= Byte.MAX_VALUE && dsfid >= Byte.MIN_VALUE) {
+      out.writeByte(DS_FIXED_ID_BYTE);
+      out.writeByte(dsfid);
+    } else if (dsfid <= Short.MAX_VALUE && dsfid >= Short.MIN_VALUE) {
+      out.writeByte(DS_FIXED_ID_SHORT);
+      out.writeShort(dsfid);
+    } else {
+      out.writeByte(DS_FIXED_ID_INT);
+      out.writeInt(dsfid);
+    }
+  }
+  
   public static final void writeDSFID(DataSerializableFixedID o, DataOutput out)
     throws IOException
   {
     int dsfid = o.getDSFID();
-    if (dsfid == DataSerializableFixedID.ILLEGAL) {
-      throw new IllegalStateException(LocalizedStrings.InternalDataSerializer_ATTEMPTED_TO_SERIALIZE_ILLEGAL_DSFID.toLocalizedString());
-    }
     if (dsfidToClassMap != null && logger.isTraceEnabled(LogMarker.DEBUG_DSFID)) {
       logger.trace(LogMarker.DEBUG_DSFID, "writeDSFID {} class={}", dsfid, o.getClass());
-      if (dsfid != DataSerializableFixedID.NO_FIXED_ID) {
+      if (dsfid != DataSerializableFixedID.NO_FIXED_ID && dsfid != DataSerializableFixedID.ILLEGAL) {
         // consistency check to make sure that the same DSFID is not used
         // for two different classes
         String newClassName = o.getClass().getName();
@@ -1380,18 +1394,11 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
         }
       }
     }
-    if (dsfid <= Byte.MAX_VALUE && dsfid >= Byte.MIN_VALUE) {
-      out.writeByte(DS_FIXED_ID_BYTE);
-      out.writeByte(dsfid);
-    } else if (dsfid <= Short.MAX_VALUE && dsfid >= Short.MIN_VALUE) {
-      out.writeByte(DS_FIXED_ID_SHORT);
-      out.writeShort(dsfid);
-    } else if (dsfid == DataSerializableFixedID.NO_FIXED_ID) {
+    if (dsfid == DataSerializableFixedID.NO_FIXED_ID) {
       out.writeByte(DS_NO_FIXED_ID);
       DataSerializer.writeClass(o.getClass(), out);
     } else {
-      out.writeByte(DS_FIXED_ID_INT);
-      out.writeInt(dsfid);
+      writeDSFIDHeader(dsfid, out);
     }
     try {
       invokeToData(o, out);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
index 692595c..fc9046c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
@@ -65,7 +65,7 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.OffHeapReference;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 
@@ -117,9 +117,9 @@ public abstract class DistributedCacheOperation {
         DataSerializer.writeObject(vObj, out);
       } else if (deserializationPolicy == DESERIALIZATION_POLICY_NONE) {
         // We only have NONE with a vObj when vObj is off-heap and not serialized.
-        OffHeapReference ohref = (OffHeapReference) vObj;
-        assert !ohref.isSerialized();
-        DataSerializer.writeByteArray(ohref.getValueAsHeapByteArray(), out);
+        StoredObject so = (StoredObject) vObj;
+        assert !so.isSerialized();
+        so.sendAsByteArray(out);
       } else { // LAZY
         // TODO OFFHEAP MERGE: cache the oldValue that is serialized here
         // into the event

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
index 3f5efeb..a879adf 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
@@ -54,7 +54,6 @@ import com.gemstone.gemfire.internal.util.Breadcrumbs;
 
 import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_OLD_VALUE;
 import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
-
 import static com.gemstone.gemfire.internal.cache.DistributedCacheOperation.VALUE_IS_BYTES;
 import static com.gemstone.gemfire.internal.cache.DistributedCacheOperation.VALUE_IS_SERIALIZED_OBJECT;
 import static com.gemstone.gemfire.internal.cache.DistributedCacheOperation.VALUE_IS_OBJECT;
@@ -880,7 +879,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
      * or null if not set.
      */
     @Unretained(ENTRY_EVENT_OLD_VALUE)
-    Object oldValue;
+    private Object oldValue;
     
     /**
      * version tag for concurrency control
@@ -898,7 +897,8 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
     public PutReplyMessage() {
     }
 
-    private PutReplyMessage(int processorId,
+    // unit tests may call this constructor
+    PutReplyMessage(int processorId,
                             boolean result,
                             Operation op,
                             ReplyException ex,
@@ -965,13 +965,15 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       dm.getStats().incReplyMessageTime(NanoTimer.getTime()-startTime);
     }
 
-    /** Return oldValue in deserialized form */
+    /** Return oldValue as a byte[] or as a CachedDeserializable.
+     * This method used to deserialize a CachedDeserializable but that is too soon.
+     * This method is called during message processing. The deserialization needs
+     * to be deferred until we get back to the application thread which happens
+     * for this oldValue when they call EntryEventImpl.getOldValue.
+     */
     public Object getOldValue() {
       // oldValue field is in serialized form, either a CachedDeserializable,
       // a byte[], or null if not set
-      if (this.oldValue instanceof CachedDeserializable) {
-        return ((CachedDeserializable)this.oldValue).getDeserializedValue(null, null);
-      }
       return this.oldValue;
     }
 
@@ -994,7 +996,29 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       }
     }
 
-    @Override
+    public static void oldValueToData(DataOutput out, Object ov, boolean ovIsSerialized) throws IOException {
+      if (ovIsSerialized && ov != null) {
+        byte[] oldValueBytes;
+        if (ov instanceof byte[]) {
+          oldValueBytes = (byte[]) ov;
+          DataSerializer.writeObject(new VMCachedDeserializable(oldValueBytes), out);
+        } else if (ov instanceof CachedDeserializable) {
+          if (ov instanceof StoredObject) {
+            ((StoredObject) ov).sendAsCachedDeserializable(out);
+          } else {
+            DataSerializer.writeObject(ov, out);
+          }
+        } else {
+          oldValueBytes = EntryEventImpl.serialize(ov);
+          DataSerializer.writeObject(new VMCachedDeserializable(oldValueBytes), out);
+        }
+      } else {
+        DataSerializer.writeObject(ov, out);
+      }
+      
+    }
+    
+     @Override
     public void toData(DataOutput out) throws IOException {
       super.toData(out);
       byte flags = 0;
@@ -1003,12 +1027,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       if (this.versionTag instanceof DiskVersionTag) flags |= FLAG_PERSISTENT;
       out.writeByte(flags);
       out.writeByte(this.op.ordinal);
-      if (this.oldValueIsSerialized) {
-        byte[] oldValueBytes = (byte[]) this.oldValue;
-        out.write(oldValueBytes);
-      } else {
-        DataSerializer.writeObject(this.oldValue, out);
-      }
+      oldValueToData(out, getOldValue(), this.oldValueIsSerialized);
       if (this.versionTag != null) {
         InternalDataSerializer.invokeToData(this.versionTag, out);
       }
@@ -1045,18 +1064,13 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
 
     @Override
     public void importOldObject(@Unretained(ENTRY_EVENT_OLD_VALUE) Object ov, boolean isSerialized) {
-      // isSerialized does not matter.
-      // toData will just call writeObject
-      // and fromData will just call readObject
+      this.oldValueIsSerialized = isSerialized;
       this.oldValue = ov;
     }
 
     @Override
     public void importOldBytes(byte[] ov, boolean isSerialized) {
-      if (isSerialized) {
-        this.oldValueIsSerialized = true;
-      }
-      this.oldValue = ov;
+      importOldObject(ov, isSerialized);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
index 1f7b08e..e47ce04 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
@@ -51,6 +51,8 @@ import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
+import com.gemstone.gemfire.internal.cache.RemotePutMessage;
+import com.gemstone.gemfire.internal.cache.VMCachedDeserializable;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -1036,6 +1038,14 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
 
     VersionTag versionTag;
 
+    /**
+     * Set to true by the import methods if the oldValue
+     * is already serialized. In that case toData
+     * should just copy the bytes to the stream.
+     * In either case fromData just calls readObject.
+     */
+    private transient boolean oldValueIsSerialized;
+
     @Override
     public boolean getInlineProcess() {
       return true;
@@ -1047,7 +1057,8 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     public PutReplyMessage() {
     }
     
-    private PutReplyMessage(int processorId,
+    // package access for unit test
+    PutReplyMessage(int processorId,
                             boolean result,
                             Operation op,
                             ReplyException ex,
@@ -1139,12 +1150,13 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
       this.versionTag = (VersionTag)DataSerializer.readObject(in);
     }
 
-    @Override
+   @Override
     public void toData(DataOutput out) throws IOException {
       super.toData(out);
       out.writeBoolean(this.result);
       out.writeByte(this.op.ordinal);
-      DataSerializer.writeObject(this.oldValue, out);
+      Object ov = getOldValue();
+      RemotePutMessage.PutReplyMessage.oldValueToData(out, getOldValue(), this.oldValueIsSerialized);
       DataSerializer.writeObject(this.versionTag, out);
     }
 
@@ -1180,11 +1192,12 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     @Override
     public void importOldObject(@Unretained(ENTRY_EVENT_OLD_VALUE) Object ov, boolean isSerialized) {
       this.oldValue = ov;
+      this.oldValueIsSerialized = isSerialized;
     }
 
     @Override
     public void importOldBytes(byte[] ov, boolean isSerialized) {
-      this.oldValue = ov;
+      importOldObject(ov, isSerialized);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
index 764a051..4d22171 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
@@ -7,6 +7,8 @@ import java.util.Arrays;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
 import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
 import com.gemstone.gemfire.internal.cache.EntryBits;
@@ -106,6 +108,24 @@ public abstract class OffHeapCachedDeserializable implements MemoryChunkWithRefC
       DataSerializer.writeObject(objToSend, out);
     }
   }
+  @Override
+  public void sendAsByteArray(DataOutput out) throws IOException {
+    byte[] bytes;
+    if (isSerialized()) {
+      bytes = getSerializedValue();
+    } else {
+      bytes = (byte[]) getDeserializedForReading();
+    }
+    DataSerializer.writeByteArray(bytes, out);
+  }
+  @Override
+  public void sendAsCachedDeserializable(DataOutput out) throws IOException {
+    if (!isSerialized()) {
+      throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
+    }
+    InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
+    sendAsByteArray(out);
+  }
   public boolean checkDataEquals(@Unretained OffHeapCachedDeserializable other) {
     if (this == other) {
       return true;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index 7cf1656..4652620 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -42,6 +42,7 @@ import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
@@ -527,6 +528,27 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     }
 
     @Override
+    public void sendAsByteArray(DataOutput out) throws IOException {
+      byte[] bytes;
+      if (isSerialized()) {
+        bytes = getSerializedValue();
+      } else {
+        bytes = (byte[]) getDeserializedForReading();
+      }
+      DataSerializer.writeByteArray(bytes, out);
+      
+    }
+    
+    @Override
+    public void sendAsCachedDeserializable(DataOutput out) throws IOException {
+      if (!isSerialized()) {
+        throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
+      }
+      InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
+      sendAsByteArray(out);
+    }
+
+    @Override
     public boolean isSerialized() {
       return OffHeapRegionEntryHelper.isSerialized(this.address);
     }
@@ -1816,6 +1838,20 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
       }
       super.sendTo(out);
     }
+    
+    @Override
+    public void sendAsByteArray(DataOutput out) throws IOException {
+      if (!isCompressed() && out instanceof HeapDataOutputStream) {
+        ByteBuffer bb = createDirectByteBuffer();
+        if (bb != null) {
+          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
+          InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
+          hdos.write(bb);
+          return;
+        }
+      }
+      super.sendAsByteArray(out);
+    }
        
     private static volatile Class dbbClass = null;
     private static volatile Constructor dbbCtor = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
index 8f179ae..90b65e9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
@@ -1,5 +1,8 @@
 package com.gemstone.gemfire.internal.offheap;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
 
 /**
@@ -11,5 +14,26 @@ import com.gemstone.gemfire.internal.cache.CachedDeserializable;
  * @since 9.0
  */
 public interface StoredObject extends OffHeapReference, CachedDeserializable {
-
+  /**
+   * Take all the bytes in the object and write them to the data output as a byte array.
+   * If the StoredObject is not serialized then its raw byte array is sent.
+   * But if it is serialized then the serialized byte array is sent.
+   * The corresponding de-serialization will need to call readByteArray.
+   * 
+   * @param out
+   *          the data output to send this object to
+   * @throws IOException
+   */
+  void sendAsByteArray(DataOutput out) throws IOException;
+  /**
+   * Take all the bytes in the object and write them to the data output as a byte array.
+   * If the StoredObject is not serialized then an exception will be thrown.
+   * The corresponding deserialization will need to call readObject and will get an
+   * instance of VMCachedDeserializable.
+   * 
+   * @param out
+   *          the data output to send this object to
+   * @throws IOException
+   */
+  void sendAsCachedDeserializable(DataOutput out) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
new file mode 100644
index 0000000..a2f6ef1
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
@@ -0,0 +1,165 @@
+package com.gemstone.gemfire.internal.cache;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
+import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.util.BlobHelper;
+
+public abstract class OldValueImporterTestBase {
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+  
+  protected abstract OldValueImporter createImporter();
+  protected abstract Object getOldValueFromImporter(OldValueImporter ovi);
+  protected abstract void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException;
+  protected abstract void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException;
+
+  @Test
+  public void testValueSerialization() throws IOException, ClassNotFoundException {
+    byte[] bytes = new byte[1024];
+    HeapDataOutputStream hdos = new HeapDataOutputStream(bytes);
+    OldValueImporter imsg = createImporter();
+
+    // null byte array value
+    {
+      OldValueImporter omsg = createImporter();
+      omsg.importOldBytes(null, false);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertEquals(null, getOldValueFromImporter(imsg));
+    }
+    
+    // null object value
+    {
+      OldValueImporter omsg = createImporter();
+      omsg.importOldObject(null, true);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertEquals(null, getOldValueFromImporter(imsg));
+    }
+    
+    // simple byte array
+    {
+      byte[] baValue = new byte[] {1,2,3,4,5,6,7,8,9};
+      OldValueImporter omsg = createImporter();
+      omsg.importOldBytes(baValue, false);
+      hdos = new HeapDataOutputStream(bytes);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertArrayEquals(baValue, (byte[])getOldValueFromImporter(imsg));
+    }
+    
+    // String in serialized form
+    {
+      String stringValue = "1,2,3,4,5,6,7,8,9";
+      byte[] stringValueBlob = EntryEventImpl.serialize(stringValue);
+      OldValueImporter omsg = createImporter();
+      omsg.importOldBytes(stringValueBlob, true);
+      hdos = new HeapDataOutputStream(bytes);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+    }
+    
+    // String in object form
+    {
+      String stringValue = "1,2,3,4,5,6,7,8,9";
+      byte[] stringValueBlob = EntryEventImpl.serialize(stringValue);
+      OldValueImporter omsg = createImporter();
+      omsg.importOldObject(stringValue, true);
+      hdos = new HeapDataOutputStream(bytes);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+    }
+    
+    // off-heap DataAsAddress byte array
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        byte[] baValue = new byte[] {1,2};
+        DataAsAddress baValueSO = (DataAsAddress) sma.allocateAndInitialize(baValue, false, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, false);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValue, (byte[])getOldValueFromImporter(imsg));
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    // off-heap Chunk byte array
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        byte[] baValue = new byte[] {1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17};
+        Chunk baValueSO = (Chunk) sma.allocateAndInitialize(baValue, false, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, false);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValue, (byte[])getOldValueFromImporter(imsg));
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    // off-heap DataAsAddress String
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        String baValue = "12";
+        byte[] baValueBlob = BlobHelper.serializeToBlob(baValue);
+        DataAsAddress baValueSO = (DataAsAddress) sma.allocateAndInitialize(baValueBlob, true, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, true);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    // off-heap Chunk String
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        String baValue = "12345678";
+        byte[] baValueBlob = BlobHelper.serializeToBlob(baValue);
+        Chunk baValueSO = (Chunk) sma.allocateAndInitialize(baValueBlob, true, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, true);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java
new file mode 100644
index 0000000..4c1ae33
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java
@@ -0,0 +1,37 @@
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.cache.RemotePutMessage.PutReplyMessage;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class RemotePutReplyMessageJUnitTest extends OldValueImporterTestBase {
+
+  @Override
+  protected OldValueImporter createImporter() {
+    return new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null);
+  }
+
+  @Override
+  protected Object getOldValueFromImporter(OldValueImporter ovi) {
+    return ((PutReplyMessage)ovi).getOldValue();
+  }
+
+  @Override
+  protected void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException {
+    ((PutReplyMessage)ovi).toData(hdos);
+  }
+
+  @Override
+  protected void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException {
+    ((PutReplyMessage)ovi).fromData(new DataInputStream(new ByteArrayInputStream(bytes)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java
new file mode 100644
index 0000000..5e89aa8
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java
@@ -0,0 +1,38 @@
+package com.gemstone.gemfire.internal.cache.partitioned;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.cache.OldValueImporterTestBase;
+import com.gemstone.gemfire.internal.cache.partitioned.PutMessage.PutReplyMessage;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PutPutReplyMessageJUnitTest extends OldValueImporterTestBase {
+
+  @Override
+  protected OldValueImporter createImporter() {
+    return new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null);
+  }
+
+  @Override
+  protected Object getOldValueFromImporter(OldValueImporter ovi) {
+    return ((PutReplyMessage)ovi).getOldValue();
+  }
+
+  @Override
+  protected void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException {
+    ((PutReplyMessage)ovi).toData(hdos);
+  }
+
+  @Override
+  protected void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException {
+    ((PutReplyMessage)ovi).fromData(new DataInputStream(new ByteArrayInputStream(bytes)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
new file mode 100644
index 0000000..2b5f5bf
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
@@ -0,0 +1,101 @@
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
+import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapWriteObjectAsByteArrayJUnitTest {
+
+  @Before
+  public void setUp() throws Exception {
+    SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+  
+  private StoredObject createStoredObject(byte[] bytes, boolean isSerialized, boolean isCompressed) {
+    return SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, isCompressed, null);
+  }
+  
+  private DataInputStream createInput(HeapDataOutputStream hdos) {
+    ByteArrayInputStream bais = new ByteArrayInputStream(hdos.toByteArray());
+    return new DataInputStream(bais);
+  }
+  
+  @Test
+  public void testByteArrayChunk() throws IOException, ClassNotFoundException {
+    byte[] expected = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16};
+    StoredObject so = createStoredObject(expected, false, false);
+    assertTrue(so instanceof Chunk);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+  }
+  
+  @Test
+  public void testByteArrayDataAsAddress() throws IOException, ClassNotFoundException {
+    byte[] expected = new byte[] {1, 2, 3};
+    StoredObject so = createStoredObject(expected, false, false);
+    assertTrue(so instanceof DataAsAddress);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+  }
+  
+  @Test
+  public void testStringChunk() throws IOException, ClassNotFoundException {
+    byte[] expected = EntryEventImpl.serialize("1234567890");
+    StoredObject so = createStoredObject(expected, true, false);
+    assertTrue(so instanceof Chunk);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+    assertNoMoreInput(in);
+  }
+  
+  @Test
+  public void testStringDataAsAddress() throws IOException, ClassNotFoundException {
+    byte[] expected = EntryEventImpl.serialize("1234");
+    StoredObject so = createStoredObject(expected, true, false);
+    assertTrue(so instanceof DataAsAddress);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+  }
+  
+  private void assertNoMoreInput(DataInputStream in) throws IOException {
+    assertEquals(0, in.available());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
index 0ef1bed..fc05868 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
@@ -19,9 +19,9 @@ import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSourceFactory;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-@Category(IntegrationTest.class)
+@Category(UnitTest.class)
 public class ByteSourceJUnitTest {
   
   protected ByteSource createByteSource(byte[] bytes) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index 4dd927d..79f5e22 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -1244,7 +1244,7 @@ toData,252,2a03b500082a2bb700852ab600862bb800872ab400043d2ab40024c600091cb200758
 
 com/gemstone/gemfire/internal/cache/RemotePutMessage$PutReplyMessage,2
 fromData,81,2a2bb700272bb9002801001100ff7e913d2a1c047e99000704a7000403b500032a2bb900280100b80029b500022a2bb8002ab500061c057e9900181c077e99000704a70004033e2a1d2bb8002bb50007b1
-toData,118,2a2bb7002c033d2ab400039900081c0480913d2ab40007c600081c0580913d2ab40007c1002d9900081c0780913d2b1cb9002e02002b2ab40002b4002fb9002e02002ab400309900182ab40006c00031c000314e2b2db900320200a7000b2ab400062bb800332ab40007c6000b2ab400072bb80034b1
+toData,94,2a2bb70032033d2ab400039900081c0480913d2ab40007c600081c0580913d2ab40007c100339900081c0780913d2b1cb9003402002b2ab40002b40035b9003402002b2ab600362ab40037b800382ab40007c6000b2ab400072bb80039b1
 
 com/gemstone/gemfire/internal/cache/RemoteRegionOperation,2
 fromData,20,2a2bb700292ab8002a2bb9002b010032b5000cb1
@@ -1770,7 +1770,7 @@ toData,374,014d2ab4001fb60072b9007301003e2ab40009b60074c600161d9900122ab40008990
 
 com/gemstone/gemfire/internal/cache/partitioned/PutMessage$PutReplyMessage,2
 fromData,48,2a2bb700242a2bb900250100b500032a2bb900260100b80027b500022a2bb80028b500062a2bb80028c00029b50007b1
-toData,45,2a2bb7002a2b2ab40003b9002b02002b2ab40002b4002cb9002d02002ab400062bb8002e2ab400072bb8002eb1
+toData,54,2a2bb7002a2b2ab40003b9002b02002b2ab40002b4002cb9002d02002ab6002e4d2b2ab6002e2ab4002fb800302ab400072bb80031b1
 
 com/gemstone/gemfire/internal/cache/partitioned/QueryMessage,2
 fromData,63,2a2bb7007b2a2bb8007cb5000a2a2bb8007db5000b2a2bb8007eb5000c2a2bb8007fb60080b5000e2a2bb8007fb60080b500792a2bb8007fb60080b50011b1