You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2016/06/02 00:09:49 UTC

[02/50] [abbrv] incubator-geode git commit: removed preferObject, eager deserialize, and KeyWithRegionContext

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
index e1766dc..726246b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.internal.cache.DataLocationException;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-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.PartitionedRegionHelper;
@@ -126,10 +125,6 @@ public class PRUpdateEntryVersionMessage extends
   protected boolean operateOnPartitionedRegion(DistributionManager dm,
       PartitionedRegion pr, long startTime) throws CacheException,
       QueryException, DataLocationException, InterruptedException, IOException {
-    if (pr.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext) key).setRegionContext(pr);
-    }
-
     // release not needed because disallowOffHeapValues called
     final EntryEventImpl event = EntryEventImpl.create(pr, getOperation(),
         getKey(), null, /* newValue */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
index 8aaf587..3120403 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
@@ -59,7 +59,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
@@ -270,10 +269,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
       EntryVersionsList versionTags = new EntryVersionsList(putAllPRDataSize);
 
       boolean hasTags = false;
-      // get the "keyRequiresRegionContext" flag from first element assuming
-      // all key objects to be uniform
-      final boolean requiresRegionContext =
-        (this.putAllPRData[0].getKey() instanceof KeyWithRegionContext);
       for (int i = 0; i < this.putAllPRDataSize; i++) {
         // If sender's version is >= 7.0.1 then we can send versions list.
         if (!hasTags && putAllPRData[i].versionTag != null) {
@@ -283,7 +278,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
         VersionTag<?> tag = putAllPRData[i].versionTag;
         versionTags.add(tag);
         putAllPRData[i].versionTag = null;
-        putAllPRData[i].toData(out, requiresRegionContext);
+        putAllPRData[i].toData(out);
         putAllPRData[i].versionTag = tag;
         // PutAllEntryData's toData did not serialize eventID to save
         // performance for DR, but in PR,
@@ -436,12 +431,8 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     // Fix the updateMsg misorder issue
     // Lock the keys when doing postPutAll
     Object keys[] = new Object[putAllPRDataSize];
-    final boolean keyRequiresRegionContext = r.keyRequiresRegionContext();
     for (int i = 0; i < putAllPRDataSize; ++i) {
       keys[i] = putAllPRData[i].getKey();
-      if (keyRequiresRegionContext) {
-        ((KeyWithRegionContext)keys[i]).setRegionContext(r);
-      }
     }
 
     if (!notificationOnly) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
index 3340cec..1bdd6dd 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
@@ -56,7 +56,6 @@ import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-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;
@@ -119,8 +118,7 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
   
   /**
    * Indicates if and when the new value should be deserialized on the
-   * the receiver. Distinguishes between Deltas which need to be eagerly
-   * deserialized (DESERIALIZATION_POLICY_EAGER), a non-byte[] value that was
+   * the receiver. Distinguishes between a non-byte[] value that was
    * serialized (DESERIALIZATION_POLICY_LAZY) and a
    * byte[] array value that didn't need to be serialized
    * (DESERIALIZATION_POLICY_NONE). While this seems like an extra data, it
@@ -349,11 +347,7 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     this.expectedOldValue = expectedOldValue;
     this.key = event.getKey();
     if (event.hasNewValue()) {
-      if (CachedDeserializableFactory.preferObject()) {
-        this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER;
-      } else {
-        this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
-      }
+      this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
       event.exportNewValue(this);
     }
     else {
@@ -627,14 +621,7 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
       this.deltaBytes = DataSerializer.readByteArray(in);
     }
     else {
-      // for eager deserialization avoid extra byte array serialization
-      if (this.deserializationPolicy ==
-          DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-        setValObj(DataSerializer.readObject(in));
-      }
-      else {
-        setValBytes(DataSerializer.readByteArray(in));
-      }
+      setValBytes(DataSerializer.readByteArray(in));
       if ((extraFlags & HAS_DELTA_WITH_FULL_VALUE) != 0) {
         this.deltaBytes = DataSerializer.readByteArray(in);
       }
@@ -788,9 +775,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     if (eventSender == null) {
        eventSender = getSender();
     }
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)this.key).setRegionContext(r);
-    }
     @Released final EntryEventImpl ev = EntryEventImpl.create(
         r,
         getOperation(),
@@ -839,10 +823,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
         case DistributedCacheOperation.DESERIALIZATION_POLICY_NONE:
           ev.setNewValue(getValBytes());
           break;
-        case DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER:
-          // new value is a Delta
-          ev.setNewValue(this.valObj); // sets the delta field
-          break;
         default:
           throw new AssertionError("unknown deserialization policy: "
               + deserializationPolicy);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
index ac95e03..bea08fc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
@@ -49,7 +49,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.RemoteOperationException;
@@ -421,14 +420,9 @@ public class RemoteFetchKeysMessage extends RemoteOperationMessage {
         
         ByteArrayInputStream byteStream = new ByteArrayInputStream(msg.chunk);
         DataInputStream in = new DataInputStream(byteStream);
-        final boolean requiresRegionContext = this.region
-            .keyRequiresRegionContext();
         while (in.available() > 0) {
           Object key = DataSerializer.readObject(in);
           if (key != null) {
-            if (requiresRegionContext) {
-              ((KeyWithRegionContext)key).setRegionContext(this.region);
-            }
             synchronized(returnValue) {
               returnValue.add(key);
             }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
index fb46e0c..252a6ab 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
@@ -59,7 +59,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
@@ -260,10 +259,6 @@ public final class RemoveAllPRMessage extends PartitionMessageWithDirectReply
       EntryVersionsList versionTags = new EntryVersionsList(removeAllPRDataSize);
 
       boolean hasTags = false;
-      // get the "keyRequiresRegionContext" flag from first element assuming
-      // all key objects to be uniform
-      final boolean requiresRegionContext =
-        (this.removeAllPRData[0].getKey() instanceof KeyWithRegionContext);
       for (int i = 0; i < this.removeAllPRDataSize; i++) {
         // If sender's version is >= 7.0.1 then we can send versions list.
         if (!hasTags && removeAllPRData[i].versionTag != null) {
@@ -273,7 +268,7 @@ public final class RemoveAllPRMessage extends PartitionMessageWithDirectReply
         VersionTag<?> tag = removeAllPRData[i].versionTag;
         versionTags.add(tag);
         removeAllPRData[i].versionTag = null;
-        removeAllPRData[i].toData(out, requiresRegionContext);
+        removeAllPRData[i].toData(out);
         removeAllPRData[i].versionTag = tag;
         // RemoveAllEntryData's toData did not serialize eventID to save
         // performance for DR, but in PR,
@@ -423,12 +418,8 @@ public final class RemoveAllPRMessage extends PartitionMessageWithDirectReply
     // Fix the updateMsg misorder issue
     // Lock the keys when doing postRemoveAll
     Object keys[] = new Object[removeAllPRDataSize];
-    final boolean keyRequiresRegionContext = r.keyRequiresRegionContext();
     for (int i = 0; i < removeAllPRDataSize; ++i) {
       keys[i] = removeAllPRData[i].getKey();
-      if (keyRequiresRegionContext) {
-        ((KeyWithRegionContext)keys[i]).setRegionContext(r);
-      }
     }
 
     if (!notificationOnly) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
index 448be92..e3a3d53 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
@@ -39,7 +39,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.EventIDHolder;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
@@ -320,9 +319,6 @@ public class GatewayReceiverCommand extends BaseCommand {
             try {
               byte[] value = valuePart.getSerializedForm();
               boolean isObject = valuePart.isObject();
-              if (region.keyRequiresRegionContext()) {
-                ((KeyWithRegionContext)key).setRegionContext(region);
-              }
               // [sumedh] This should be done on client while sending
               // since that is the WAN gateway
               AuthorizeRequest authzRequest = servConn.getAuthzRequest();
@@ -428,9 +424,6 @@ public class GatewayReceiverCommand extends BaseCommand {
             try {
               byte[] value = valuePart.getSerializedForm();
               boolean isObject = valuePart.isObject();
-              if (region.keyRequiresRegionContext()) {
-                ((KeyWithRegionContext)key).setRegionContext(region);
-              }
               AuthorizeRequest authzRequest = servConn.getAuthzRequest();
               if (authzRequest != null) {
                 PutOperationContext putContext = authzRequest.putAuthorize(
@@ -523,9 +516,6 @@ public class GatewayReceiverCommand extends BaseCommand {
             }
             handleMessageRetry(region, clientEvent);
             // Destroy the entry
-            if (region.keyRequiresRegionContext()) {
-              ((KeyWithRegionContext)key).setRegionContext(region);
-            }
             try {
               AuthorizeRequest authzRequest = servConn.getAuthzRequest();
               if (authzRequest != null) {
@@ -606,9 +596,6 @@ public class GatewayReceiverCommand extends BaseCommand {
               }
               
               // Update the version tag
-              if (region.keyRequiresRegionContext()) {
-                ((KeyWithRegionContext) key).setRegionContext(region);
-              }
               try {
 
                 region.basicBridgeUpdateVersionStamp(key, callbackArg, servConn.getProxyID(), false, clientEvent);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
index 6e7c21c..c84fb99 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
@@ -138,12 +138,7 @@ public class DistTxEntryEvent extends EntryEventImpl {
     EntryVersionsList versionTags = new EntryVersionsList(
         this.putAllOp.putAllDataSize);
     boolean hasTags = false;
-    // get the "keyRequiresRegionContext" flag from first element assuming
-    // all key objects to be uniform
     final PutAllEntryData[] putAllData = this.putAllOp.getPutAllEntryData();
-    // final boolean requiresRegionContext =
-    // (putAllData[0].key instanceof KeyWithRegionContext);
-    final boolean requiresRegionContext = false;
     for (int i = 0; i < this.putAllOp.putAllDataSize; i++) {
       if (!hasTags && putAllData[i].versionTag != null) {
         hasTags = true;
@@ -151,7 +146,7 @@ public class DistTxEntryEvent extends EntryEventImpl {
       VersionTag<?> tag = putAllData[i].versionTag;
       versionTags.add(tag);
       putAllData[i].versionTag = null;
-      putAllData[i].toData(out, requiresRegionContext);
+      putAllData[i].toData(out);
       putAllData[i].versionTag = tag;
     }
     out.writeBoolean(hasTags);
@@ -206,13 +201,8 @@ public class DistTxEntryEvent extends EntryEventImpl {
         this.removeAllOp.removeAllDataSize);
   
     boolean hasTags = false;
-    // get the "keyRequiresRegionContext" flag from first element assuming
-    // all key objects to be uniform
-    // final boolean requiresRegionContext =
-    // (this.removeAllData[0].key instanceof KeyWithRegionContext);
     final RemoveAllEntryData[] removeAllData = this.removeAllOp
         .getRemoveAllEntryData();
-    final boolean requiresRegionContext = false;
     for (int i = 0; i < this.removeAllOp.removeAllDataSize; i++) {
       if (!hasTags && removeAllData[i].versionTag != null) {
         hasTags = true;
@@ -220,7 +210,7 @@ public class DistTxEntryEvent extends EntryEventImpl {
       VersionTag<?> tag = removeAllData[i].versionTag;
       versionTags.add(tag);
       removeAllData[i].versionTag = null;
-      removeAllData[i].toData(out, requiresRegionContext);
+      removeAllData[i].toData(out);
       removeAllData[i].versionTag = tag;
     }
     out.writeBoolean(hasTags);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
index 2079616..445b243 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
@@ -515,10 +515,6 @@ public class GatewaySenderEventImpl implements
    *  //OFFHEAP TODO: Optimize callers by returning a reference to the off heap value
    */
   public Object getValue() {
-    if (CachedDeserializableFactory.preferObject()) {
-      // sqlf does not use CacheDeserializable wrappers
-      return getDeserializedValue();
-    }
     Object rawValue = this.value;
     if (rawValue == null) {
       rawValue = this.substituteValue;
@@ -535,9 +531,6 @@ public class GatewaySenderEventImpl implements
     if (valueIsObject == 0x00) {
       //if the value is a byte array, just return it
       return rawValue;
-    } else if (CachedDeserializableFactory.preferObject()) {
-      // sqlf does not use CacheDeserializable wrappers
-      return rawValue;
     } else if (rawValue instanceof byte[]) {
       return CachedDeserializableFactory.create((byte[]) rawValue);
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a6be9497/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
index e08a268..bc974f7 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
@@ -73,7 +73,6 @@ public class FetchEntriesMessageJUnitTest {
     cache = Fakes.cache();
     PartitionedRegion pr = mock(PartitionedRegion.class);
     InternalDistributedSystem system = cache.getDistributedSystem();
-    when(pr.keyRequiresRegionContext()).thenReturn(false);
 
     FetchEntriesResponse response = new FetchEntriesResponse(system, pr, null, 0);
     HeapDataOutputStream chunkStream = createDummyChunk();