Return-Path: X-Original-To: apmail-geode-commits-archive@minotaur.apache.org Delivered-To: apmail-geode-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 8368A18F7D for ; Thu, 24 Sep 2015 23:50:02 +0000 (UTC) Received: (qmail 47068 invoked by uid 500); 24 Sep 2015 23:50:02 -0000 Delivered-To: apmail-geode-commits-archive@geode.apache.org Received: (qmail 47036 invoked by uid 500); 24 Sep 2015 23:50:02 -0000 Mailing-List: contact commits-help@geode.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@geode.incubator.apache.org Delivered-To: mailing list commits@geode.incubator.apache.org Received: (qmail 47027 invoked by uid 99); 24 Sep 2015 23:50:02 -0000 Received: from Unknown (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 24 Sep 2015 23:50:02 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id EFB0F1A06F9 for ; Thu, 24 Sep 2015 23:50:01 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.782 X-Spam-Level: * X-Spam-Status: No, score=1.782 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=0.001, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-us-west.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id 7joA6vQGf9va for ; Thu, 24 Sep 2015 23:49:58 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-us-west.apache.org (ASF Mail Server at mx1-us-west.apache.org) with SMTP id F03DD231B8 for ; Thu, 24 Sep 2015 23:49:57 +0000 (UTC) Received: (qmail 46850 invoked by uid 99); 24 Sep 2015 23:49:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 24 Sep 2015 23:49:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 92670E09D2; Thu, 24 Sep 2015 23:49:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: dschneider@apache.org To: commits@geode.incubator.apache.org Date: Thu, 24 Sep 2015 23:49:59 -0000 Message-Id: In-Reply-To: <123886935a834535911fc5c4b28424a5@git.apache.org> References: <123886935a834535911fc5c4b28424a5@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [3/3] incubator-geode git commit: PutReplyMessage now defers deserialization of its value PutReplyMessage now defers deserialization of its 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/7d5db1aa Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/7d5db1aa Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/7d5db1aa Branch: refs/heads/feature/GEODE-324 Commit: 7d5db1aa6192ef98abe1e519e736c82461d6b104 Parents: d5ceac5 Author: Darrel Schneider Authored: Thu Sep 24 16:48:54 2015 -0700 Committer: Darrel Schneider Committed: Thu Sep 24 16:48:54 2015 -0700 ---------------------------------------------------------------------- .../internal/InternalDataSerializer.java | 33 ++-- .../internal/cache/RemotePutMessage.java | 37 +++-- .../offheap/OffHeapCachedDeserializable.java | 10 ++ .../offheap/SimpleMemoryAllocatorImpl.java | 10 ++ .../gemfire/internal/offheap/StoredObject.java | 11 ++ .../cache/PutReplyMessageJUnitTest.java | 165 +++++++++++++++++++ 6 files changed, 240 insertions(+), 26 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d5db1aa/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/7d5db1aa/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..20e3725 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 @@ -898,7 +898,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 +966,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; } @@ -1004,8 +1007,20 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl out.writeByte(flags); out.writeByte(this.op.ordinal); if (this.oldValueIsSerialized) { - byte[] oldValueBytes = (byte[]) this.oldValue; - out.write(oldValueBytes); + byte[] oldValueBytes; + if (this.oldValue instanceof byte[]) { + oldValueBytes = (byte[]) this.oldValue; + DataSerializer.writeObject(new VMCachedDeserializable(oldValueBytes), out); + } else if (this.oldValue instanceof CachedDeserializable) { + if (this.oldValue instanceof StoredObject) { + ((StoredObject) this.oldValue).sendAsCachedDeserializable(out); + } else { + DataSerializer.writeObject(this.oldValue, out); + } + } else { + oldValueBytes = EntryEventImpl.serialize(this.oldValue); + DataSerializer.writeObject(new VMCachedDeserializable(oldValueBytes), out); + } } else { DataSerializer.writeObject(this.oldValue, out); } @@ -1045,17 +1060,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.oldValueIsSerialized = isSerialized; this.oldValue = ov; } } http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d5db1aa/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 4b25da9..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; @@ -116,6 +118,14 @@ public abstract class OffHeapCachedDeserializable implements MemoryChunkWithRefC } 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/7d5db1aa/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 9d9a087..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; @@ -537,6 +538,15 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI 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() { http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d5db1aa/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 6415e7f..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 @@ -25,4 +25,15 @@ public interface StoredObject extends OffHeapReference, CachedDeserializable { * @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/7d5db1aa/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PutReplyMessageJUnitTest.java ---------------------------------------------------------------------- diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PutReplyMessageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PutReplyMessageJUnitTest.java new file mode 100644 index 0000000..e6a9efb --- /dev/null +++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PutReplyMessageJUnitTest.java @@ -0,0 +1,165 @@ +package com.gemstone.gemfire.internal.cache; + +import static org.junit.Assert.*; + +import java.io.ByteArrayInputStream; +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.cache.Operation; +import com.gemstone.gemfire.internal.HeapDataOutputStream; +import com.gemstone.gemfire.internal.cache.RemotePutMessage.PutReplyMessage; +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.internal.util.BlobHelper; +import com.gemstone.gemfire.test.junit.categories.UnitTest; + +@Category(UnitTest.class) +public class PutReplyMessageJUnitTest { + @Before + public void setUp() throws Exception { + } + + @After + public void tearDown() throws Exception { + } + + @Test + public void testValueSerialization() throws IOException, ClassNotFoundException { + byte[] bytes = new byte[1024]; + HeapDataOutputStream hdos = new HeapDataOutputStream(bytes); + PutReplyMessage imsg = new PutReplyMessage(); + + // null value + { + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertEquals(null, imsg.oldValue); + assertEquals(null, imsg.getOldValue()); + } + + // simple byte array + { + byte[] baValue = new byte[] {1,2,3,4,5,6,7,8,9}; + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldBytes(baValue, false); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(baValue, (byte[])imsg.oldValue); + assertArrayEquals(baValue, (byte[])imsg.getOldValue()); + } + + // String in serialized form + { + String stringValue = "1,2,3,4,5,6,7,8,9"; + byte[] stringValueBlob = EntryEventImpl.serialize(stringValue); + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldBytes(stringValueBlob, true); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)imsg.oldValue).getSerializedValue()); + assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)imsg.getOldValue()).getSerializedValue()); + } + + // String in object form + { + String stringValue = "1,2,3,4,5,6,7,8,9"; + byte[] stringValueBlob = EntryEventImpl.serialize(stringValue); + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldObject(stringValue, true); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)imsg.oldValue).getSerializedValue()); + assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)imsg.getOldValue()).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); + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldObject(baValueSO, false); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(baValue, (byte[])imsg.oldValue); + assertArrayEquals(baValue, (byte[])imsg.getOldValue()); + } 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); + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldObject(baValueSO, false); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(baValue, (byte[])imsg.oldValue); + assertArrayEquals(baValue, (byte[])imsg.getOldValue()); + } 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); + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldObject(baValueSO, true); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(baValueBlob, ((VMCachedDeserializable)imsg.oldValue).getSerializedValue()); + assertArrayEquals(baValueBlob, ((VMCachedDeserializable)imsg.getOldValue()).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); + PutReplyMessage omsg = new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null); + omsg.importOldObject(baValueSO, true); + hdos = new HeapDataOutputStream(bytes); + omsg.toData(hdos); + imsg.fromData(new DataInputStream(new ByteArrayInputStream(bytes))); + assertArrayEquals(baValueBlob, ((VMCachedDeserializable)imsg.oldValue).getSerializedValue()); + assertArrayEquals(baValueBlob, ((VMCachedDeserializable)imsg.getOldValue()).getSerializedValue()); + } finally { + SimpleMemoryAllocatorImpl.freeOffHeapMemory(); + } + } + } +}