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

[2/2] incubator-geode git commit: GEODE-73: Removing unnecessary P2P backward-compatibility serialization code

GEODE-73: Removing unnecessary P2P backward-compatibility serialization code

Geode 1.0 will not be backward-compatible with GemFire 8.X (and older) peers
due to upgrading to Apache 2.0 licensed JGroups.  This removes the
serialization code that maintained that backward-compatibility.

This does not remove backward-compatibility code for client/server
communications, so older GemFire clients can still be used with Geode 1.0.


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

Branch: refs/heads/develop
Commit: f7b2714c14a6eeab4e810a85f65d650483f2ce6f
Parents: 60caab4
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Feb 2 13:56:36 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Feb 2 13:58:02 2016 -0800

----------------------------------------------------------------------
 .../distributed/internal/StartupMessage.java    |  12 -
 .../internal/StartupResponseMessage.java        |  16 --
 .../admin/remote/RemoteBridgeServer.java        |   8 +-
 .../admin/remote/RemoteRegionAttributes.java    | 116 +---------
 .../cache/DistributedCacheOperation.java        |  11 +-
 .../cache/DistributedClearOperation.java        |   8 +-
 .../cache/DistributedPutAllOperation.java       |   2 +-
 .../cache/DistributedRemoveAllOperation.java    |   2 +-
 .../cache/DistributedTombstoneOperation.java    |  15 +-
 .../internal/cache/InitialImageOperation.java   |  88 +++----
 .../gemfire/internal/cache/QueuedOperation.java |   9 +-
 .../internal/cache/RemoteOperationMessage.java  |  14 +-
 .../internal/cache/RemotePutAllMessage.java     |  10 +-
 .../internal/cache/control/ResourceAdvisor.java | 137 ++---------
 .../cache/partitioned/FetchEntriesMessage.java  |  19 +-
 .../cache/partitioned/FetchKeysMessage.java     |  20 +-
 .../cache/partitioned/IndexCreationMsg.java     |  31 +--
 .../cache/partitioned/PRTombstoneMessage.java   |  19 +-
 .../cache/partitioned/PutAllPRMessage.java      |  10 +-
 .../sanctionedDataSerializables.txt             | 229 ++++++++-----------
 20 files changed, 183 insertions(+), 593 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
index 01f8c62..6f5feb7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
@@ -310,12 +310,6 @@ public final class StartupMessage extends HighPriorityDistributionMessage implem
   public void toData(DataOutput out) throws IOException {
     super.toData(out);
 
-    boolean pre9_0_0_0 = InternalDataSerializer.
-        getVersionForDataStream(out).compareTo(Version.GFE_90) < 0;
-    if (pre9_0_0_0) {
-      DataSerializer.writeObject(new Properties(), out);
-    }
-
     DataSerializer.writeString(this.version, out);
     out.writeInt(this.replyProcessorId);
     out.writeBoolean(this.isMcastEnabled);
@@ -378,12 +372,6 @@ public final class StartupMessage extends HighPriorityDistributionMessage implem
     throws IOException, ClassNotFoundException {
     super.fromData(in);
 
-    boolean pre9_0_0_0 = InternalDataSerializer.
-        getVersionForDataStream(in).compareTo(Version.GFE_90) < 0;
-    if (pre9_0_0_0) {
-      DataSerializer.readObject(in);
-    }
-    
     this.version = DataSerializer.readString(in);
     this.replyProcessorId = in.readInt();
     this.isMcastEnabled = in.readBoolean();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupResponseMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupResponseMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupResponseMessage.java
index 762c111..3fbf7c7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupResponseMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupResponseMessage.java
@@ -223,12 +223,7 @@ public class StartupResponseMessage extends HighPriorityDistributionMessage impl
     
     super.toData(out);
     
-    Version ver = InternalDataSerializer.getVersionForDataStream(out);
-    
     out.writeInt(processorId);
-    if (ver.compareTo(Version.GFE_80) < 0) {
-      out.writeLong(System.currentTimeMillis());
-    }
     DataSerializer.writeString(this.rejectionMessage, out);
     out.writeBoolean(this.responderIsAdmin);
 
@@ -248,9 +243,6 @@ public class StartupResponseMessage extends HighPriorityDistributionMessage impl
     }
     
     DataSerializer.writeObject(interfaces, out);
-    if (ver.compareTo(Version.GFE_90) < 0) {
-      DataSerializer.writeObject(new Properties(), out);
-    }
     out.writeInt(distributedSystemId);
     DataSerializer.writeString(redundancyZone, out);
   }
@@ -261,12 +253,7 @@ public class StartupResponseMessage extends HighPriorityDistributionMessage impl
       
     super.fromData(in);
     
-    Version ver = InternalDataSerializer.getVersionForDataStream(in);
-    
     this.processorId = in.readInt();
-    if (ver.compareTo(Version.GFE_80) < 0) {
-      in.readLong();
-    }
     this.rejectionMessage = DataSerializer.readString(in);
     this.responderIsAdmin = in.readBoolean();
 
@@ -295,9 +282,6 @@ public class StartupResponseMessage extends HighPriorityDistributionMessage impl
     } // for
     
     interfaces = (Set)DataSerializer.readObject(in);
-    if (ver.compareTo(Version.GFE_90) < 0) {
-      DataSerializer.readObject(in);
-    }
     distributedSystemId = in.readInt();
     redundancyZone = DataSerializer.readString(in);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
index d9fc2e6..0758f02 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
@@ -187,9 +187,7 @@ public class RemoteBridgeServer
     DataSerializer.writeObject(this.loadProbe, out);
     DataSerializer.writePrimitiveLong(this.loadPollInterval, out);
     out.writeInt(this.socketBufferSize);
-    if (InternalDataSerializer.getVersionForDataStream(out).compareTo(Version.GFE_80) >= 0) {
-      out.writeBoolean(this.tcpNoDelay);
-    }
+    out.writeBoolean(this.tcpNoDelay);
     out.writeInt(this.getClientSubscriptionConfig().getCapacity());
     DataSerializer.writeString(this.getClientSubscriptionConfig()
         .getEvictionPolicy(), out);
@@ -219,9 +217,7 @@ public class RemoteBridgeServer
     setLoadProbe((ServerLoadProbe)DataSerializer.readObject(in));
     setLoadPollInterval(DataSerializer.readPrimitiveLong(in));
     this.socketBufferSize = in.readInt();
-    if (InternalDataSerializer.getVersionForDataStream(in).compareTo(Version.GFE_80) >= 0) {
-      this.tcpNoDelay = in.readBoolean();
-    }
+    this.tcpNoDelay = in.readBoolean();
     this.getClientSubscriptionConfig().setCapacity(in.readInt());
     this.getClientSubscriptionConfig().setEvictionPolicy(
         DataSerializer.readString(in));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteRegionAttributes.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteRegionAttributes.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteRegionAttributes.java
index 4363baf..4d4197e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteRegionAttributes.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteRegionAttributes.java
@@ -375,53 +375,7 @@ public class RemoteRegionAttributes implements RegionAttributes,
     return this.offHeap;
   }
   
-  public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException {
-    DataSerializer.writeString(this.cacheLoaderDesc, out);
-    DataSerializer.writeString(this.cacheWriterDesc, out);
-    DataSerializer.writeStringArray(this.cacheListenerDescs, out);
-    DataSerializer.writeString(this.capacityControllerDesc, out);
-    DataSerializer.writeObject(this.keyConstraint, out);
-    DataSerializer.writeObject(this.valueConstraint, out);
-    DataSerializer.writeObject(this.rTtl, out);
-    DataSerializer.writeObject(this.rIdleTimeout, out);
-    DataSerializer.writeObject(this.eTtl, out);
-    DataSerializer.writeString(this.customEttlDesc, out);
-    DataSerializer.writeObject(this.eIdleTimeout, out);
-    DataSerializer.writeString(this.customEIdleDesc, out);
-    DataSerializer.writeObject(this.dataPolicy, out);
-    DataSerializer.writeObject(this.scope, out);
-    out.writeBoolean(this.statsEnabled);
-    out.writeBoolean(this.ignoreJTA);
-    out.writeInt(this.concurrencyLevel);
-    out.writeFloat(this.loadFactor);
-    out.writeInt(this.initialCapacity);
-    out.writeBoolean(this.earlyAck);
-    out.writeBoolean(this.multicastEnabled);
-    out.writeBoolean(this.enableGateway);
-    DataSerializer.writeString(this.gatewayHubId, out);
-    out.writeBoolean(this.enableSubscriptionConflation);
-    out.writeBoolean(this.publisher);
-    out.writeBoolean(this.enableAsyncConflation);
-
-    DataSerializer.writeObject(this.diskWriteAttributes, out);
-    DataSerializer.writeObject(this.diskDirs, out);
-    DataSerializer.writeObject(this.diskSizes, out);
-    out.writeBoolean(this.indexMaintenanceSynchronous);
-    DataSerializer.writeObject(this.partitionAttributes, out);
-    DataSerializer.writeObject(this.membershipAttributes, out);
-    DataSerializer.writeObject(this.subscriptionAttributes, out);
-    DataSerializer.writeObject(this.evictionAttributes, out);
-    out.writeBoolean(this.cloningEnable);
-    DataSerializer.writeString(this.diskStoreName, out);
-    out.writeBoolean(this.isDiskSynchronous);
-    DataSerializer.writeStringArray(this.gatewaySendersDescs, out);
-    out.writeBoolean(this.isGatewaySenderEnabled);
-    
-    out.writeBoolean(this.concurrencyChecksEnabled);
-  }
-  
-  public void toDataPre_GFE_9_0_0_0(DataOutput out) throws IOException {
-
+  public void toData(DataOutput out) throws IOException {
     DataSerializer.writeString(this.cacheLoaderDesc, out);
     DataSerializer.writeString(this.cacheWriterDesc, out);
     DataSerializer.writeStringArray(this.cacheListenerDescs, out);
@@ -443,12 +397,6 @@ public class RemoteRegionAttributes implements RegionAttributes,
     out.writeInt(this.initialCapacity);
     out.writeBoolean(this.earlyAck);
     out.writeBoolean(this.multicastEnabled);
-    if (InternalDataSerializer.getVersionForDataStream(out).compareTo(
-        Version.CURRENT) < 0) {
-      out.writeBoolean(this.enableGateway);
-      DataSerializer.writeString(this.gatewayHubId, out);
-    }
-    
     out.writeBoolean(this.enableSubscriptionConflation);
     out.writeBoolean(this.publisher);
     out.writeBoolean(this.enableAsyncConflation);
@@ -470,62 +418,11 @@ public class RemoteRegionAttributes implements RegionAttributes,
     out.writeBoolean(this.concurrencyChecksEnabled);
   
     DataSerializer.writeString(this.compressorDesc, out);
-  }
-  
-  public void toData(DataOutput out) throws IOException {
-    toDataPre_GFE_9_0_0_0(out);
     out.writeBoolean(this.offHeap);
     DataSerializer.writeString(this.hdfsStoreName, out);
   }
   
-  public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException, ClassNotFoundException {
-    this.cacheLoaderDesc = DataSerializer.readString(in);
-    this.cacheWriterDesc = DataSerializer.readString(in);
-    this.cacheListenerDescs = DataSerializer.readStringArray(in);
-    this.capacityControllerDesc = DataSerializer.readString(in);
-    this.keyConstraint = (Class) DataSerializer.readObject(in);
-    this.valueConstraint = (Class) DataSerializer.readObject(in);
-    this.rTtl = (ExpirationAttributes) DataSerializer.readObject(in);
-    this.rIdleTimeout = (ExpirationAttributes) DataSerializer.readObject(in);
-    this.eTtl = (ExpirationAttributes) DataSerializer.readObject(in);
-    this.customEttlDesc = DataSerializer.readString(in);
-    this.eIdleTimeout = (ExpirationAttributes) DataSerializer.readObject(in);
-    this.customEIdleDesc = DataSerializer.readString(in);
-    this.dataPolicy = (DataPolicy) DataSerializer.readObject(in);
-    this.scope = (Scope) DataSerializer.readObject(in);
-    this.statsEnabled = in.readBoolean();
-    this.ignoreJTA = in.readBoolean();
-    this.concurrencyLevel = in.readInt();
-    this.loadFactor = in.readFloat();
-    this.initialCapacity = in.readInt();
-    this.earlyAck = in.readBoolean();
-    this.multicastEnabled = in.readBoolean();
-    this.enableGateway = in.readBoolean();
-    this.gatewayHubId = DataSerializer.readString(in);
-    this.enableSubscriptionConflation = in.readBoolean();
-    this.publisher = in.readBoolean();
-    this.enableAsyncConflation = in.readBoolean();
-
-    this.diskWriteAttributes = (DiskWriteAttributes) DataSerializer.readObject(in);
-    this.diskDirs = (File[]) DataSerializer.readObject(in);
-    this.diskSizes = (int[] )DataSerializer.readObject(in);
-    this.indexMaintenanceSynchronous = in.readBoolean();
-    this.partitionAttributes = (PartitionAttributes) DataSerializer
-    .readObject(in);
-    this.membershipAttributes = (MembershipAttributes) DataSerializer
-        .readObject(in);
-    this.subscriptionAttributes = (SubscriptionAttributes) DataSerializer
-        .readObject(in);
-    this.evictionAttributes = (EvictionAttributesImpl) DataSerializer.readObject(in);
-    this.cloningEnable = in.readBoolean();
-    this.diskStoreName = DataSerializer.readString(in);
-    this.isDiskSynchronous = in.readBoolean();
-    this.gatewaySendersDescs = DataSerializer.readStringArray(in);
-    this.isGatewaySenderEnabled = in.readBoolean();
-    this.concurrencyChecksEnabled = in.readBoolean();
-  }
-  
-  public void fromDataPre_GFE_9_0_0_0(DataInput in) throws IOException, ClassNotFoundException {
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
     this.cacheLoaderDesc = DataSerializer.readString(in);
     this.cacheWriterDesc = DataSerializer.readString(in);
     this.cacheListenerDescs = DataSerializer.readStringArray(in);
@@ -547,11 +444,6 @@ public class RemoteRegionAttributes implements RegionAttributes,
     this.initialCapacity = in.readInt();
     this.earlyAck = in.readBoolean();
     this.multicastEnabled = in.readBoolean();
-    if (InternalDataSerializer.getVersionForDataStream(in).compareTo(
-        Version.CURRENT) < 0) {
-      this.enableGateway = in.readBoolean();
-      this.gatewayHubId = DataSerializer.readString(in);
-    }
     this.enableSubscriptionConflation = in.readBoolean();
     this.publisher = in.readBoolean();
     this.enableAsyncConflation = in.readBoolean();
@@ -575,10 +467,6 @@ public class RemoteRegionAttributes implements RegionAttributes,
     this.concurrencyChecksEnabled = in.readBoolean();
   
     this.compressorDesc = DataSerializer.readString(in);
-  }
-  
-  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    fromDataPre_GFE_9_0_0_0(in);
     this.offHeap = in.readBoolean();
     this.hdfsStoreName = DataSerializer.readString(in);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/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 7406935..8408a4b 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
@@ -1332,12 +1332,8 @@ public abstract class DistributedCacheOperation {
     public void fromData(DataInput in) throws IOException,
             ClassNotFoundException {
       // super.fromData(in);
-      Version senderVersion = InternalDataSerializer.getVersionForDataStream(in);
       short bits = in.readShort();
-      short extBits = 0;
-      if (senderVersion.compareTo(Version.GFE_80) >= 0){ 
-        extBits = in.readShort();
-      }
+      short extBits = in.readShort();
       this.flags = bits;
       setFlags(bits, in);
       this.regionPath = DataSerializer.readString(in);
@@ -1390,10 +1386,7 @@ public abstract class DistributedCacheOperation {
       bits = computeCompressedShort(bits);
       extendedBits = computeCompressedExtBits(extendedBits);
       out.writeShort(bits);
-      Version receiverVersion = InternalDataSerializer.getVersionForDataStream(out);
-      if (receiverVersion.compareTo(Version.GFE_80) >= 0) {
-        out.writeShort(extendedBits);
-      }
+      out.writeShort(extendedBits);
       if (this.processorId > 0) {
         out.writeInt(this.processorId);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
index cd322d6..29d9b26 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
@@ -241,9 +241,7 @@ public class DistributedClearOperation extends DistributedCacheOperation
       this.clearOp = OperationType.values()[in.readByte()];
       this.eventID = (EventID)DataSerializer.readObject(in);
       this.rvv = (RegionVersionVector)DataSerializer.readObject(in);
-      if (InternalDataSerializer.getVersionForDataStream(in).compareTo(Version.GFE_80) >= 0) {
-        this.operationTag = (VersionTag<?>)DataSerializer.readObject(in);
-      }
+      this.operationTag = (VersionTag<?>)DataSerializer.readObject(in);
     }
 
     @Override
@@ -253,9 +251,7 @@ public class DistributedClearOperation extends DistributedCacheOperation
       out.writeByte(this.clearOp.ordinal());
       DataSerializer.writeObject(this.eventID, out);
       DataSerializer.writeObject(this.rvv, out);
-      if (InternalDataSerializer.getVersionForDataStream(out).compareTo(Version.GFE_80) >= 0) {
-        DataSerializer.writeObject(this.operationTag, out);
-      }
+      DataSerializer.writeObject(this.operationTag, out);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
index 9cd6885..84da441 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
@@ -404,7 +404,7 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
 
     /**
      * Used to serialize this instances data to <code>out</code>.
-     * Post 7.1, if changes are made to this method make sure that it is backwards
+     * If changes are made to this method make sure that it is backwards
      * compatible by creating toDataPreXX methods. Also make sure that the callers
      * to this method are backwards compatible by creating toDataPreXX methods for
      * them even if they are not changed. <br>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
index 9e7d49c..6292a7b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
@@ -363,7 +363,7 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
 
     /**
      * Used to serialize this instances data to <code>out</code>.
-     * Post 7.1, if changes are made to this method make sure that it is backwards
+     * If changes are made to this method make sure that it is backwards
      * compatible by creating toDataPreXX methods. Also make sure that the callers
      * to this method are backwards compatible by creating toDataPreXX methods for
      * them even if they are not changed. <br>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedTombstoneOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedTombstoneOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedTombstoneOperation.java
index aef13ee..c251b3a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedTombstoneOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedTombstoneOperation.java
@@ -117,7 +117,7 @@ public class DistributedTombstoneOperation extends DistributedCacheOperation {
     protected TOperation op;
     protected EventID eventID;
     
-    private static Version[] serializationVersions = new Version[]{ Version.GFE_80 };
+    private static Version[] serializationVersions = null; // new Version[]{ };
 
     /**
      * for deserialization
@@ -168,12 +168,6 @@ public class DistributedTombstoneOperation extends DistributedCacheOperation {
     @Override
     public void fromData(DataInput in) throws IOException,
             ClassNotFoundException {
-      fromDataPre_GFE_8_0_0_0(in);
-      this.eventID = (EventID)DataSerializer.readObject(in);
-    }
-    
-    public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException,
-            ClassNotFoundException {
       super.fromData(in);
       this.op = TOperation.values()[in.readByte()];
       //      this.regionVersion = in.readLong();
@@ -191,15 +185,11 @@ public class DistributedTombstoneOperation extends DistributedCacheOperation {
         }
         this.regionGCVersions.put(mbr, Long.valueOf(in.readLong()));
       }
+      this.eventID = (EventID)DataSerializer.readObject(in);
     }
     
     @Override
     public void toData(DataOutput out) throws IOException {
-      toDataPre_GFE_8_0_0_0(out);
-      DataSerializer.writeObject(this.eventID, out);
-    }
-    
-    public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException {
       super.toData(out);
       out.writeByte(this.op.ordinal());
       //      out.writeLong(this.regionVersion);
@@ -221,6 +211,7 @@ public class DistributedTombstoneOperation extends DistributedCacheOperation {
         }
         out.writeLong(entry.getValue());
       }
+      DataSerializer.writeObject(this.eventID, out);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
index 7ca8a29..34e3855 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
@@ -1596,8 +1596,7 @@ public class InitialImageOperation  {
     protected Set unfinishedKeys;
 
     /** The versions in which this message was modified */
-    private static final Version[] dsfidVersions = new Version[] {
-          Version.GFE_80 };
+    private static final Version[] dsfidVersions = null;
 
     @Override  
     public int getProcessorId() {
@@ -2083,11 +2082,6 @@ public class InitialImageOperation  {
 
     @Override  
     public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-      fromDataPre_GFE_8_0_0_0(in);
-      this.unfinishedKeys = (Set)DataSerializer.readObject(in);
-    }
-    
-    public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException, ClassNotFoundException {
       super.fromData(in);
       this.regionPath = DataSerializer.readString(in);
       this.processorId = in.readInt();
@@ -2097,15 +2091,11 @@ public class InitialImageOperation  {
       this.lostMemberVersionID = (VersionSource)DataSerializer.readObject(in);
       this.versionVector = (RegionVersionVector)DataSerializer.readObject(in);
       this.lostMemberID = (InternalDistributedMember)DataSerializer.readObject(in);
+      this.unfinishedKeys = (Set)DataSerializer.readObject(in);
     }
     
     @Override  
     public void toData(DataOutput out) throws IOException {
-      toDataPre_GFE_8_0_0_0(out);
-      DataSerializer.writeObject(this.unfinishedKeys, out);
-    }
-
-    public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException {
       super.toData(out);
       DataSerializer.writeString(this.regionPath, out);
       out.writeInt(this.processorId);
@@ -2115,6 +2105,7 @@ public class InitialImageOperation  {
       DataSerializer.writeObject(this.lostMemberVersionID, out);
       DataSerializer.writeObject(this.versionVector, out);
       DataSerializer.writeObject(this.lostMemberID, out);
+      DataSerializer.writeObject(this.unfinishedKeys, out);
     }
     
     @Override
@@ -2756,8 +2747,7 @@ public class InitialImageOperation  {
     private transient Version remoteVersion;
 
     /** The versions in which this message was modified */
-    private static final Version[] dsfidVersions = new Version[] {
-          Version.GFE_80 };
+    private static final Version[] dsfidVersions = null;
 
     @Override
     public boolean getInlineProcess() {
@@ -2838,29 +2828,7 @@ public class InitialImageOperation  {
 
     @Override  
     public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-      fromDataPre_GFE_8_0_0_0(in);
-      this.isDeltaGII = in.readBoolean();
-      this.hasHolderToSend = in.readBoolean();
-      if (this.hasHolderToSend) {
-        this.holderToSend = new RegionVersionHolder(in);
-      }
-      
-      int gcVersionsLength = in.readShort();
-      if(gcVersionsLength >= 0) {
-        gcVersions = new HashMap<VersionSource, Long>(gcVersionsLength);
-      }
-      for(int i =0; i < gcVersionsLength; i++) {
-        VersionSource key = InternalDataSerializer.readObject(in);
-        long value = InternalDataSerializer.readUnsignedVL(in);
-        gcVersions.put(key, value);
-      }
-    }
-
-    public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException, ClassNotFoundException {
       super.fromData(in);
-      
-      // 701 peers can get ArrayList from 700 peers so we always have to read
-      // an ArrayList. This hack will be simplified in later versions (> 7.0.1)
       ArrayList list = DataSerializer.readArrayList(in);
       Object listData = null;
       if (list != null /* fix bug 46874 */ && list.size() > 0) {
@@ -2878,30 +2846,26 @@ public class InitialImageOperation  {
       this.flowControlId = in.readInt();
       this.remoteVersion = InternalDataSerializer
                         .getVersionForDataStreamOrNull(in);
-    }
-    
-    @Override
-    public void toData(DataOutput out) throws IOException {
-      toDataPre_GFE_8_0_0_0(out);
-      out.writeBoolean(this.isDeltaGII);
-      out.writeBoolean(this.hasHolderToSend);
+      this.isDeltaGII = in.readBoolean();
+      this.hasHolderToSend = in.readBoolean();
       if (this.hasHolderToSend) {
-        InternalDataSerializer.invokeToData(this.holderToSend, out);
+        this.holderToSend = new RegionVersionHolder(in);
       }
-      out.writeShort(gcVersions == null ? -1 : gcVersions.size());
-      if(gcVersions != null) {
-        for(Map.Entry<VersionSource, Long> entry : gcVersions.entrySet()) {
-          InternalDataSerializer.writeObject(entry.getKey(), out);
-          InternalDataSerializer.writeUnsignedVL(entry.getValue(), out);
-        }
+      
+      int gcVersionsLength = in.readShort();
+      if(gcVersionsLength >= 0) {
+        gcVersions = new HashMap<VersionSource, Long>(gcVersionsLength);
+      }
+      for(int i =0; i < gcVersionsLength; i++) {
+        VersionSource key = InternalDataSerializer.readObject(in);
+        long value = InternalDataSerializer.readUnsignedVL(in);
+        gcVersions.put(key, value);
       }
     }
-    
-    public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException {
+
+    @Override
+    public void toData(DataOutput out) throws IOException {
       super.toData(out);
-      // We still need to send an ArrayList for backward compatibility.
-      // All 700 peers will always read an ArrayList. So we can not give
-      // them InitialImageVersionedEntryList when they are expecting ArrayList.
       if (this.entries instanceof InitialImageVersionedEntryList) {
         ArrayList list = new ArrayList(1);
         list.add(this.entries);
@@ -2914,8 +2878,20 @@ public class InitialImageOperation  {
       out.writeInt(this.numSeries);
       out.writeBoolean(this.lastInSeries);
       out.writeInt(this.flowControlId);
+      out.writeBoolean(this.isDeltaGII);
+      out.writeBoolean(this.hasHolderToSend);
+      if (this.hasHolderToSend) {
+        InternalDataSerializer.invokeToData(this.holderToSend, out);
+      }
+      out.writeShort(gcVersions == null ? -1 : gcVersions.size());
+      if(gcVersions != null) {
+        for(Map.Entry<VersionSource, Long> entry : gcVersions.entrySet()) {
+          InternalDataSerializer.writeObject(entry.getKey(), out);
+          InternalDataSerializer.writeUnsignedVL(entry.getValue(), out);
+        }
+      }
     }
-
+    
     @Override  
     public String toString() {
       StringBuffer buff = new StringBuffer();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
index 6abd70c..92a5a6c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
@@ -174,14 +174,7 @@ public class QueuedOperation
         cbArg);
   }
 
-  /**
-   * Post 7.1, if changes are made to this method make sure that it is backwards
-   * compatible by creating toDataPreXX methods. Also make sure that the callers
-   * to this method are backwards compatible by creating toDataPreXX methods for
-   * them even if they are not changed. <br>
-   * Callers for this method are: <br>
-   * SendQueueMessage.toData(DataOutput) <br>
-   */
+
   public void toData(DataOutput out) throws IOException
   {
     out.writeByte(this.op.ordinal);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
index b6af62a..27d5bc2 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
@@ -360,12 +360,7 @@ public abstract class RemoteOperationMessage extends DistributionMessage impleme
     this.flags = in.readShort();
     setFlags(this.flags, in);
     this.regionPath = DataSerializer.readString(in);
-
-    // extra field post 9.0
-    if (InternalDataSerializer.getVersionForDataStream(in).compareTo(
-        Version.GFE_90) >= 0) {
-      this.isTransactionDistributed = in.readBoolean();
-    }
+    this.isTransactionDistributed = in.readBoolean();
   }
 
   public InternalDistributedMember getTXOriginatorClient() {
@@ -395,12 +390,7 @@ public abstract class RemoteOperationMessage extends DistributionMessage impleme
       DataSerializer.writeObject(this.getTXMemberId(),out);
     }
     DataSerializer.writeString(this.regionPath,out);
-
-    // extra field post 9.0
-    if (InternalDataSerializer.getVersionForDataStream(out).compareTo(
-        Version.GFE_90) >= 0) {
-      out.writeBoolean(this.isTransactionDistributed);
-    }
+    out.writeBoolean(this.isTransactionDistributed);
   }
 
   protected short computeCompressedShort() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
index f97683d..c706e4f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
@@ -234,10 +234,7 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
       ClassNotFoundException {
     super.fromData(in);
     this.eventId = (EventID)DataSerializer.readObject(in);
-    Version sourceVersion = InternalDataSerializer.getVersionForDataStream(in);
-    if (sourceVersion.compareTo(Version.GFE_81) >= 0) {
-      this.callbackArg = DataSerializer.readObject(in);
-    }
+    this.callbackArg = DataSerializer.readObject(in);
     this.posDup = (flags & POS_DUP) != 0;
     if ((flags & HAS_BRIDGE_CONTEXT) != 0) {
       this.bridgeContext = DataSerializer.readObject(in);
@@ -269,10 +266,7 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
   public final void toData(DataOutput out) throws IOException {
     super.toData(out);
     DataSerializer.writeObject(this.eventId, out);
-    Version receiverVersion = InternalDataSerializer.getVersionForDataStream(out);
-    if (receiverVersion.compareTo(Version.GFE_81) >= 0) {
-      DataSerializer.writeObject(this.callbackArg, out);
-    }
+    DataSerializer.writeObject(this.callbackArg, out);
     if (this.bridgeContext != null) {
       DataSerializer.writeObject(this.bridgeContext, out);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/ResourceAdvisor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/ResourceAdvisor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/ResourceAdvisor.java
index 3b28628..6288b4c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/ResourceAdvisor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/ResourceAdvisor.java
@@ -383,98 +383,15 @@ public class ResourceAdvisor extends DistributionAdvisor {
     public void fromData(DataInput in) throws IOException, ClassNotFoundException {
       super.fromData(in);
       
-      if (InternalDataSerializer.getVersionForDataStream(in).compareTo(Version.GFE_90) >= 0) {
-        final long heapBytesUsed = in.readLong();
-        MemoryState heapState = MemoryState.fromData(in);
-        MemoryThresholds heapThresholds = MemoryThresholds.fromData(in);
-        setHeapData(heapBytesUsed, heapState, heapThresholds);
-
-        final long offHeapBytesUsed = in.readLong();
-        MemoryState offHeapState = MemoryState.fromData(in);
-        MemoryThresholds offHeapThresholds = MemoryThresholds.fromData(in);
-        setOffHeapData(offHeapBytesUsed, offHeapState, offHeapThresholds);
-      } else {
-        // pre 9.0
-        in.readInt(); // currentHeapUsagePercent
-        long currentHeapBytesUsed = in.readLong();
-        //enum MemoryEventType (the byte GEMFIRE_ENUM followed by two Strings. The enums: UNKNOWN, EVICTION_UP, EVICT_MORE, EVICTION_DOWN, EVICTION_DISABLED, CRITICAL_UP, CRITICAL_DOWN, CRITICAL_DISABLED)
-        byte b = in.readByte();
-        String enumName;
-        if (b == DSCODE.GEMFIRE_ENUM) {
-          String enumClass = DataSerializer.readString(in);
-          assert enumClass.equals("com.gemstone.gemfire.internal.cache.control.MemoryEventType");
-          enumName = DataSerializer.readString(in);
-        } else if (b == DSCODE.SERIALIZABLE) {
-          final byte TC_ENUM = 0x7e;
-          final byte TC_CLASSDESC = 0x72;
-          final byte TC_ENDBLOCK = 0x78;
-          final byte TC_NULL = 0x70;
-          final byte TC_STRING = 0x74;
-          in.readShort(); // STREAM_MAGIC
-          in.readShort(); // STREAM_VERSION
-          b = in.readByte();
-          assert b == TC_ENUM : "expected " + b + " to be TC_ENUM " + TC_ENUM;
-          b = in.readByte();
-          assert b == TC_CLASSDESC : "expected " + b + " to be TC_CLASSDESC " + TC_CLASSDESC;
-          String cn = in.readUTF();
-          //System.out.println("DEBUG enum className=" + cn);
-          assert cn.equals("com.gemstone.gemfire.internal.cache.control.MemoryEventType");
-          in.readLong();
-          in.readByte();
-          int fields = in.readShort();
-          while (fields != 0) {
-            in.readByte();
-            in.readUTF();
-            fields--;
-          }
-          b = in.readByte();
-          assert b == TC_ENDBLOCK : "expected " + b + " to be TC_ENDBLOCK " + TC_ENDBLOCK;
-          // parent classDesc
-          b = in.readByte();
-          assert b == TC_CLASSDESC : "expected " + b + " to be TC_CLASSDESC " + TC_CLASSDESC;
-          cn = in.readUTF();
-          //System.out.println("DEBUG parent className=" + cn);
-          assert cn.equals("java.lang.Enum");
-          in.readLong();
-          in.readByte();
-          fields = in.readShort();
-          while (fields != 0) {
-            in.readByte();
-            in.readUTF();
-            fields--;
-          }
-          b = in.readByte();
-          assert b == TC_ENDBLOCK : "expected " + b + " to be TC_ENDBLOCK " + TC_ENDBLOCK;
-          b = in.readByte();
-          assert b == TC_NULL : "expected " + b + " to be TC_NULL " + TC_NULL;
-          b = in.readByte();
-          assert b == TC_STRING : "expected " + b + " to be TC_STRING " + TC_STRING;
-
-          enumName = in.readUTF();
-          //System.out.println("DEBUG enumName=" + enumName);
-        } else {
-          throw new IllegalStateException("Unexpected byte " + b);
-        }
-        in.readDouble(); // tenuredGenerationMaxBytes
-        in.readBoolean(); // hasTenuredGenerationMaxBytes
-        float criticalThreshold = in.readFloat();
-        in.readBoolean(); // hasCriticalThreshold
-        float evictionThreshold = in.readFloat();
-        in.readBoolean(); // hasEvictionThreshold
-        MemoryState heapState;
-        if (enumName.equals("CRITICAL_UP")) {
-          heapState = MemoryState.CRITICAL;
-        } else if (enumName.equals("CRITICAL_DISABLED")) {
-          heapState = MemoryState.CRITICAL_DISABLED;
-        } else if (enumName.equals("CRITICAL_DOWN")) {
-          heapState = MemoryState.NORMAL;
-        } else {
-          // We really don't care about the other old states so we just call them normal
-          heapState = MemoryState.NORMAL;
-        }
-        setHeapData(currentHeapBytesUsed, heapState, new MemoryThresholds(0, criticalThreshold, evictionThreshold));
-        setOffHeapData(0, MemoryState.DISABLED, new MemoryThresholds(0));
-      }
+      final long heapBytesUsed = in.readLong();
+      MemoryState heapState = MemoryState.fromData(in);
+      MemoryThresholds heapThresholds = MemoryThresholds.fromData(in);
+      setHeapData(heapBytesUsed, heapState, heapThresholds);
+
+      final long offHeapBytesUsed = in.readLong();
+      MemoryState offHeapState = MemoryState.fromData(in);
+      MemoryThresholds offHeapThresholds = MemoryThresholds.fromData(in);
+      setOffHeapData(offHeapBytesUsed, offHeapState, offHeapThresholds);
     }
 
     @Override
@@ -492,35 +409,13 @@ public class ResourceAdvisor extends DistributionAdvisor {
       }
       super.toData(out);
       
-      if (InternalDataSerializer.getVersionForDataStream(out).compareTo(Version.GFE_90) >= 0) {
-        out.writeLong(heapBytesUsed);
-        heapState.toData(out);
-        heapThresholds.toData(out);
-
-        out.writeLong(offHeapBytesUsed);
-        offHeapState.toData(out);
-        offHeapThresholds.toData(out);
-      } else {
-        out.writeInt(0); // currentHeapUsagePercent
-        out.writeLong(heapBytesUsed); // currentHeapBytesUsed
-        String memoryEventTypeName;
-        switch (heapState) {
-        case EVICTION_CRITICAL:
-        case CRITICAL: memoryEventTypeName = "CRITICAL_UP"; break;
-        case EVICTION_CRITICAL_DISABLED:
-        case CRITICAL_DISABLED: memoryEventTypeName = "CRITICAL_DISABLED"; break;
-        default: memoryEventTypeName = "CRITICAL_DOWN";
-        }
-        out.writeByte(DSCODE.GEMFIRE_ENUM);
-        DataSerializer.writeString("com.gemstone.gemfire.internal.cache.control.MemoryEventType", out);
-        DataSerializer.writeString(memoryEventTypeName, out);
-        out.writeDouble(0.0); // tenuredGenerationMaxBytes
-        out.writeBoolean(false); // hasTenuredGenerationMaxBytes
-        out.writeFloat(heapThresholds.getCriticalThreshold());
-        out.writeBoolean(heapThresholds.isCriticalThresholdEnabled());
-        out.writeFloat(heapThresholds.getEvictionThreshold());
-        out.writeBoolean(heapThresholds.isEvictionThresholdEnabled());
-      }
+      out.writeLong(heapBytesUsed);
+      heapState.toData(out);
+      heapThresholds.toData(out);
+
+      out.writeLong(offHeapBytesUsed);
+      offHeapState.toData(out);
+      offHeapThresholds.toData(out);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
index ef22609..0342164 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
@@ -182,8 +182,7 @@ public final class FetchEntriesMessage extends PartitionMessage
     private boolean hasRVV;
     
     /** The versions in which this message was modified */
-    private static final Version[] dsfidVersions = new Version[] {
-          Version.GFE_80 };
+    private static final Version[] dsfidVersions = null;
 
     @Override
     public Version[] getSerializationVersions() {
@@ -392,11 +391,6 @@ public final class FetchEntriesMessage extends PartitionMessage
    
     @Override
     public void toData(DataOutput out) throws IOException {
-      toDataPre_GFE_8_0_0_0(out);
-      out.writeBoolean(this.hasRVV);
-    }
-    
-    public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException {
       super.toData(out);
       out.writeInt(this.bucketId);
       out.writeInt(this.seriesNum);
@@ -404,8 +398,9 @@ public final class FetchEntriesMessage extends PartitionMessage
       out.writeInt(this.numSeries);
       out.writeBoolean(this.lastInSeries);
       DataSerializer.writeObjectAsByteArray(this.chunkStream, out);
+      out.writeBoolean(this.hasRVV);
     }
-
+    
     @Override
     public int getDSFID() {
       return PR_FETCH_ENTRIES_REPLY_MESSAGE;
@@ -414,11 +409,6 @@ public final class FetchEntriesMessage extends PartitionMessage
     @Override
     public void fromData(DataInput in)
       throws IOException, ClassNotFoundException {
-      fromDataPre_GFE_8_0_0_0(in);
-      hasRVV = in.readBoolean();
-    }
-    
-    public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException, ClassNotFoundException {
       super.fromData(in);
       this.bucketId = in.readInt();
       this.seriesNum = in.readInt();
@@ -426,8 +416,9 @@ public final class FetchEntriesMessage extends PartitionMessage
       this.numSeries = in.readInt();
       this.lastInSeries = in.readBoolean();
       this.chunk = DataSerializer.readByteArray(in);
+      hasRVV = in.readBoolean();
     }
-  
+    
     @Override
     public String toString() {
       StringBuffer sb = new StringBuffer();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
index edcac66..41e08b3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
@@ -198,7 +198,7 @@ public final class FetchKeysMessage extends PartitionMessage
   /**
    * Versions in which on-wire form has changed, requiring new toData/fromData methods
    */
-  public Version[] serializationVersions = new Version[]{Version.GFE_80};
+  public Version[] serializationVersions = null;
   
   public Version[] getSerializationVersions() {
     return serializationVersions;
@@ -206,30 +206,22 @@ public final class FetchKeysMessage extends PartitionMessage
 
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException  {
-    fromDataPre_GFE_8_0_0_0(in);
-    this.allowTombstones = in.readBoolean();
-  }
-
-  @Override
-  public void toData(DataOutput out) throws IOException  {
-    toDataPre_GFE_8_0_0_0(out);
-    out.writeBoolean(this.allowTombstones);
-  }
-
-  public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException, ClassNotFoundException  {
     super.fromData(in);
     this.bucketId = Integer.valueOf(in.readInt());
     this.interestType = in.readInt();
     this.interestArg = DataSerializer.readObject(in);
+    this.allowTombstones = in.readBoolean();
   }
 
-  public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException  {
+  @Override
+  public void toData(DataOutput out) throws IOException  {
     super.toData(out);
     out.writeInt(this.bucketId.intValue());
     out.writeInt(interestType);
     DataSerializer.writeObject(interestArg, out);
+    out.writeBoolean(this.allowTombstones);
   }
-  
+
   public static final class FetchKeysReplyMessage extends ReplyMessage {
     /** The number of the series */
     int seriesNum;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/IndexCreationMsg.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/IndexCreationMsg.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/IndexCreationMsg.java
index 50e80f6..225e864 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/IndexCreationMsg.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/IndexCreationMsg.java
@@ -433,36 +433,7 @@ public class IndexCreationMsg extends PartitionMessage {
   
   @Override
   public Version[] getSerializationVersions() {
-    return new Version[]{Version.GFE_81};
-  }
-  
-  public final void fromDataPre_GFE_8_1_0_0(DataInput in) throws IOException,
-      ClassNotFoundException {
-    super.fromData(in);
-    String name = in.readUTF();
-    String fromClause = in.readUTF();
-    String indexedExpression = in.readUTF();
-    byte indexType = in.readByte();
-    IndexType itype = null;
-    if (0 == indexType) {
-      itype = IndexType.PRIMARY_KEY;
-    } else if (1 == indexType) {
-      itype = IndexType.FUNCTIONAL;
-    } else if (2 == indexType) {
-      itype = IndexType.HASH;
-    }
-    boolean importsNeeded = in.readBoolean();
-    String importStr = null;
-    if (importsNeeded) {
-      importStr = in.readUTF();
-    }
-    IndexCreationData icd = new IndexCreationData(name);
-    icd.setIndexData(itype, fromClause, indexedExpression, importStr);
-    this.indexDefinitions = new HashSet<IndexCreationData>();
-    this.indexDefinitions.add(icd);
-  }
-  
-  public final void toDataPre_GFE_8_1_0_0(DataOutput out) throws IOException {    
+    return null;
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRTombstoneMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRTombstoneMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRTombstoneMessage.java
index 808bfe9..57b6d43 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRTombstoneMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRTombstoneMessage.java
@@ -58,7 +58,7 @@ public final class PRTombstoneMessage extends PartitionMessageWithDirectReply
 
   private static final Logger logger = LogService.getLogger();
   
-  private static Version[] serializationVersions = new Version[]{ Version.GFE_80 };
+  private static Version[] serializationVersions = null;
 
   private Set<Object> keys;
   private EventID eventID;
@@ -136,36 +136,25 @@ public final class PRTombstoneMessage extends PartitionMessageWithDirectReply
   @Override
   public void fromData(DataInput in) throws IOException,
           ClassNotFoundException {
-    fromDataPre_GFE_8_0_0_0(in);
-    this.eventID = (EventID)DataSerializer.readObject(in);
-  }
-  
-  public void fromDataPre_GFE_8_0_0_0(DataInput in) throws IOException,
-          ClassNotFoundException {
     super.fromData(in);
     int numKeys = in.readInt();
     this.keys = new HashSet<Object>(numKeys);
     for (int i=0; i<numKeys; i++) {
       this.keys.add(DataSerializer.readObject(in));
     }
+    this.eventID = (EventID)DataSerializer.readObject(in);
   }
   
-
   @Override
   public void toData(DataOutput out) throws IOException {
-    toDataPre_GFE_8_0_0_0(out);
-    DataSerializer.writeObject(this.eventID, out);
-  }
-  
-  public void toDataPre_GFE_8_0_0_0(DataOutput out) throws IOException {
     super.toData(out);
     out.writeInt(this.keys.size());
     for (Object key: keys) {
       DataSerializer.writeObject(key, out);
     }
+    DataSerializer.writeObject(this.eventID, out);
   }
-
-
+  
   private static class Response extends PartitionResponse
   {
 //    Set<InternalDistributedMember> forceReattemptSenders = new HashSet<InternalDistributedMember>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7b2714c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
index 046edf4..b6c7b1b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
@@ -233,10 +233,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     if ((flags & HAS_BRIDGE_CONTEXT) != 0) {
       this.bridgeContext = DataSerializer.readObject(in);
     }
-    Version sourceVersion = InternalDataSerializer.getVersionForDataStream(in);
-    if (sourceVersion.compareTo(Version.GFE_81) >= 0) {
-      this.callbackArg = DataSerializer.readObject(in);
-    }
+    this.callbackArg = DataSerializer.readObject(in);
     this.putAllPRDataSize = (int)InternalDataSerializer.readUnsignedVL(in);
     this.putAllPRData = new PutAllEntryData[putAllPRDataSize];
     if (this.putAllPRDataSize > 0) {
@@ -271,10 +268,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     if (this.bridgeContext != null) {
       DataSerializer.writeObject(this.bridgeContext, out);
     }
-    Version receiverVersion = InternalDataSerializer.getVersionForDataStream(out);
-    if (receiverVersion.compareTo(Version.GFE_81) >= 0) {
-      DataSerializer.writeObject(this.callbackArg, out);
-    }
+    DataSerializer.writeObject(this.callbackArg, out);
     InternalDataSerializer.writeUnsignedVL(this.putAllPRDataSize, out);
     if (this.putAllPRDataSize > 0) {
       EntryVersionsList versionTags = new EntryVersionsList(putAllPRDataSize);