GEODE-8837: Establishes GFE_81 as the oldest supported client. (#5905)

* Deprecate old Versions.
* Remove old gossip from TcpClient.
* Removes old commands from command map.
* Removes GFE_56 completely as example.
* Removes obsolete ContainsKey as example.
* Removes obsolete Commands.
* Extracts CommandRegistry interface for testing.
* Adds exception if commands fail to register.
* Tests that CQ commands can register.
* Removes obsolete CQ command.
* Refactor CommandInitializer for more immutability and thread safety.
* Removed unreleased GFE_82 version.
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/BackwardCompatibilitySerializationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/BackwardCompatibilitySerializationDUnitTest.java
index 05f4e4f..ef71762 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/BackwardCompatibilitySerializationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/BackwardCompatibilitySerializationDUnitTest.java
@@ -47,10 +47,7 @@
 import org.apache.geode.test.junit.categories.SerializationTest;
 
 /**
- * Test the DSFID serialization framework added for rolling upgrades in 7.1
- *
- *
- *
+ * Test the DSFID serialization framework added for rolling upgrades
  */
 @Category({SerializationTest.class})
 public class BackwardCompatibilitySerializationDUnitTest extends JUnit4CacheTestCase {
@@ -59,11 +56,11 @@
   private transient ByteArrayInputStream bais;
 
   public static boolean toDataCalled = false;
-  public static boolean toDataPre66Called = false;
-  public static boolean toDataPre70called = false;
+  public static boolean toDataPre11Called = false;
+  public static boolean toDataPre15called = false;
   public static boolean fromDataCalled = false;
-  public static boolean fromDataPre66Called = false;
-  public static boolean fromDataPre70Called = false;
+  public static boolean fromDataPre11Called = false;
+  public static boolean fromDataPre15Called = false;
 
   public TestMessage msg = new TestMessage();
 
@@ -86,69 +83,61 @@
     InternalDataSerializer.getDSFIDSerializer().registerDSFID(
         DataSerializableFixedID.PUTALL_VERSIONS_LIST,
         EntryVersionsList.class);
-    this.baos = null;
-    this.bais = null;
+    baos = null;
+    bais = null;
   }
 
   /**
-   * Test if correct toData/toDataPreXXX is called when changes are made to the TestMessage in 66
-   * and 70 and version of peer is 56
-   *
+   * Test if correct toData/toDataPreXXX is called when changes are made to the TestMessage
    */
   @Test
   public void testToDataFromHigherVersionToLower() throws Exception {
     DataOutputStream dos =
-        new VersionedDataOutputStream(new DataOutputStream(baos), KnownVersion.GFE_56);
+        new VersionedDataOutputStream(new DataOutputStream(baos), KnownVersion.OLDEST);
     InternalDataSerializer.writeDSFID(msg, dos);
-    assertTrue(toDataPre66Called);
+    assertTrue(toDataPre11Called);
     assertFalse(toDataCalled);
   }
 
   /**
-   * Test if correct toData/toDataXXX is called when changes are made to the TestMessage in 66 and
-   * 70 and version of peer is 70
-   *
+   * Test if correct toData/toDataXXX is called when changes are made to the TestMessage
    */
   @Test
   public void testToDataFromLowerVersionToHigher() throws Exception {
     DataOutputStream dos =
-        new VersionedDataOutputStream(new DataOutputStream(baos), KnownVersion.GFE_701);
+        new VersionedDataOutputStream(new DataOutputStream(baos), KnownVersion.GEODE_1_5_0);
     InternalDataSerializer.writeDSFID(msg, dos);
     assertTrue(toDataCalled);
   }
 
   /**
-   * Test if correct fromData/fromDataXXX is called when changes are made to the TestMessage in 66
-   * and 70 and version of peer is 70
-   *
+   * Test if correct fromData/fromDataXXX is called when changes are made to the TestMessage
    */
   @Test
   public void testFromDataFromHigherVersionToLower() throws Exception {
     InternalDataSerializer.writeDSFID(msg, new DataOutputStream(baos));
-    this.bais = new ByteArrayInputStream(baos.toByteArray());
+    bais = new ByteArrayInputStream(baos.toByteArray());
 
     DataInputStream dis =
-        new VersionedDataInputStream(new DataInputStream(bais), KnownVersion.GFE_701);
+        new VersionedDataInputStream(new DataInputStream(bais), KnownVersion.GEODE_1_5_0);
     Object o = InternalDataSerializer.basicReadObject(dis);
     assertTrue(o instanceof TestMessage);
     assertTrue(fromDataCalled);
   }
 
   /**
-   * Test if correct fromData/fromDataXXX is called when changes are made to the TestMessage in 66
-   * and 70 and version of peer is 56
-   *
+   * Test if correct fromData/fromDataXXX is called when changes are made to the TestMessage
    */
   @Test
   public void testFromDataFromLowerVersionToHigher() throws Exception {
     InternalDataSerializer.writeDSFID(msg, new DataOutputStream(baos));
-    this.bais = new ByteArrayInputStream(baos.toByteArray());
+    bais = new ByteArrayInputStream(baos.toByteArray());
 
     DataInputStream dis =
-        new VersionedDataInputStream(new DataInputStream(bais), KnownVersion.GFE_56);
+        new VersionedDataInputStream(new DataInputStream(bais), KnownVersion.OLDEST);
     Object o = InternalDataSerializer.basicReadObject(dis);
     assertTrue(o instanceof TestMessage);
-    assertTrue(fromDataPre66Called);
+    assertTrue(fromDataPre11Called);
   }
 
   /**
@@ -160,7 +149,7 @@
   public void testAllMessages() throws Exception {
     // list of msgs not created using reflection
     // taken from DSFIDFactory.create()
-    ArrayList<Integer> constdsfids = new ArrayList<Integer>();
+    ArrayList<Integer> constdsfids = new ArrayList<>();
     constdsfids.add(new Byte(DataSerializableFixedID.REGION).intValue());
     constdsfids.add(new Byte(DataSerializableFixedID.END_OF_STREAM_TOKEN).intValue());
     constdsfids.add(new Byte(DataSerializableFixedID.DLOCK_REMOTE_TOKEN).intValue());
@@ -204,14 +193,14 @@
       versions = ((SerializationVersions) ds).getSerializationVersions();
     }
     if (versions != null && versions.length > 0) {
-      for (int i = 0; i < versions.length; i++) {
+      for (final KnownVersion version : versions) {
         if (ds instanceof DataSerializableFixedID) {
           try {
-            ds.getClass().getMethod("toDataPre_" + versions[i].getMethodSuffix(),
-                new Class[] {DataOutput.class, SerializationContext.class});
+            ds.getClass().getMethod("toDataPre_" + version.getMethodSuffix(),
+                DataOutput.class, SerializationContext.class);
 
-            ds.getClass().getMethod("fromDataPre_" + versions[i].getMethodSuffix(),
-                new Class[] {DataInput.class, DeserializationContext.class});
+            ds.getClass().getMethod("fromDataPre_" + version.getMethodSuffix(),
+                DataInput.class, DeserializationContext.class);
           } catch (NoSuchMethodException e) {
             fail(
                 "toDataPreXXX or fromDataPreXXX for previous versions not found " + e.getMessage());
@@ -219,11 +208,11 @@
         }
         if (ds instanceof DataSerializable) {
           try {
-            ds.getClass().getMethod("toDataPre_" + versions[i].getMethodSuffix(),
-                new Class[] {DataOutput.class});
+            ds.getClass().getMethod("toDataPre_" + version.getMethodSuffix(),
+                DataOutput.class);
 
-            ds.getClass().getMethod("fromDataPre_" + versions[i].getMethodSuffix(),
-                new Class[] {DataInput.class});
+            ds.getClass().getMethod("fromDataPre_" + version.getMethodSuffix(),
+                DataInput.class);
           } catch (NoSuchMethodException e) {
             fail(
                 "toDataPreXXX or fromDataPreXXX for previous versions not found " + e.getMessage());
@@ -248,17 +237,17 @@
 
   private void resetFlags() {
     toDataCalled = false;
-    toDataPre66Called = false;
-    toDataPre70called = false;
+    toDataPre11Called = false;
+    toDataPre15called = false;
     fromDataCalled = false;
-    fromDataPre66Called = false;
-    fromDataPre70Called = false;
+    fromDataPre11Called = false;
+    fromDataPre15Called = false;
   }
 
   public static class TestMessage implements DataSerializableFixedID {
     /** The versions in which this message was modified */
     private static final KnownVersion[] dsfidVersions =
-        new KnownVersion[] {KnownVersion.GFE_66, KnownVersion.GFE_70};
+        new KnownVersion[] {KnownVersion.GEODE_1_1_0, KnownVersion.GEODE_1_5_0};
 
     public TestMessage() {}
 
@@ -273,14 +262,16 @@
       toDataCalled = true;
     }
 
-    public void toDataPre_GFE_6_6_0_0(DataOutput out, SerializationContext context)
-        throws IOException {
-      toDataPre66Called = true;
+    @SuppressWarnings("unused")
+    public void toDataPre_GEODE_1_1_0_0(@SuppressWarnings("unused") DataOutput out,
+        @SuppressWarnings("unused") SerializationContext context) {
+      toDataPre11Called = true;
     }
 
-    public void toDataPre_GFE_7_0_0_0(DataOutput out, SerializationContext context)
-        throws IOException {
-      toDataPre70called = true;
+    @SuppressWarnings("unused")
+    public void toDataPre_GEODE_1_5_0_0(@SuppressWarnings("unused") DataOutput out,
+        @SuppressWarnings("unused") SerializationContext context) {
+      toDataPre15called = true;
     }
 
     @Override
@@ -289,14 +280,16 @@
       fromDataCalled = true;
     }
 
-    public void fromDataPre_GFE_6_6_0_0(DataInput out, DeserializationContext context)
-        throws IOException {
-      fromDataPre66Called = true;
+    @SuppressWarnings("unused")
+    public void fromDataPre_GEODE_1_1_0_0(@SuppressWarnings("unused") DataInput out,
+        @SuppressWarnings("unused") DeserializationContext context) {
+      fromDataPre11Called = true;
     }
 
-    public void fromDataPre_GFE_7_0_0_0(DataInput out, DeserializationContext context)
-        throws IOException {
-      fromDataPre70Called = true;
+    @SuppressWarnings("unused")
+    public void fromDataPre_GEODE_1_5_0_0(@SuppressWarnings("unused") DataInput out,
+        @SuppressWarnings("unused") DeserializationContext context) {
+      fromDataPre15Called = true;
     }
 
     @Override
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java
index aae71c2..ab198ec 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java
@@ -57,7 +57,6 @@
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.api.MemberIdentifier;
 import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
@@ -353,12 +352,11 @@
     clientCache = new ClientCacheFactory().create();
     InternalDistributedMember memberID =
         (InternalDistributedMember) clientCache.getDistributedSystem().getDistributedMember();
-    MemberIdentifier gmsID = memberID;
-    memberID.setVersionForTest(KnownVersion.GFE_82);
-    assertThat(memberID.getVersion()).isEqualTo(KnownVersion.GFE_82);
+    memberID.setVersionForTest(KnownVersion.GFE_81);
+    assertThat(memberID.getVersion()).isEqualTo(KnownVersion.GFE_81);
 
     ClientProxyMembershipID clientID = ClientProxyMembershipID.getClientId(memberID);
-    HeapDataOutputStream out = new HeapDataOutputStream(KnownVersion.GFE_82);
+    HeapDataOutputStream out = new HeapDataOutputStream(KnownVersion.GFE_81);
     DataSerializer.writeObject(clientID, out);
 
     DataInputStream in =
@@ -367,13 +365,13 @@
     ClientProxyMembershipID newID = DataSerializer.readObject(in);
     InternalDistributedMember newMemberID =
         (InternalDistributedMember) newID.getDistributedMember();
-    assertThat(newMemberID.getVersion()).isEqualTo(KnownVersion.GFE_82);
-    assertThat(newID.getClientVersion()).isEqualTo(KnownVersion.GFE_82);
+    assertThat(newMemberID.getVersion()).isEqualTo(KnownVersion.GFE_81);
+    assertThat(newID.getClientVersion()).isEqualTo(KnownVersion.GFE_81);
 
     assertThat(newMemberID.getUuidLeastSignificantBits()).isEqualTo(0);
     assertThat(newMemberID.getUuidMostSignificantBits()).isEqualTo(0);
 
-    gmsID.setUUID(new UUID(1234L, 5678L));
+    memberID.setUUID(new UUID(1234L, 5678L));
     memberID.setVersionForTest(KnownVersion.CURRENT);
     clientID = ClientProxyMembershipID.getClientId(memberID);
     out = new HeapDataOutputStream(KnownVersion.CURRENT);
@@ -387,9 +385,9 @@
     assertThat(newID.getClientVersion()).isEqualTo(KnownVersion.CURRENT);
 
     assertThat(newMemberID.getUuidLeastSignificantBits())
-        .isEqualTo(gmsID.getUuidLeastSignificantBits());
+        .isEqualTo(memberID.getUuidLeastSignificantBits());
     assertThat(newMemberID.getUuidMostSignificantBits())
-        .isEqualTo(gmsID.getUuidMostSignificantBits());
+        .isEqualTo(memberID.getUuidMostSignificantBits());
   }
 
   @Test
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java
index 708d4b2..50c1515 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java
@@ -25,8 +25,6 @@
 import java.io.InputStream;
 import java.net.Socket;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -95,13 +93,13 @@
       InternalDistributedSystem distributedSystem, SecurityService securityService,
       boolean multiuserSecureMode) {
     this.multiuserSecureMode = multiuserSecureMode;
-    this.id = proxyId;
-    this.system = distributedSystem;
+    id = proxyId;
+    system = distributedSystem;
     this.securityService = securityService;
-    this.replyCode = REPLY_OK;
+    replyCode = REPLY_OK;
     setOverrides();
-    this.credentials = null;
-    this.encryptor = new EncryptorImpl(distributedSystem.getSecurityLogWriter());
+    credentials = null;
+    encryptor = new EncryptorImpl(distributedSystem.getSecurityLogWriter());
   }
 
   /**
@@ -109,8 +107,8 @@
    */
   public ClientSideHandshakeImpl(ClientSideHandshakeImpl handshake) {
     super(handshake);
-    this.multiuserSecureMode = handshake.multiuserSecureMode;
-    this.replyCode = handshake.getReplyCode();
+    multiuserSecureMode = handshake.multiuserSecureMode;
+    replyCode = handshake.getReplyCode();
   }
 
   public static void setVersionForTesting(short ver) {
@@ -125,7 +123,7 @@
   }
 
   private void setOverrides() {
-    this.clientConflation = determineClientConflation();
+    clientConflation = determineClientConflation();
 
     // As of May 2009 ( GFE 6.0 ):
     // Note that this.clientVersion is used by server side for accepting
@@ -133,7 +131,7 @@
     // Client side handshake code uses this.currentClientVersion which can be
     // set via tests.
     if (currentClientVersion.isNotOlderThan(KnownVersion.GFE_603)) {
-      this.overrides = new byte[] {this.clientConflation};
+      overrides = new byte[] {clientConflation};
     }
   }
 
@@ -141,7 +139,7 @@
   private byte determineClientConflation() {
     byte result = CONFLATION_DEFAULT;
 
-    String clientConflationValue = this.system.getProperties().getProperty(CONFLATE_EVENTS);
+    String clientConflationValue = system.getProperties().getProperty(CONFLATE_EVENTS);
     if (DistributionConfig.CLIENT_CONFLATION_PROP_VALUE_ON
         .equalsIgnoreCase(clientConflationValue)) {
       result = CONFLATION_ON;
@@ -171,7 +169,6 @@
       CommunicationMode communicationMode) throws IOException, AuthenticationRequiredException,
       AuthenticationFailedException, ServerRefusedConnectionException {
     try {
-      ServerQueueStatus serverQStatus = null;
       Socket sock = conn.getSocket();
       DataOutputStream dos = new DataOutputStream(sock.getOutputStream());
       final InputStream in = sock.getInputStream();
@@ -179,22 +176,22 @@
       InternalDistributedMember member = getIDForSocket(sock);
       // if running in a loner system, use the new port number in the ID to
       // help differentiate from other clients
-      DistributionManager dm = ((InternalDistributedSystem) this.system).getDistributionManager();
+      DistributionManager dm = ((InternalDistributedSystem) system).getDistributionManager();
       InternalDistributedMember idm = dm.getDistributionManagerId();
       synchronized (idm) {
         if (idm.getMembershipPort() == 0 && dm instanceof LonerDistributionManager) {
           int port = sock.getLocalPort();
           ((LonerDistributionManager) dm).updateLonerPort(port);
-          this.id.updateID(dm.getDistributionManagerId());
+          id.updateID(dm.getDistributionManagerId());
         }
       }
       if (communicationMode.isWAN()) {
-        this.credentials = getCredentials(member);
+        credentials = getCredentials(member);
       }
       byte intermediateAcceptanceCode = write(dos, dis, communicationMode, REPLY_OK,
-          this.clientReadTimeout, null, this.credentials, member, false);
+          clientReadTimeout, null, credentials, member, false);
 
-      String authInit = this.system.getProperties().getProperty(SECURITY_CLIENT_AUTH_INIT);
+      String authInit = system.getProperties().getProperty(SECURITY_CLIENT_AUTH_INIT);
       if (!communicationMode.isWAN()
           && intermediateAcceptanceCode != REPLY_AUTH_NOT_REQUIRED
           && (StringUtils.isNotBlank(authInit) || multiuserSecureMode)) {
@@ -228,12 +225,13 @@
 
       // No need to check for return value since DataInputStream already throws
       // EOFException in case of EOF
-      byte endpointType = dis.readByte();
-      int queueSize = dis.readInt();
+      final byte endpointType = dis.readByte();
+      final int queueSize = dis.readInt();
 
       member = readServerMember(dis);
 
-      serverQStatus = new ServerQueueStatus(endpointType, queueSize, member);
+      final ServerQueueStatus serverQStatus =
+          new ServerQueueStatus(endpointType, queueSize, member);
 
       // Read the message (if any)
       readMessage(dis, dos, acceptanceCode, member);
@@ -271,7 +269,7 @@
 
       return serverQStatus;
     } catch (IOException ex) {
-      CancelCriterion stopper = this.system.getCancelCriterion();
+      CancelCriterion stopper = system.getCancelCriterion();
       stopper.checkCancelInProgress(null);
       throw ex;
     }
@@ -301,22 +299,21 @@
   public ServerQueueStatus handshakeWithSubscriptionFeed(Socket sock, boolean isPrimary)
       throws IOException, AuthenticationRequiredException, AuthenticationFailedException,
       ServerRefusedConnectionException, ClassNotFoundException {
-    ServerQueueStatus serverQueueStatus = null;
     try {
-      DataOutputStream dos = new DataOutputStream(sock.getOutputStream());
+      final DataOutputStream dos = new DataOutputStream(sock.getOutputStream());
       final InputStream in = sock.getInputStream();
-      DataInputStream dis = new DataInputStream(in);
-      DistributedMember member = getIDForSocket(sock);
-      if (!this.multiuserSecureMode) {
-        this.credentials = getCredentials(member);
+      final DataInputStream dis = new DataInputStream(in);
+      final DistributedMember member = getIDForSocket(sock);
+      if (!multiuserSecureMode) {
+        credentials = getCredentials(member);
       }
-      CommunicationMode mode = isPrimary ? CommunicationMode.PrimaryServerToClient
+      final CommunicationMode mode = isPrimary ? CommunicationMode.PrimaryServerToClient
           : CommunicationMode.SecondaryServerToClient;
-      write(dos, dis, mode, REPLY_OK, 0, new ArrayList(), this.credentials, member, true);
+      write(dos, dis, mode, REPLY_OK, 0, new ArrayList<>(), credentials, member, true);
 
       // Wait here for a reply before continuing. This ensures that the client
       // updater is registered with the server before continuing.
-      byte acceptanceCode = dis.readByte();
+      final byte acceptanceCode = dis.readByte();
       if (acceptanceCode == (byte) 21 && !(sock instanceof SSLSocket)) {
         // This is likely the case of server setup with SSL and client not using
         // SSL
@@ -324,58 +321,51 @@
             "Server expecting SSL connection");
       }
 
-      byte endpointType = dis.readByte();
-      int queueSize = dis.readInt();
+      final byte endpointType = dis.readByte();
+      final int queueSize = dis.readInt();
 
       // Read the message (if any)
       readMessage(dis, dos, acceptanceCode, member);
 
-      // [sumedh] nothing more to be done for older clients used in tests
+      // nothing more to be done for older clients used in tests
       // there is a difference in serializer map registration for >= 6.5.1.6
       // clients but that is not used in tests
       if (currentClientVersion.isOlderThan(KnownVersion.GFE_61)) {
         return new ServerQueueStatus(endpointType, queueSize, member);
       }
-      HashMap instantiatorMap = DataSerializer.readHashMap(dis);
-      for (Iterator itr = instantiatorMap.entrySet().iterator(); itr.hasNext();) {
-        Map.Entry instantiator = (Map.Entry) itr.next();
-        Integer id = (Integer) instantiator.getKey();
-        ArrayList instantiatorArguments = (ArrayList) instantiator.getValue();
-        InternalInstantiator.register((String) instantiatorArguments.get(0),
-            (String) instantiatorArguments.get(1), id, false);
+
+      final Map<Integer, List<String>> instantiatorMap = DataSerializer.readHashMap(dis);
+      for (final Map.Entry<Integer, List<String>> entry : instantiatorMap.entrySet()) {
+        final Integer id = entry.getKey();
+        final List<String> instantiatorArguments = entry.getValue();
+        InternalInstantiator.register(instantiatorArguments.get(0), instantiatorArguments.get(1),
+            id, false);
       }
 
-      HashMap dataSerializersMap = DataSerializer.readHashMap(dis);
-      for (Iterator itr = dataSerializersMap.entrySet().iterator(); itr.hasNext();) {
-        Map.Entry dataSerializer = (Map.Entry) itr.next();
-        Integer id = (Integer) dataSerializer.getKey();
-        InternalDataSerializer.register((String) dataSerializer.getValue(), false, null, null, id);
+      final Map<Integer, String> dataSerializersMap = DataSerializer.readHashMap(dis);
+      for (final Map.Entry<Integer, String> entry : dataSerializersMap.entrySet()) {
+        InternalDataSerializer.register(entry.getValue(), false, null, null, entry.getKey());
       }
-      Map<Integer, List<String>> dsToSupportedClassNames = DataSerializer.readHashMap(dis);
+      final Map<Integer, List<String>> dsToSupportedClassNames = DataSerializer.readHashMap(dis);
       InternalDataSerializer.updateSupportedClassesMap(dsToSupportedClassNames);
 
       // the server's ping interval is only sent to subscription feeds so we can't read it as
       // part of a "standard" server response along with the other status data.
-      int pingInterval = dis.readInt();
-      serverQueueStatus = new ServerQueueStatus(endpointType, queueSize, member, pingInterval);
+      final int pingInterval = dis.readInt();
+      return new ServerQueueStatus(endpointType, queueSize, member, pingInterval);
 
-    } catch (IOException ex) {
-      CancelCriterion stopper = this.system.getCancelCriterion();
-      stopper.checkCancelInProgress(null);
-      throw ex;
-    } catch (ClassNotFoundException ex) {
-      CancelCriterion stopper = this.system.getCancelCriterion();
+    } catch (IOException | ClassNotFoundException ex) {
+      CancelCriterion stopper = system.getCancelCriterion();
       stopper.checkCancelInProgress(null);
       throw ex;
     }
-    return serverQueueStatus;
   }
 
   /**
    * client-to-server handshake. Nothing is sent to the server prior to invoking this method.
    */
   private byte write(DataOutputStream dos, DataInputStream dis, CommunicationMode communicationMode,
-      int replyCode, int readTimeout, List ports, Properties p_credentials,
+      int replyCode, int readTimeout, List<String> ports, Properties p_credentials,
       DistributedMember member, boolean isCallbackConnection) throws IOException {
     HeapDataOutputStream hdos = new HeapDataOutputStream(32, KnownVersion.CURRENT);
     byte acceptanceCode = -1;
@@ -391,8 +381,8 @@
       hdos.writeByte(replyCode);
       if (ports != null) {
         hdos.writeInt(ports.size());
-        for (int i = 0; i < ports.size(); i++) {
-          hdos.writeInt(Integer.parseInt((String) ports.get(i)));
+        for (String port : ports) {
+          hdos.writeInt(Integer.parseInt(port));
         }
       } else {
         hdos.writeInt(readTimeout);
@@ -400,25 +390,25 @@
       // we do not know the receiver's version at this point, but the on-wire
       // form of InternalDistributedMember changed in 9.0, so we must serialize
       // it using the previous version
-      DataOutput idOut = new VersionedDataOutputStream(hdos, KnownVersion.GFE_82);
-      DataSerializer.writeObject(this.id, idOut);
+      DataOutput idOut = new VersionedDataOutputStream(hdos, KnownVersion.GFE_81);
+      DataSerializer.writeObject(id, idOut);
 
       if (currentClientVersion.isNotOlderThan(KnownVersion.GFE_603)) {
         byte[] overrides = getOverrides();
-        for (int bytes = 0; bytes < overrides.length; bytes++) {
-          hdos.writeByte(overrides[bytes]);
+        for (final byte override : overrides) {
+          hdos.writeByte(override);
         }
       } else {
         // write the client conflation setting byte
         if (setClientConflationForTesting) {
           hdos.writeByte(clientConflationForTesting);
         } else {
-          hdos.writeByte(this.clientConflation);
+          hdos.writeByte(clientConflation);
         }
       }
 
       if (isCallbackConnection || communicationMode.isWAN()) {
-        if (isCallbackConnection && this.multiuserSecureMode && !communicationMode.isWAN()) {
+        if (isCallbackConnection && multiuserSecureMode && !communicationMode.isWAN()) {
           hdos.writeByte(SECURITY_MULTIUSER_NOTIFICATIONCHANNEL);
           hdos.flush();
           dos.write(hdos.toByteArray());
@@ -427,7 +417,7 @@
           writeCredentials(dos, dis, p_credentials, ports != null, member, hdos);
         }
       } else {
-        String authInitMethod = this.system.getProperties().getProperty(SECURITY_CLIENT_AUTH_INIT);
+        String authInitMethod = system.getProperties().getProperty(SECURITY_CLIENT_AUTH_INIT);
         acceptanceCode = writeCredential(dos, dis, authInitMethod, ports != null, member, hdos);
       }
     } finally {
@@ -441,7 +431,7 @@
       boolean isNotification, DistributedMember member, HeapDataOutputStream heapdos)
       throws IOException, GemFireSecurityException {
 
-    if (!this.multiuserSecureMode && (authInit == null || authInit.length() == 0)) {
+    if (!multiuserSecureMode && (authInit == null || authInit.length() == 0)) {
       // No credentials indicator
       heapdos.writeByte(CREDENTIALS_NONE);
       heapdos.flush();
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
index e61c27b..ac0cb22 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
@@ -23,6 +23,7 @@
 import org.apache.geode.annotations.internal.MutableForTesting;
 import org.apache.geode.cache.query.internal.cq.spi.CqServiceFactory;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.tier.sockets.CommandInitializer;
 import org.apache.geode.util.internal.GeodeGlossary;
 
 public class CqServiceProvider {
@@ -59,7 +60,7 @@
       return new MissingCqService();
     }
 
-    return factory.create(cache);
+    return factory.create(cache, CommandInitializer.getDefaultInstance());
   }
 
   public static ServerCQ readCq(DataInput in) throws ClassNotFoundException, IOException {
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
index 2b8a47e..d5197a3 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
@@ -20,6 +20,7 @@
 import org.apache.geode.cache.query.internal.cq.CqService;
 import org.apache.geode.cache.query.internal.cq.ServerCQ;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.tier.sockets.CommandRegistry;
 
 public interface CqServiceFactory {
 
@@ -28,7 +29,7 @@
   /**
    * Create a new CqService for the given cache
    */
-  CqService create(InternalCache cache);
+  CqService create(InternalCache cache, CommandRegistry commandRegistry);
 
   ServerCQ readCqQuery(DataInput in) throws ClassNotFoundException, IOException;
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/MessageType.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/MessageType.java
index 4e02232..171d219 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/MessageType.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/MessageType.java
@@ -280,8 +280,6 @@
 
   public static final int PUTALL = 56;
 
-  public static final int GET_ALL = 57;
-
   public static final int GET_ALL_DATA_ERROR = 58;
 
   public static final int EXECUTE_REGION_FUNCTION = 59;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientRegistrationMetadata.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientRegistrationMetadata.java
index 2790f6f..3d7e926 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientRegistrationMetadata.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientRegistrationMetadata.java
@@ -126,9 +126,10 @@
     if (clientVersion == null) {
       message = KnownVersion.unsupportedVersionMessage(clientVersionOrdinal);
     } else {
-      final Map<Integer, Command> commands = CommandInitializer.getCommands(clientVersion);
+      final Map<Integer, Command> commands =
+          CommandInitializer.getDefaultInstance().get(clientVersion);
       if (commands == null) {
-        message = "Client version {} is not supported";
+        message = "No commands registered for version " + clientVersion + ".";
       } else {
         if (isVersionOlderThan57(clientVersion)) {
           throw new IOException(new UnsupportedVersionException(clientVersionOrdinal));
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializer.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializer.java
index aa9053b..65b1757 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializer.java
@@ -15,13 +15,15 @@
 
 package org.apache.geode.internal.cache.tier.sockets;
 
-import java.util.Collections;
-import java.util.HashMap;
+import static java.util.Collections.unmodifiableMap;
+
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.geode.InternalGemFireError;
-import org.apache.geode.annotations.Immutable;
+import org.apache.geode.annotations.internal.MakeNotStatic;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.command.AddPdxEnum;
@@ -30,35 +32,21 @@
 import org.apache.geode.internal.cache.tier.sockets.command.ClientReady;
 import org.apache.geode.internal.cache.tier.sockets.command.CloseConnection;
 import org.apache.geode.internal.cache.tier.sockets.command.CommitCommand;
-import org.apache.geode.internal.cache.tier.sockets.command.ContainsKey;
 import org.apache.geode.internal.cache.tier.sockets.command.ContainsKey66;
 import org.apache.geode.internal.cache.tier.sockets.command.CreateRegion;
-import org.apache.geode.internal.cache.tier.sockets.command.Destroy;
-import org.apache.geode.internal.cache.tier.sockets.command.Destroy65;
 import org.apache.geode.internal.cache.tier.sockets.command.Destroy70;
 import org.apache.geode.internal.cache.tier.sockets.command.DestroyRegion;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction65;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction66;
 import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction70;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction65;
 import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction66;
 import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunctionGeode18;
 import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunctionSingleHop;
 import org.apache.geode.internal.cache.tier.sockets.command.GatewayReceiverCommand;
 import org.apache.geode.internal.cache.tier.sockets.command.Get70;
-import org.apache.geode.internal.cache.tier.sockets.command.GetAll;
-import org.apache.geode.internal.cache.tier.sockets.command.GetAll651;
 import org.apache.geode.internal.cache.tier.sockets.command.GetAll70;
-import org.apache.geode.internal.cache.tier.sockets.command.GetAllForRI;
 import org.apache.geode.internal.cache.tier.sockets.command.GetAllWithCallback;
-import org.apache.geode.internal.cache.tier.sockets.command.GetClientPRMetadataCommand;
 import org.apache.geode.internal.cache.tier.sockets.command.GetClientPRMetadataCommand66;
-import org.apache.geode.internal.cache.tier.sockets.command.GetClientPartitionAttributesCommand;
 import org.apache.geode.internal.cache.tier.sockets.command.GetClientPartitionAttributesCommand66;
 import org.apache.geode.internal.cache.tier.sockets.command.GetEntry70;
-import org.apache.geode.internal.cache.tier.sockets.command.GetEntryCommand;
 import org.apache.geode.internal.cache.tier.sockets.command.GetFunctionAttribute;
 import org.apache.geode.internal.cache.tier.sockets.command.GetPDXEnumById;
 import org.apache.geode.internal.cache.tier.sockets.command.GetPDXIdForEnum;
@@ -67,18 +55,12 @@
 import org.apache.geode.internal.cache.tier.sockets.command.GetPdxEnums70;
 import org.apache.geode.internal.cache.tier.sockets.command.GetPdxTypes70;
 import org.apache.geode.internal.cache.tier.sockets.command.Invalid;
-import org.apache.geode.internal.cache.tier.sockets.command.Invalidate;
 import org.apache.geode.internal.cache.tier.sockets.command.Invalidate70;
 import org.apache.geode.internal.cache.tier.sockets.command.KeySet;
 import org.apache.geode.internal.cache.tier.sockets.command.MakePrimary;
 import org.apache.geode.internal.cache.tier.sockets.command.PeriodicAck;
 import org.apache.geode.internal.cache.tier.sockets.command.Ping;
-import org.apache.geode.internal.cache.tier.sockets.command.Put;
-import org.apache.geode.internal.cache.tier.sockets.command.Put61;
-import org.apache.geode.internal.cache.tier.sockets.command.Put65;
 import org.apache.geode.internal.cache.tier.sockets.command.Put70;
-import org.apache.geode.internal.cache.tier.sockets.command.PutAll;
-import org.apache.geode.internal.cache.tier.sockets.command.PutAll70;
 import org.apache.geode.internal.cache.tier.sockets.command.PutAll80;
 import org.apache.geode.internal.cache.tier.sockets.command.PutAllWithCallback;
 import org.apache.geode.internal.cache.tier.sockets.command.PutUserCredentials;
@@ -87,14 +69,10 @@
 import org.apache.geode.internal.cache.tier.sockets.command.QueryWithParametersGeode10;
 import org.apache.geode.internal.cache.tier.sockets.command.RegisterDataSerializers;
 import org.apache.geode.internal.cache.tier.sockets.command.RegisterInstantiators;
-import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterest;
 import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterest61;
-import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterestList;
-import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterestList61;
 import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterestList66;
 import org.apache.geode.internal.cache.tier.sockets.command.RemoveAll;
 import org.apache.geode.internal.cache.tier.sockets.command.RemoveUserAuth;
-import org.apache.geode.internal.cache.tier.sockets.command.Request;
 import org.apache.geode.internal.cache.tier.sockets.command.RequestEventValue;
 import org.apache.geode.internal.cache.tier.sockets.command.RollbackCommand;
 import org.apache.geode.internal.cache.tier.sockets.command.Size;
@@ -111,37 +89,76 @@
  * @since GemFire 5.7
  */
 
-public class CommandInitializer {
+public class CommandInitializer implements CommandRegistry {
 
-  @Immutable
-  static final Map<KnownVersion, Map<Integer, Command>> ALL_COMMANDS = initializeAllCommands();
+  @Deprecated
+  @MakeNotStatic
+  static final CommandInitializer instance = new CommandInitializer();
 
   /**
-   * Register a new command with the system.
+   * Gets legacy singleton instance.
    *
-   * @param messageType - An ordinal for this message. This must be something defined in MessageType
-   *        that has not already been allocated to a different command.
-   * @param versionToNewCommand The command to register, for different versions. The key is the
-   *        earliest version for which this command class is valid (starting with GFE_57). The value
-   *        is the command object for clients starting with that version.
+   * @deprecated Efforts should be made to get and instance from the cache or other object.
+   *
+   * @return legacy singleton instance. Instance is not immutable.
    */
-  public static void registerCommand(int messageType,
+  @Deprecated
+  public static CommandInitializer getDefaultInstance() {
+    return instance;
+  }
+
+  final Map<KnownVersion, Map<Integer, Command>> unmodifiableRegisteredCommands;
+  final LinkedHashMap<KnownVersion, ConcurrentMap<Integer, Command>> modifiableRegisteredCommands;
+
+  public CommandInitializer() {
+    modifiableRegisteredCommands = initializeAllCommands();
+    unmodifiableRegisteredCommands = makeUnmodifiable(modifiableRegisteredCommands);
+  }
+
+  @Override
+  public void register(int messageType,
       Map<KnownVersion, Command> versionToNewCommand) {
+    if (!registerCommand(messageType, versionToNewCommand, modifiableRegisteredCommands)) {
+      throw new InternalGemFireError(String.format("Message %d was not registered.", messageType));
+    }
+  }
+
+  /**
+   * Gets the command map for a given version.
+   *
+   * @param version of command map to return.
+   *
+   * @return immutable {@link Map} for {@link MessageType} to {@link Command}.
+   */
+  @Override
+  public Map<Integer, Command> get(final KnownVersion version) {
+    return unmodifiableRegisteredCommands.get(version);
+  }
+
+  /**
+   * Iterate through all the Geode versions add a command to the map for that version
+   *
+   * @return returns true if command was registered or same command was already registered,
+   *         otherwise false.
+   * @throws InternalGemFireError if a different command was already registered.
+   */
+  boolean registerCommand(final int messageType,
+      final Map<KnownVersion, Command> versionToNewCommand,
+      final LinkedHashMap<KnownVersion, ConcurrentMap<Integer, Command>> allCommands) {
+    boolean modified = false;
     Command command = null;
-    // Iterate through all the gemfire versions, and
-    // add a command to the map for that version
-    for (Map.Entry<KnownVersion, Map<Integer, Command>> entry : ALL_COMMANDS.entrySet()) {
+
+    for (Map.Entry<KnownVersion, ConcurrentMap<Integer, Command>> entry : allCommands.entrySet()) {
       KnownVersion version = entry.getKey();
 
       // Get the current set of commands for this version.
       Map<Integer, Command> commandMap = entry.getValue();
 
       // See if we have a new command to insert into this map. Otherwise, keep using the command we
-      // have
-      // already read
-      Command newerVersion = versionToNewCommand.get(version);
-      if (newerVersion != null) {
-        command = newerVersion;
+      // have already read
+      Command newCommand = versionToNewCommand.get(version);
+      if (newCommand != null) {
+        command = newCommand;
       }
       if (command != null) {
         Command oldCommand = commandMap.get(messageType);
@@ -151,190 +168,33 @@
               + ", newValue=" + command + ", version=" + version);
         }
         commandMap.put(messageType, command);
+        modified = true;
       }
     }
+    return modified;
   }
 
-  private static Map<KnownVersion, Map<Integer, Command>> initializeAllCommands() {
-    final LinkedHashMap<KnownVersion, Map<Integer, Command>> allCommands =
-        new LinkedHashMap<KnownVersion, Map<Integer, Command>>();
+  private static LinkedHashMap<KnownVersion, ConcurrentMap<Integer, Command>> initializeAllCommands() {
+    final LinkedHashMap<KnownVersion, ConcurrentMap<Integer, Command>> allCommands =
+        new LinkedHashMap<>();
 
-    allCommands.put(KnownVersion.GFE_56, new HashMap<>());
-
-    // Initialize the GFE 5.7 commands
-    Map<Integer, Command> gfe57Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_57, gfe57Commands);
-    gfe57Commands.put(MessageType.PING, Ping.getCommand());
-    gfe57Commands.put(MessageType.REQUEST, Request.getCommand());
-    gfe57Commands.put(MessageType.PUT, Put.getCommand());
-    gfe57Commands.put(MessageType.PUTALL, PutAll.getCommand());
-    gfe57Commands.put(MessageType.DESTROY, Destroy.getCommand());
-    gfe57Commands.put(MessageType.QUERY,
-        org.apache.geode.internal.cache.tier.sockets.command.Query.getCommand());
-    gfe57Commands.put(MessageType.CLEAR_REGION, ClearRegion.getCommand());
-    gfe57Commands.put(MessageType.DESTROY_REGION, DestroyRegion.getCommand());
-    gfe57Commands.put(MessageType.REGISTER_INTEREST, RegisterInterest.getCommand());
-    gfe57Commands.put(MessageType.UNREGISTER_INTEREST, UnregisterInterest.getCommand());
-    gfe57Commands.put(MessageType.REGISTER_INTEREST_LIST, RegisterInterestList.getCommand());
-    gfe57Commands.put(MessageType.UNREGISTER_INTEREST_LIST, UnregisterInterestList.getCommand());
-    gfe57Commands.put(MessageType.KEY_SET, KeySet.getCommand());
-    gfe57Commands.put(MessageType.CONTAINS_KEY, ContainsKey.getCommand());
-    gfe57Commands.put(MessageType.CREATE_REGION, CreateRegion.getCommand());
-    gfe57Commands.put(MessageType.MAKE_PRIMARY, MakePrimary.getCommand());
-    gfe57Commands.put(MessageType.PERIODIC_ACK, PeriodicAck.getCommand());
-    gfe57Commands.put(MessageType.REGISTER_INSTANTIATORS, RegisterInstantiators.getCommand());
-    gfe57Commands.put(MessageType.UPDATE_CLIENT_NOTIFICATION,
-        UpdateClientNotification.getCommand());
-    gfe57Commands.put(MessageType.CLOSE_CONNECTION, CloseConnection.getCommand());
-    gfe57Commands.put(MessageType.CLIENT_READY, ClientReady.getCommand());
-    gfe57Commands.put(MessageType.INVALID, Invalid.getCommand());
-
-
-    gfe57Commands.put(MessageType.GET_ALL, GetAll.getCommand());
-
-    // Initialize the GFE 5.8 commands example
-    Map<Integer, Command> gfe58Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_58, gfe58Commands);
-    gfe58Commands.putAll(allCommands.get(KnownVersion.GFE_57));
-    gfe58Commands.put(MessageType.EXECUTE_REGION_FUNCTION, ExecuteRegionFunction.getCommand());
-    gfe58Commands.put(MessageType.EXECUTE_FUNCTION, ExecuteFunction.getCommand());
-
-    // Initialize the GFE 6.0.3 commands map
-    Map<Integer, Command> gfe603Commands =
-        new HashMap<Integer, Command>(allCommands.get(KnownVersion.GFE_58));
-    allCommands.put(KnownVersion.GFE_603, gfe603Commands);
-
-    // Initialize the GFE 6.1 commands
-    Map<Integer, Command> gfe61Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_61, gfe61Commands);
-    gfe61Commands.putAll(allCommands.get(KnownVersion.GFE_603));
-    gfe61Commands.put(MessageType.REGISTER_INTEREST, RegisterInterest61.getCommand());
-    gfe61Commands.put(MessageType.REGISTER_INTEREST_LIST, RegisterInterestList61.getCommand());
-    gfe61Commands.put(MessageType.REQUEST_EVENT_VALUE, RequestEventValue.getCommand());
-    gfe61Commands.put(MessageType.PUT, Put61.getCommand());
-    gfe61Commands.put(MessageType.REGISTER_DATASERIALIZERS, RegisterDataSerializers.getCommand());
-
-    // Initialize the GFE 6.5 commands
-    Map<Integer, Command> gfe65Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_65, gfe65Commands);
-    gfe65Commands.putAll(allCommands.get(KnownVersion.GFE_61));
-    gfe65Commands.put(MessageType.DESTROY, Destroy65.getCommand());
-    gfe65Commands.put(MessageType.PUT, Put65.getCommand());
-    gfe65Commands.put(MessageType.EXECUTE_REGION_FUNCTION, ExecuteRegionFunction65.getCommand());
-    gfe65Commands.put(MessageType.EXECUTE_FUNCTION, ExecuteFunction65.getCommand());
-    gfe65Commands.put(MessageType.GET_CLIENT_PR_METADATA, GetClientPRMetadataCommand.getCommand());
-    gfe65Commands.put(MessageType.GET_CLIENT_PARTITION_ATTRIBUTES,
-        GetClientPartitionAttributesCommand.getCommand());
-    gfe65Commands.put(MessageType.USER_CREDENTIAL_MESSAGE, PutUserCredentials.getCommand());
-    gfe65Commands.put(MessageType.REMOVE_USER_AUTH, RemoveUserAuth.getCommand());
-    gfe65Commands.put(MessageType.EXECUTE_REGION_FUNCTION_SINGLE_HOP,
-        ExecuteRegionFunctionSingleHop.getCommand());
-
-    // Initialize the GFE 6.5.1 commands
-    Map<Integer, Command> gfe651Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_651, gfe651Commands);
-    gfe651Commands.putAll(allCommands.get(KnownVersion.GFE_65));
-    gfe651Commands.put(MessageType.QUERY_WITH_PARAMETERS, Query651.getCommand());
-
-    // Initialize the GFE 6.5.1.6 commands
-    Map<Integer, Command> gfe6516Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_6516, gfe6516Commands);
-    gfe6516Commands.putAll(allCommands.get(KnownVersion.GFE_651));
-    gfe6516Commands.put(MessageType.GET_ALL, GetAll651.getCommand());
-    gfe6516Commands.put(MessageType.GET_CLIENT_PR_METADATA,
-        GetClientPRMetadataCommand66.getCommand());
-
-    // Initialize the GFE 6.6 commands
-    Map<Integer, Command> gfe66Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_66, gfe66Commands);
-    gfe66Commands.putAll(allCommands.get(KnownVersion.GFE_6516));
-    gfe66Commands.put(MessageType.ADD_PDX_TYPE, AddPdxType.getCommand());
-    gfe66Commands.put(MessageType.GET_PDX_ID_FOR_TYPE, GetPDXIdForType.getCommand());
-    gfe66Commands.put(MessageType.GET_PDX_TYPE_BY_ID, GetPDXTypeById.getCommand());
-    gfe66Commands.put(MessageType.SIZE, Size.getCommand());
-    gfe66Commands.put(MessageType.INVALIDATE, Invalidate.getCommand());
-    gfe66Commands.put(MessageType.COMMIT, CommitCommand.getCommand());
-    gfe66Commands.put(MessageType.ROLLBACK, RollbackCommand.getCommand());
-    gfe66Commands.put(MessageType.TX_FAILOVER, TXFailoverCommand.getCommand());
-    gfe66Commands.put(MessageType.GET_ENTRY, GetEntryCommand.getCommand());
-    gfe66Commands.put(MessageType.TX_SYNCHRONIZATION, TXSynchronizationCommand.getCommand());
-    gfe66Commands.put(MessageType.GET_CLIENT_PARTITION_ATTRIBUTES,
-        GetClientPartitionAttributesCommand66.getCommand());
-    gfe66Commands.put(MessageType.REGISTER_INTEREST_LIST, RegisterInterestList66.getCommand());
-    gfe66Commands.put(MessageType.GET_FUNCTION_ATTRIBUTES, GetFunctionAttribute.getCommand());
-    gfe66Commands.put(MessageType.EXECUTE_REGION_FUNCTION, ExecuteRegionFunction66.getCommand());
-    gfe66Commands.put(MessageType.EXECUTE_FUNCTION, ExecuteFunction66.getCommand());
-    gfe66Commands.put(MessageType.GET_ALL_FOR_RI, GetAllForRI.getCommand());
-    gfe66Commands.put(MessageType.GATEWAY_RECEIVER_COMMAND, GatewayReceiverCommand.getCommand());
-    gfe66Commands.put(MessageType.CONTAINS_KEY, ContainsKey66.getCommand());
-
-    // Initialize the GFE 6.6.2 commands
-    Map<Integer, Command> gfe662Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_662, gfe662Commands);
-    gfe662Commands.putAll(allCommands.get(KnownVersion.GFE_66));
-    gfe662Commands.put(MessageType.ADD_PDX_ENUM, AddPdxEnum.getCommand());
-    gfe662Commands.put(MessageType.GET_PDX_ID_FOR_ENUM, GetPDXIdForEnum.getCommand());
-    gfe662Commands.put(MessageType.GET_PDX_ENUM_BY_ID, GetPDXEnumById.getCommand());
-
-    // Initialize the GFE 6.6.2.2 commands (same commands as the GFE 6.6.2 commands)
-    // The SERVER_TO_CLIENT_PING message was added, but it doesn't need to be registered here
-    allCommands.put(KnownVersion.GFE_6622, gfe662Commands);
-
-    // Initialize the GFE 70 commands
-    Map<Integer, Command> gfe70Commands = new HashMap<Integer, Command>();
-    allCommands.put(KnownVersion.GFE_70, gfe70Commands);
-    gfe70Commands.putAll(allCommands.get(KnownVersion.GFE_662));
-    gfe70Commands.remove(MessageType.GET_ALL_FOR_RI);
-    gfe70Commands.put(MessageType.REQUEST, Get70.getCommand());
-    gfe70Commands.put(MessageType.GET_ENTRY, GetEntry70.getCommand());
-    gfe70Commands.put(MessageType.GET_ALL_70, GetAll70.getCommand());
-    gfe70Commands.put(MessageType.PUTALL, PutAll70.getCommand());
-    gfe70Commands.put(MessageType.PUT, Put70.getCommand());
-    gfe70Commands.put(MessageType.DESTROY, Destroy70.getCommand());
-    gfe70Commands.put(MessageType.INVALIDATE, Invalidate70.getCommand());
-    gfe70Commands.put(MessageType.GET_PDX_TYPES, GetPdxTypes70.getCommand());
-    gfe70Commands.put(MessageType.GET_PDX_ENUMS, GetPdxEnums70.getCommand());
-    gfe70Commands.put(MessageType.EXECUTE_FUNCTION, ExecuteFunction70.getCommand());
-
-    allCommands.put(KnownVersion.GFE_701, gfe70Commands);
-    allCommands.put(KnownVersion.GFE_7099, gfe70Commands);
-    allCommands.put(KnownVersion.GFE_71, gfe70Commands);
-
-    Map<Integer, Command> gfe80Commands =
-        new HashMap<Integer, Command>(allCommands.get(KnownVersion.GFE_71));
-    allCommands.put(KnownVersion.GFE_80, gfe80Commands);
-    // PutAll is changed to chunk responses back to the client
-    gfe80Commands.put(MessageType.PUTALL, PutAll80.getCommand());
-
-    allCommands.put(KnownVersion.GFE_8009, gfe80Commands);
-
-    Map<Integer, Command> gfe81Commands = new HashMap<Integer, Command>(gfe80Commands);
-    gfe81Commands.put(MessageType.GET_ALL_WITH_CALLBACK, GetAllWithCallback.getCommand());
-    gfe81Commands.put(MessageType.PUT_ALL_WITH_CALLBACK, PutAllWithCallback.getCommand());
-    gfe81Commands.put(MessageType.REMOVE_ALL, RemoveAll.getCommand());
-
+    final ConcurrentMap<Integer, Command> gfe81Commands = buildGfe81Commands();
     allCommands.put(KnownVersion.GFE_81, gfe81Commands);
-    allCommands.put(KnownVersion.GFE_82, gfe81Commands);
 
-    Map<Integer, Command> commands =
-        new HashMap<Integer, Command>(allCommands.get(KnownVersion.GFE_82));
-    allCommands.put(KnownVersion.GFE_90, commands);
-    commands.put(MessageType.QUERY_WITH_PARAMETERS, QueryWithParametersGeode10.getCommand());
-    commands.put(MessageType.QUERY, QueryGeode10.getCommand());
+    final ConcurrentMap<Integer, Command> gfe90Commands =
+        buildGfe90Commands(allCommands.get(KnownVersion.GFE_81));
+    allCommands.put(KnownVersion.GFE_90, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_1_0, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_1_1, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_2_0, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_3_0, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_4_0, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_5_0, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_6_0, gfe90Commands);
+    allCommands.put(KnownVersion.GEODE_1_7_0, gfe90Commands);
 
-    allCommands.put(KnownVersion.GEODE_1_1_0, commands);
-    allCommands.put(KnownVersion.GEODE_1_1_1, commands);
-    allCommands.put(KnownVersion.GEODE_1_2_0, commands);
-    allCommands.put(KnownVersion.GEODE_1_3_0, commands);
-    allCommands.put(KnownVersion.GEODE_1_4_0, commands);
-    allCommands.put(KnownVersion.GEODE_1_5_0, commands);
-    allCommands.put(KnownVersion.GEODE_1_6_0, commands);
-    allCommands.put(KnownVersion.GEODE_1_7_0, commands);
-
-    Map<Integer, Command> geode18Commands =
-        new HashMap<Integer, Command>(allCommands.get(KnownVersion.GEODE_1_7_0));
-    geode18Commands.put(MessageType.EXECUTE_REGION_FUNCTION,
-        ExecuteRegionFunctionGeode18.getCommand());
+    final ConcurrentMap<Integer, Command> geode18Commands =
+        buildGeode18Commands(allCommands.get(KnownVersion.GEODE_1_7_0));
     allCommands.put(KnownVersion.GEODE_1_8_0, geode18Commands);
     allCommands.put(KnownVersion.GEODE_1_9_0, geode18Commands);
     allCommands.put(KnownVersion.GEODE_1_10_0, geode18Commands);
@@ -345,14 +205,115 @@
     allCommands.put(KnownVersion.GEODE_1_13_1, geode18Commands);
 
     allCommands.put(KnownVersion.GEODE_1_14_0, geode18Commands);
-    return Collections.unmodifiableMap(allCommands);
+
+    return allCommands;
   }
 
-  public static Map<Integer, Command> getCommands(KnownVersion version) {
-    return ALL_COMMANDS.get(version);
+  private static ConcurrentMap<Integer, Command> buildGeode18Commands(
+      final ConcurrentMap<Integer, Command> baseCommands) {
+    final ConcurrentMap<Integer, Command> commands = new ConcurrentHashMap<>(baseCommands);
+    initializeGeode18Commands(commands);
+    return commands;
   }
 
-  public static Map<Integer, Command> getCommands(ServerConnection connection) {
-    return getCommands(connection.getClientVersion());
+  private static ConcurrentMap<Integer, Command> buildGfe90Commands(
+      final ConcurrentMap<Integer, Command> baseCommands) {
+    final ConcurrentMap<Integer, Command> commands = new ConcurrentHashMap<>(baseCommands);
+    initializeGfe90Commands(commands);
+    return commands;
   }
+
+  private static ConcurrentMap<Integer, Command> buildGfe81Commands() {
+    final ConcurrentMap<Integer, Command> commands = new ConcurrentHashMap<>();
+    initializeGfe81Commands(commands);
+    return commands;
+  }
+
+  static void initializeGeode18Commands(final Map<Integer, Command> commands) {
+    commands.put(MessageType.EXECUTE_REGION_FUNCTION, ExecuteRegionFunctionGeode18.getCommand());
+  }
+
+  static void initializeGfe90Commands(final Map<Integer, Command> commands) {
+    commands.put(MessageType.QUERY_WITH_PARAMETERS, QueryWithParametersGeode10.getCommand());
+    commands.put(MessageType.QUERY, QueryGeode10.getCommand());
+  }
+
+  static void initializeGfe81Commands(final Map<Integer, Command> commands) {
+    commands.put(MessageType.PING, Ping.getCommand());
+    commands.put(MessageType.QUERY,
+        org.apache.geode.internal.cache.tier.sockets.command.Query.getCommand());
+    commands.put(MessageType.CLEAR_REGION, ClearRegion.getCommand());
+    commands.put(MessageType.DESTROY_REGION, DestroyRegion.getCommand());
+    commands.put(MessageType.UNREGISTER_INTEREST, UnregisterInterest.getCommand());
+    commands.put(MessageType.UNREGISTER_INTEREST_LIST, UnregisterInterestList.getCommand());
+    commands.put(MessageType.KEY_SET, KeySet.getCommand());
+    commands.put(MessageType.CREATE_REGION, CreateRegion.getCommand());
+    commands.put(MessageType.MAKE_PRIMARY, MakePrimary.getCommand());
+    commands.put(MessageType.PERIODIC_ACK, PeriodicAck.getCommand());
+    commands.put(MessageType.REGISTER_INSTANTIATORS, RegisterInstantiators.getCommand());
+    commands.put(MessageType.UPDATE_CLIENT_NOTIFICATION,
+        UpdateClientNotification.getCommand());
+    commands.put(MessageType.CLOSE_CONNECTION, CloseConnection.getCommand());
+    commands.put(MessageType.CLIENT_READY, ClientReady.getCommand());
+    commands.put(MessageType.INVALID, Invalid.getCommand());
+
+    commands.put(MessageType.REGISTER_INTEREST, RegisterInterest61.getCommand());
+    commands.put(MessageType.REQUEST_EVENT_VALUE, RequestEventValue.getCommand());
+    commands.put(MessageType.REGISTER_DATASERIALIZERS, RegisterDataSerializers.getCommand());
+
+    commands.put(MessageType.USER_CREDENTIAL_MESSAGE, PutUserCredentials.getCommand());
+    commands.put(MessageType.REMOVE_USER_AUTH, RemoveUserAuth.getCommand());
+    commands.put(MessageType.EXECUTE_REGION_FUNCTION_SINGLE_HOP,
+        ExecuteRegionFunctionSingleHop.getCommand());
+
+    commands.put(MessageType.QUERY_WITH_PARAMETERS, Query651.getCommand());
+
+    commands.put(MessageType.GET_CLIENT_PR_METADATA, GetClientPRMetadataCommand66.getCommand());
+
+    commands.put(MessageType.ADD_PDX_TYPE, AddPdxType.getCommand());
+    commands.put(MessageType.GET_PDX_ID_FOR_TYPE, GetPDXIdForType.getCommand());
+    commands.put(MessageType.GET_PDX_TYPE_BY_ID, GetPDXTypeById.getCommand());
+    commands.put(MessageType.SIZE, Size.getCommand());
+    commands.put(MessageType.COMMIT, CommitCommand.getCommand());
+    commands.put(MessageType.ROLLBACK, RollbackCommand.getCommand());
+    commands.put(MessageType.TX_FAILOVER, TXFailoverCommand.getCommand());
+    commands.put(MessageType.TX_SYNCHRONIZATION, TXSynchronizationCommand.getCommand());
+    commands.put(MessageType.GET_CLIENT_PARTITION_ATTRIBUTES,
+        GetClientPartitionAttributesCommand66.getCommand());
+    commands.put(MessageType.REGISTER_INTEREST_LIST, RegisterInterestList66.getCommand());
+    commands.put(MessageType.GET_FUNCTION_ATTRIBUTES, GetFunctionAttribute.getCommand());
+    commands.put(MessageType.EXECUTE_REGION_FUNCTION, ExecuteRegionFunction66.getCommand());
+    commands.put(MessageType.GATEWAY_RECEIVER_COMMAND, GatewayReceiverCommand.getCommand());
+    commands.put(MessageType.CONTAINS_KEY, ContainsKey66.getCommand());
+
+    commands.put(MessageType.ADD_PDX_ENUM, AddPdxEnum.getCommand());
+    commands.put(MessageType.GET_PDX_ID_FOR_ENUM, GetPDXIdForEnum.getCommand());
+    commands.put(MessageType.GET_PDX_ENUM_BY_ID, GetPDXEnumById.getCommand());
+
+    commands.put(MessageType.REQUEST, Get70.getCommand());
+    commands.put(MessageType.GET_ENTRY, GetEntry70.getCommand());
+    commands.put(MessageType.GET_ALL_70, GetAll70.getCommand());
+    commands.put(MessageType.PUT, Put70.getCommand());
+    commands.put(MessageType.DESTROY, Destroy70.getCommand());
+    commands.put(MessageType.INVALIDATE, Invalidate70.getCommand());
+    commands.put(MessageType.GET_PDX_TYPES, GetPdxTypes70.getCommand());
+    commands.put(MessageType.GET_PDX_ENUMS, GetPdxEnums70.getCommand());
+    commands.put(MessageType.EXECUTE_FUNCTION, ExecuteFunction70.getCommand());
+
+    commands.put(MessageType.PUTALL, PutAll80.getCommand());
+    commands.put(MessageType.GET_ALL_WITH_CALLBACK, GetAllWithCallback.getCommand());
+    commands.put(MessageType.PUT_ALL_WITH_CALLBACK, PutAllWithCallback.getCommand());
+    commands.put(MessageType.REMOVE_ALL, RemoveAll.getCommand());
+  }
+
+  static Map<KnownVersion, Map<Integer, Command>> makeUnmodifiable(
+      final Map<KnownVersion, ConcurrentMap<Integer, Command>> modifiableMap) {
+    final Map<KnownVersion, Map<Integer, Command>> unmodifiableMap =
+        new LinkedHashMap<>(modifiableMap.size());
+    for (Map.Entry<KnownVersion, ConcurrentMap<Integer, Command>> e : modifiableMap.entrySet()) {
+      unmodifiableMap.put(e.getKey(), unmodifiableMap(e.getValue()));
+    }
+    return unmodifiableMap(unmodifiableMap);
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandRegistry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandRegistry.java
new file mode 100644
index 0000000..b0b948c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CommandRegistry.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.internal.cache.tier.sockets;
+
+import java.util.Map;
+
+import org.apache.geode.InternalGemFireError;
+import org.apache.geode.internal.cache.tier.Command;
+import org.apache.geode.internal.serialization.KnownVersion;
+
+/**
+ * Registry of server commands by version.
+ */
+public interface CommandRegistry {
+
+  /**
+   * Register a new command with the system.
+   *
+   * @param messageType - An ordinal for this message. This must be something defined in MessageType
+   *        that has not already been allocated to a different command.
+   * @param versionToNewCommand The command to register, for different versions. The key is the
+   *        earliest version for which this command class is valid (starting with
+   *        {@link KnownVersion#OLDEST}).
+   *
+   * @throws InternalGemFireError if a different command is already registered for given version or
+   *         if no command was added to the registry.
+   */
+  void register(int messageType, Map<KnownVersion, Command> versionToNewCommand);
+
+  Map<Integer, Command> get(KnownVersion version);
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
index 4f1553f..9dfc9f2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
@@ -986,9 +986,9 @@
   }
 
   void initializeCommands() {
-    // The commands are cached here, but are just referencing the ones
-    // stored in the CommandInitializer
-    commands = CommandInitializer.getCommands(this);
+    // The commands are cached here, but are just referencing the ones stored in the
+    // CommandInitializer
+    commands = CommandInitializer.getDefaultInstance().get(this.getClientVersion());
   }
 
   private Command getCommand(Integer messageType) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerSideHandshakeFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerSideHandshakeFactory.java
index e86b1de..d4b1581 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerSideHandshakeFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerSideHandshakeFactory.java
@@ -51,7 +51,7 @@
       logger.debug("Client version: {}", clientVersion);
     }
 
-    if (clientVersion.isOlderThan(KnownVersion.GFE_57)) {
+    if (clientVersion.isOlderThan(KnownVersion.OLDEST)) {
       throw new UnsupportedVersionException("Unsupported version " + clientVersion
           + "Server's current version " + currentServerVersion);
     }
@@ -78,9 +78,10 @@
       if (clientVersion == null) {
         message = KnownVersion.unsupportedVersionMessage(clientVersionOrdinal);
       } else {
-        final Map<Integer, Command> commands = CommandInitializer.getCommands(clientVersion);
+        final Map<Integer, Command> commands =
+            CommandInitializer.getDefaultInstance().get(clientVersion);
         if (commands == null) {
-          message = "Client version {} is not supported";
+          message = "No commands registered for version " + clientVersion + ".";
         } else {
           return clientVersion;
         }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ContainsKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ContainsKey.java
deleted file mode 100644
index 5623d33..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ContainsKey.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class ContainsKey extends BaseCommand {
-
-  @Immutable
-  private static final ContainsKey singleton = new ContainsKey();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private static void writeContainsKeyResponse(boolean containsKey, Message origMsg,
-      ServerConnection servConn) throws IOException {
-    Message responseMsg = servConn.getResponseMessage();
-    responseMsg.setMessageType(MessageType.RESPONSE);
-    responseMsg.setNumberOfParts(1);
-    responseMsg.setTransactionId(origMsg.getTransactionId());
-    responseMsg.addObjPart(containsKey ? Boolean.TRUE : Boolean.FALSE);
-    responseMsg.send(servConn);
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    Part regionNamePart = null;
-    Part keyPart = null;
-    String regionName = null;
-    Object key = null;
-
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadContainsKeyRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    keyPart = clientMessage.getPart(1);
-    regionName = regionNamePart.getCachedString();
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received containsKey request ({} bytes) from {} for region {} key {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key);
-    }
-
-    // Process the containsKey request
-    if (key == null || regionName == null) {
-      String errMessage = "";
-      if (key == null) {
-        logger.warn("{}: The input key for the containsKey request is null",
-            serverConnection.getName());
-        errMessage = "The input key for the containsKey request is null";
-      }
-      if (regionName == null) {
-        logger.warn("{}: The input region name for the containsKey request is null",
-            serverConnection.getName());
-        errMessage =
-            "The input region name for the containsKey request is null";
-      }
-      writeErrorResponse(clientMessage, MessageType.CONTAINS_KEY_DATA_ERROR, errMessage,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason =
-          "was not found during containsKey request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    try {
-      securityService.authorize(Resource.DATA, Operation.READ, regionName, key);
-    } catch (NotAuthorizedException ex) {
-      writeException(clientMessage, ex, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-    if (authzRequest != null) {
-      try {
-        authzRequest.containsKeyAuthorize(regionName, key);
-      } catch (NotAuthorizedException ex) {
-        writeException(clientMessage, ex, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    // Execute the containsKey
-    boolean containsKey = region.containsKey(key);
-
-    // Update the statistics and write the reply
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessContainsKeyTime(start - oldStart);
-    }
-    writeContainsKeyResponse(containsKey, clientMessage, serverConnection);
-    serverConnection.setAsTrue(RESPONDED);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sent containsKey response for region {} key {}", serverConnection.getName(),
-          regionName, key);
-    }
-    stats.incWriteContainsKeyResponseTime(DistributionStats.getStatTime() - start);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy.java
deleted file mode 100644
index b0ecfaa..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.EntryNotFoundException;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.operations.DestroyOperationContext;
-import org.apache.geode.cache.operations.RegionDestroyOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class Destroy extends BaseCommand {
-
-  @Immutable
-  private static final Destroy singleton = new Destroy();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long startparam)
-      throws IOException, InterruptedException {
-    long start = startparam;
-
-    Part regionNamePart = null, keyPart = null, callbackArgPart = null;
-    String regionName = null;
-    Object callbackArg = null, key = null;
-    Part eventPart = null;
-    StringBuffer errMessage = new StringBuffer();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadDestroyRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    keyPart = clientMessage.getPart(1);
-    eventPart = clientMessage.getPart(2);
-    // callbackArgPart = null; (redundant assignment)
-    if (clientMessage.getNumberOfParts() > 3) {
-      callbackArgPart = clientMessage.getPart(3);
-      try {
-        callbackArg = callbackArgPart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    regionName = regionNamePart.getCachedString();
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received destroy request ({} bytes) from {} for region {} key {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key);
-    }
-
-    // Process the destroy request
-    if (key == null || regionName == null) {
-      if (key == null) {
-        logger.warn("{}: The input key for the destroy request is null",
-            serverConnection.getName());
-        errMessage.append("The input key for the destroy request is null");
-      }
-      if (regionName == null) {
-        logger.warn("{}: The input region name for the destroy request is null",
-            serverConnection.getName());
-        errMessage
-            .append("The input region name for the destroy request is null");
-      }
-      writeErrorResponse(clientMessage, MessageType.DESTROY_DATA_ERROR, errMessage.toString(),
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during destroy request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    // Destroy the entry
-    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    EventID eventId =
-        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-
-    try {
-      // for integrated security
-      securityService.authorize(Resource.DATA, Operation.WRITE, regionName, key);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          RegionDestroyOperationContext destroyContext =
-              authzRequest.destroyRegionAuthorize((String) key, callbackArg);
-          callbackArg = destroyContext.getCallbackArg();
-        } else {
-          DestroyOperationContext destroyContext =
-              authzRequest.destroyAuthorize(regionName, key, callbackArg);
-          callbackArg = destroyContext.getCallbackArg();
-        }
-      }
-      region.basicBridgeDestroy(key, callbackArg, serverConnection.getProxyID(), true,
-          new EventIDHolder(eventId));
-      serverConnection.setModificationInfo(true, regionName, key);
-    } catch (EntryNotFoundException e) {
-      // Don't send an exception back to the client if this
-      // exception happens. Just log it and continue.
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: during entry destroy no entry was found for key {}",
-            serverConnection.getName(), key);
-      }
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (Exception e) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, e);
-
-      // If an exception occurs during the destroy, preserve the connection
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      if (e instanceof GemFireSecurityException) {
-        // Fine logging for security exceptions since these are already
-        // logged by the security logger
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), e);
-        }
-      } else {
-        logger.warn(String.format("%s: Unexpected Exception",
-            serverConnection.getName()), e);
-      }
-      return;
-    }
-
-    // Update the statistics and write the reply
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessDestroyTime(start - oldStart);
-    }
-    if (region instanceof PartitionedRegion) {
-      PartitionedRegion pr = (PartitionedRegion) region;
-      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, pr.getNetworkHopType());
-        pr.clearNetworkHopData();
-      } else {
-        writeReply(clientMessage, serverConnection);
-      }
-    } else {
-      writeReply(clientMessage, serverConnection);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sent destroy response for region {} key {}", serverConnection.getName(),
-          regionName, key);
-    }
-    stats.incWriteDestroyResponseTime(DistributionStats.getStatTime() - start);
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy65.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy65.java
deleted file mode 100644
index 9e49712..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy65.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.EntryNotFoundException;
-import org.apache.geode.cache.Operation;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.operations.DestroyOperationContext;
-import org.apache.geode.cache.operations.RegionDestroyOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.OpType;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.versions.VersionTag;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.util.Breadcrumbs;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class Destroy65 extends BaseCommand {
-
-  @Immutable
-  private static final Destroy65 singleton = new Destroy65();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection serverConnection,
-      PartitionedRegion pr, byte nwHop) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection servConn,
-      PartitionedRegion pr, boolean entryNotFoundForRemove, byte nwHop, VersionTag tag)
-      throws IOException {
-    Message replyMsg = servConn.getReplyMessage();
-    servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
-    replyMsg.setMessageType(MessageType.REPLY);
-    replyMsg.setNumberOfParts(2);
-    replyMsg.setTransactionId(origMsg.getTransactionId());
-    replyMsg.addBytesPart(new byte[] {pr.getMetadataVersion(), nwHop});
-    pr.getPrStats().incPRMetaDataSentCount();
-    replyMsg.addIntPart(entryNotFoundForRemove ? 1 : 0);
-    replyMsg.send(servConn);
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: rpl with REFRESH_METADATA tx: {}", servConn.getName(),
-          origMsg.getTransactionId());
-    }
-  }
-
-  protected void writeReply(Message origMsg, ServerConnection servConn, boolean entryNotFound,
-      VersionTag tag) throws IOException {
-    Message replyMsg = servConn.getReplyMessage();
-    servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
-    replyMsg.setMessageType(MessageType.REPLY);
-    replyMsg.setNumberOfParts(2);
-    replyMsg.setTransactionId(origMsg.getTransactionId());
-    replyMsg.addBytesPart(okBytes());
-    replyMsg.addIntPart(entryNotFound ? 1 : 0);
-    replyMsg.send(servConn);
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: rpl tx: {} parts={}", servConn.getName(), origMsg.getTransactionId(),
-          replyMsg.getNumberOfParts());
-    }
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    StringBuilder errMessage = new StringBuilder();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-
-    long now = DistributionStats.getStatTime();
-    stats.incReadDestroyRequestTime(now - start);
-
-    // Retrieve the data from the message parts
-    final Part regionNamePart = clientMessage.getPart(0);
-    final Part keyPart = clientMessage.getPart(1);
-    final Part expectedOldValuePart = clientMessage.getPart(2);
-
-    final Operation operation;
-    try {
-      final Part operationPart = clientMessage.getPart(3);
-
-      if (operationPart.isBytes()) {
-        final byte[] bytes = operationPart.getSerializedForm();
-        if (null == bytes || 0 == bytes.length) {
-          // older clients can send empty bytes for default operation.
-          operation = Operation.DESTROY;
-        } else {
-          operation = Operation.fromOrdinal(bytes[0]);
-        }
-      } else {
-        // Fallback for older clients.
-        final Object operationObject = operationPart.getObject();
-        if (operationObject == null) {
-          // native clients may send a null since the op is java-serialized.
-          operation = Operation.DESTROY;
-        } else if (operationObject instanceof Byte
-            && (Byte) operationObject == OpType.DESTROY) {
-          // older native clients may send Byte object OpType.DESTROY value treated as
-          // Operation.REMOVE.
-          operation = Operation.REMOVE;
-        } else {
-          operation = (Operation) operationObject;
-        }
-      }
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    Object expectedOldValue = null;
-    if (operation == Operation.REMOVE) {
-      try {
-        expectedOldValue = expectedOldValuePart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    final Part eventPart = clientMessage.getPart(4);
-
-    Object callbackArg = null;
-    if (clientMessage.getNumberOfParts() > 5) {
-      final Part callbackArgPart = clientMessage.getPart(5);
-      try {
-        callbackArg = callbackArgPart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    final Object key;
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final String regionName = regionNamePart.getCachedString();
-
-    if (logger.isDebugEnabled()) {
-      logger.debug(
-          "{}: Received destroy65 request ({} bytes; op={}) from {} for region {} key {}{} txId {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(), operation,
-          serverConnection.getSocketString(), regionName, key,
-          (operation == Operation.REMOVE ? " value=" + expectedOldValue : ""),
-          clientMessage.getTransactionId());
-    }
-    boolean entryNotFoundForRemove = false;
-
-    // Process the destroy request
-    if (key == null || regionName == null) {
-      if (key == null) {
-        logger.warn("{}: The input key for the destroy request is null",
-            serverConnection.getName());
-        errMessage.append("The input key for the destroy request is null");
-      }
-      if (regionName == null) {
-        logger.warn("{}: The input region name for the destroy request is null",
-            serverConnection.getName());
-        errMessage
-            .append("The input region name for the destroy request is null");
-      }
-      writeErrorResponse(clientMessage, MessageType.DESTROY_DATA_ERROR, errMessage.toString(),
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during destroy request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    // Destroy the entry
-    final ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-    final long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    final long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    final EventID eventId =
-        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-    final EventIDHolder clientEvent = new EventIDHolder(eventId);
-
-    Breadcrumbs.setEventId(eventId);
-
-    // msg.isRetry might be set by v7.0 and later clients
-    if (clientMessage.isRetry()) {
-      // if (logger.isDebugEnabled()) {
-      // logger.debug("DEBUG: encountered isRetry in Destroy65");
-      // }
-      clientEvent.setPossibleDuplicate(true);
-      if (region.getAttributes().getConcurrencyChecksEnabled()) {
-        // recover the version tag from other servers
-        clientEvent.setRegion(region);
-        if (!recoverVersionTagForRetriedOperation(clientEvent)) {
-          clientEvent.setPossibleDuplicate(false); // no-one has seen this event
-        }
-      }
-    }
-
-    try {
-      // for integrated security
-      securityService.authorize(Resource.DATA, ResourcePermission.Operation.WRITE, regionName,
-          key);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          RegionDestroyOperationContext destroyContext =
-              authzRequest.destroyRegionAuthorize((String) key, callbackArg);
-          callbackArg = destroyContext.getCallbackArg();
-        } else {
-          DestroyOperationContext destroyContext =
-              authzRequest.destroyAuthorize(regionName, key, callbackArg);
-          callbackArg = destroyContext.getCallbackArg();
-        }
-      }
-      if (operation == null || operation == Operation.DESTROY) {
-        region.basicBridgeDestroy(key, callbackArg, serverConnection.getProxyID(), true,
-            clientEvent);
-      } else {
-        // this throws exceptions if expectedOldValue checks fail
-        try {
-          if (expectedOldValue == null) {
-            expectedOldValue = Token.INVALID;
-          }
-          if (operation == Operation.REMOVE && clientMessage.isRetry()
-              && clientEvent.getVersionTag() != null) {
-            // the operation was successful last time it was tried, so there's
-            // no need to perform it again. Just return the version tag and
-            // success status
-            if (logger.isDebugEnabled()) {
-              logger.debug("remove(k,v) operation was successful last time with version {}",
-                  clientEvent.getVersionTag());
-            }
-            // try the operation anyway to ensure that it's been distributed to all servers
-            try {
-              region.basicBridgeRemove(key, expectedOldValue, callbackArg,
-                  serverConnection.getProxyID(), true, clientEvent);
-            } catch (EntryNotFoundException e) {
-              // ignore, and don't set entryNotFoundForRemove because this was a successful
-              // operation - bug #51664
-            }
-          } else {
-            region.basicBridgeRemove(key, expectedOldValue, callbackArg,
-                serverConnection.getProxyID(), true, clientEvent);
-            if (logger.isDebugEnabled()) {
-              logger.debug("region.remove succeeded");
-            }
-          }
-        } catch (EntryNotFoundException e) {
-          serverConnection.setModificationInfo(true, regionName, key);
-          if (logger.isDebugEnabled()) {
-            logger.debug("writing entryNotFound response");
-          }
-          entryNotFoundForRemove = true;
-        }
-      }
-      serverConnection.setModificationInfo(true, regionName, key);
-    } catch (EntryNotFoundException e) {
-      // Don't send an exception back to the client if this
-      // exception happens. Just log it and continue.
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: during entry destroy no entry was found for key {}",
-            serverConnection.getName(), key);
-      }
-      entryNotFoundForRemove = true;
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (Exception e) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, e);
-
-      // If an exception occurs during the destroy, preserve the connection
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      if (e instanceof GemFireSecurityException) {
-        // Fine logging for security exceptions since these are already
-        // logged by the security logger
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), e);
-        }
-      } else {
-        logger.warn(String.format("%s: Unexpected Exception",
-            serverConnection.getName()), e);
-      }
-      return;
-    }
-
-    // Update the statistics and write the reply
-    now = DistributionStats.getStatTime();
-    stats.incProcessDestroyTime(now - start);
-
-    if (region instanceof PartitionedRegion) {
-      PartitionedRegion pr = (PartitionedRegion) region;
-      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, entryNotFoundForRemove,
-            pr.getNetworkHopType(), clientEvent.getVersionTag());
-        pr.clearNetworkHopData();
-      } else {
-        writeReply(clientMessage, serverConnection,
-            entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(),
-            clientEvent.getVersionTag());
-      }
-    } else {
-      writeReply(clientMessage, serverConnection,
-          entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(),
-          clientEvent.getVersionTag());
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sent destroy response for region {} key {}", serverConnection.getName(),
-          regionName, key);
-    }
-    stats.incWriteDestroyResponseTime(DistributionStats.getStatTime() - start);
-
-
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy70.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy70.java
index 58f6038..519edbb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy70.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy70.java
@@ -15,17 +15,38 @@
 package org.apache.geode.internal.cache.tier.sockets.command;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.geode.annotations.Immutable;
+import org.apache.geode.cache.DynamicRegionFactory;
+import org.apache.geode.cache.EntryNotFoundException;
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.RegionDestroyedException;
 import org.apache.geode.cache.client.internal.DestroyOp;
+import org.apache.geode.cache.operations.DestroyOperationContext;
+import org.apache.geode.cache.operations.RegionDestroyOperationContext;
+import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.internal.cache.EventID;
+import org.apache.geode.internal.cache.EventIDHolder;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.OpType;
 import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
+import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
 import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.cache.versions.VersionTag;
+import org.apache.geode.internal.security.AuthorizeRequest;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.util.Breadcrumbs;
+import org.apache.geode.security.GemFireSecurityException;
+import org.apache.geode.security.ResourcePermission;
 
-public class Destroy70 extends Destroy65 {
+public class Destroy70 extends BaseCommand {
   @Immutable
   private static final Destroy70 singleton = new Destroy70();
 
@@ -35,7 +56,6 @@
 
   private Destroy70() {}
 
-  @Override
   protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection servConn,
       PartitionedRegion pr, boolean entryNotFoundForRemove, byte nwHop, VersionTag versionTag)
       throws IOException {
@@ -68,7 +88,6 @@
     }
   }
 
-  @Override
   protected void writeReply(Message origMsg, ServerConnection servConn, boolean entryNotFound,
       VersionTag versionTag) throws IOException {
     if (logger.isDebugEnabled()) {
@@ -108,4 +127,273 @@
           replyMsg.getNumberOfParts());
     }
   }
+
+  @Override
+  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
+      final SecurityService securityService, long start) throws IOException, InterruptedException {
+    StringBuilder errMessage = new StringBuilder();
+    CacheServerStats stats = serverConnection.getCacheServerStats();
+    serverConnection.setAsTrue(REQUIRES_RESPONSE);
+
+    long now = DistributionStats.getStatTime();
+    stats.incReadDestroyRequestTime(now - start);
+
+    // Retrieve the data from the message parts
+    final Part regionNamePart = clientMessage.getPart(0);
+    final Part keyPart = clientMessage.getPart(1);
+    final Part expectedOldValuePart = clientMessage.getPart(2);
+
+    final Operation operation;
+    try {
+      final Part operationPart = clientMessage.getPart(3);
+
+      if (operationPart.isBytes()) {
+        final byte[] bytes = operationPart.getSerializedForm();
+        if (null == bytes || 0 == bytes.length) {
+          // older clients can send empty bytes for default operation.
+          operation = Operation.DESTROY;
+        } else {
+          operation = Operation.fromOrdinal(bytes[0]);
+        }
+      } else {
+        // Fallback for older clients.
+        final Object operationObject = operationPart.getObject();
+        if (operationObject == null) {
+          // native clients may send a null since the op is java-serialized.
+          operation = Operation.DESTROY;
+        } else if (operationObject instanceof Byte
+            && (Byte) operationObject == OpType.DESTROY) {
+          // older native clients may send Byte object OpType.DESTROY value treated as
+          // Operation.REMOVE.
+          operation = Operation.REMOVE;
+        } else {
+          operation = (Operation) operationObject;
+        }
+      }
+    } catch (Exception e) {
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    Object expectedOldValue = null;
+    if (operation == Operation.REMOVE) {
+      try {
+        expectedOldValue = expectedOldValuePart.getObject();
+      } catch (Exception e) {
+        writeException(clientMessage, e, false, serverConnection);
+        serverConnection.setAsTrue(RESPONDED);
+        return;
+      }
+    }
+
+    final Part eventPart = clientMessage.getPart(4);
+
+    Object callbackArg = null;
+    if (clientMessage.getNumberOfParts() > 5) {
+      final Part callbackArgPart = clientMessage.getPart(5);
+      try {
+        callbackArg = callbackArgPart.getObject();
+      } catch (Exception e) {
+        writeException(clientMessage, e, false, serverConnection);
+        serverConnection.setAsTrue(RESPONDED);
+        return;
+      }
+    }
+
+    final Object key;
+    try {
+      key = keyPart.getStringOrObject();
+    } catch (Exception e) {
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    final String regionName = regionNamePart.getCachedString();
+
+    if (logger.isDebugEnabled()) {
+      logger.debug(
+          "{}: Received destroy65 request ({} bytes; op={}) from {} for region {} key {}{} txId {}",
+          serverConnection.getName(), clientMessage.getPayloadLength(), operation,
+          serverConnection.getSocketString(), regionName, key,
+          (operation == Operation.REMOVE ? " value=" + expectedOldValue : ""),
+          clientMessage.getTransactionId());
+    }
+    boolean entryNotFoundForRemove = false;
+
+    // Process the destroy request
+    if (key == null || regionName == null) {
+      if (key == null) {
+        logger.warn("{}: The input key for the destroy request is null",
+            serverConnection.getName());
+        errMessage.append("The input key for the destroy request is null");
+      }
+      if (regionName == null) {
+        logger.warn("{}: The input region name for the destroy request is null",
+            serverConnection.getName());
+        errMessage
+            .append("The input region name for the destroy request is null");
+      }
+      writeErrorResponse(clientMessage, MessageType.DESTROY_DATA_ERROR, errMessage.toString(),
+          serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    final LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during destroy request";
+      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    // Destroy the entry
+    final ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    final long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    final long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    final EventID eventId =
+        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
+    final EventIDHolder clientEvent = new EventIDHolder(eventId);
+
+    Breadcrumbs.setEventId(eventId);
+
+    // msg.isRetry might be set by v7.0 and later clients
+    if (clientMessage.isRetry()) {
+      // if (logger.isDebugEnabled()) {
+      // logger.debug("DEBUG: encountered isRetry in Destroy65");
+      // }
+      clientEvent.setPossibleDuplicate(true);
+      if (region.getAttributes().getConcurrencyChecksEnabled()) {
+        // recover the version tag from other servers
+        clientEvent.setRegion(region);
+        if (!recoverVersionTagForRetriedOperation(clientEvent)) {
+          clientEvent.setPossibleDuplicate(false); // no-one has seen this event
+        }
+      }
+    }
+
+    try {
+      // for integrated security
+      securityService.authorize(ResourcePermission.Resource.DATA,
+          ResourcePermission.Operation.WRITE, regionName,
+          key);
+
+      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
+      if (authzRequest != null) {
+        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          RegionDestroyOperationContext destroyContext =
+              authzRequest.destroyRegionAuthorize((String) key, callbackArg);
+          callbackArg = destroyContext.getCallbackArg();
+        } else {
+          DestroyOperationContext destroyContext =
+              authzRequest.destroyAuthorize(regionName, key, callbackArg);
+          callbackArg = destroyContext.getCallbackArg();
+        }
+      }
+      if (operation == null || operation == Operation.DESTROY) {
+        region.basicBridgeDestroy(key, callbackArg, serverConnection.getProxyID(), true,
+            clientEvent);
+      } else {
+        // this throws exceptions if expectedOldValue checks fail
+        try {
+          if (expectedOldValue == null) {
+            expectedOldValue = Token.INVALID;
+          }
+          if (operation == Operation.REMOVE && clientMessage.isRetry()
+              && clientEvent.getVersionTag() != null) {
+            // the operation was successful last time it was tried, so there's
+            // no need to perform it again. Just return the version tag and
+            // success status
+            if (logger.isDebugEnabled()) {
+              logger.debug("remove(k,v) operation was successful last time with version {}",
+                  clientEvent.getVersionTag());
+            }
+            // try the operation anyway to ensure that it's been distributed to all servers
+            try {
+              region.basicBridgeRemove(key, expectedOldValue, callbackArg,
+                  serverConnection.getProxyID(), true, clientEvent);
+            } catch (EntryNotFoundException e) {
+              // ignore, and don't set entryNotFoundForRemove because this was a successful
+              // operation - bug #51664
+            }
+          } else {
+            region.basicBridgeRemove(key, expectedOldValue, callbackArg,
+                serverConnection.getProxyID(), true, clientEvent);
+            if (logger.isDebugEnabled()) {
+              logger.debug("region.remove succeeded");
+            }
+          }
+        } catch (EntryNotFoundException e) {
+          serverConnection.setModificationInfo(true, regionName, key);
+          if (logger.isDebugEnabled()) {
+            logger.debug("writing entryNotFound response");
+          }
+          entryNotFoundForRemove = true;
+        }
+      }
+      serverConnection.setModificationInfo(true, regionName, key);
+    } catch (EntryNotFoundException e) {
+      // Don't send an exception back to the client if this
+      // exception happens. Just log it and continue.
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: during entry destroy no entry was found for key {}",
+            serverConnection.getName(), key);
+      }
+      entryNotFoundForRemove = true;
+    } catch (RegionDestroyedException rde) {
+      writeException(clientMessage, rde, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(serverConnection, e);
+
+      // If an exception occurs during the destroy, preserve the connection
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      if (e instanceof GemFireSecurityException) {
+        // Fine logging for security exceptions since these are already
+        // logged by the security logger
+        if (logger.isDebugEnabled()) {
+          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), e);
+        }
+      } else {
+        logger.warn(String.format("%s: Unexpected Exception",
+            serverConnection.getName()), e);
+      }
+      return;
+    }
+
+    // Update the statistics and write the reply
+    now = DistributionStats.getStatTime();
+    stats.incProcessDestroyTime(now - start);
+
+    if (region instanceof PartitionedRegion) {
+      PartitionedRegion pr = (PartitionedRegion) region;
+      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
+        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, entryNotFoundForRemove,
+            pr.getNetworkHopType(), clientEvent.getVersionTag());
+        pr.clearNetworkHopData();
+      } else {
+        writeReply(clientMessage, serverConnection,
+            entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(),
+            clientEvent.getVersionTag());
+      }
+    } else {
+      writeReply(clientMessage, serverConnection,
+          entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(),
+          clientEvent.getVersionTag());
+    }
+    serverConnection.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent destroy response for region {} key {}", serverConnection.getName(),
+          regionName, key);
+    }
+    stats.incWriteDestroyResponseTime(DistributionStats.getStatTime() - start);
+
+
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction.java
deleted file mode 100755
index baea0a7..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.annotations.VisibleForTesting;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.ResultSender;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.execute.FunctionContextImpl;
-import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
-import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
-import org.apache.geode.internal.cache.execute.InternalFunctionService;
-import org.apache.geode.internal.cache.execute.MemberMappedArgument;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.execute.metrics.FunctionStats;
-import org.apache.geode.internal.cache.execute.metrics.FunctionStatsManager;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- * This is the base command which read the parts for the MessageType.EXECUTE_FUNCTION.<br>
- * If the hasResult byte is 1, then this command send back the result after the execution to the
- * client else do not send the reply back to the client
- *
- * @since GemFire 5.8Beta
- */
-public class ExecuteFunction extends BaseCommand {
-
-  @Immutable
-  private static final ExecuteFunction singleton = new ExecuteFunction();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private final InternalFunctionExecutionService internalFunctionExecutionService;
-  private final ServerToClientFunctionResultSenderFactory serverToClientFunctionResultSenderFactory;
-  private final FunctionContextImplFactory functionContextImplFactory;
-
-  private ExecuteFunction() {
-    this(InternalFunctionService.getInternalFunctionExecutionService(),
-        new DefaultServerToClientFunctionResultSenderFactory(),
-        new DefaultFunctionContextImplFactory());
-  }
-
-  @VisibleForTesting
-  ExecuteFunction(InternalFunctionExecutionService internalFunctionExecutionService,
-      ServerToClientFunctionResultSenderFactory serverToClientFunctionResultSenderFactory,
-      FunctionContextImplFactory functionContextImplFactory) {
-    this.internalFunctionExecutionService = internalFunctionExecutionService;
-    this.serverToClientFunctionResultSenderFactory = serverToClientFunctionResultSenderFactory;
-    this.functionContextImplFactory = functionContextImplFactory;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    Object function = null;
-    Object args = null;
-    MemberMappedArgument memberMappedArg = null;
-    byte hasResult = 0;
-
-    try {
-      hasResult = clientMessage.getPart(0).getSerializedForm()[0];
-      if (hasResult == 1) {
-        serverConnection.setAsTrue(REQUIRES_RESPONSE);
-        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-      }
-      function = clientMessage.getPart(1).getStringOrObject();
-      args = clientMessage.getPart(2).getObject();
-
-      Part part = clientMessage.getPart(3);
-      if (part != null) {
-        memberMappedArg = (MemberMappedArgument) part.getObject();
-      }
-    } catch (ClassNotFoundException e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      if (hasResult == 1) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    if (function == null) {
-      String message = "The input function for the execute function request is null";
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    // Execute function on the cache
-    try {
-      Function<?> functionObject;
-      if (function instanceof String) {
-        functionObject = internalFunctionExecutionService.getFunction((String) function);
-        if (functionObject == null) {
-          String message = String.format("Function named %s is not registered to FunctionService",
-              function);
-          logger.warn("{}: {}", serverConnection.getName(), message);
-          sendError(hasResult, clientMessage, message, serverConnection);
-          return;
-        }
-      } else {
-        functionObject = (Function) function;
-      }
-
-      FunctionStats stats = FunctionStatsManager.getFunctionStats(functionObject.getId());
-
-      // check if the caller is authorized to do this operation on server
-      functionObject.getRequiredPermissions(null, args).forEach(securityService::authorize);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      ExecuteFunctionOperationContext executeContext = null;
-      if (authzRequest != null) {
-        executeContext = authzRequest.executeFunctionAuthorize(functionObject.getId(), null, null,
-            args, functionObject.optimizeForWrite());
-      }
-
-      ChunkedMessage chunkedMessage = serverConnection.getFunctionResponseMessage();
-      chunkedMessage.setTransactionId(clientMessage.getTransactionId());
-      ResultSender resultSender = serverToClientFunctionResultSenderFactory.create(chunkedMessage,
-          MessageType.EXECUTE_FUNCTION_RESULT, serverConnection, functionObject, executeContext);
-
-      FunctionContext context;
-      InternalCache cache = serverConnection.getCache();
-      InternalDistributedMember localVM =
-          (InternalDistributedMember) cache.getDistributedSystem().getDistributedMember();
-
-      if (memberMappedArg != null) {
-        context = functionContextImplFactory.create(cache, functionObject.getId(),
-            memberMappedArg.getArgumentsForMember(localVM.getId()), resultSender);
-      } else {
-        context =
-            functionContextImplFactory.create(cache, functionObject.getId(), args, resultSender);
-      }
-
-      ServerSideHandshake handshake = serverConnection.getHandshake();
-      int earlierClientReadTimeout = handshake.getClientReadTimeout();
-      handshake.setClientReadTimeout(0);
-
-      long startExecution = stats.startFunctionExecution(functionObject.hasResult());
-      try {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function on Server: {} with context: {}", serverConnection,
-              context);
-        }
-        cache.getInternalResourceManager().getHeapMonitor().checkForLowMemory(functionObject,
-            cache.getMyId());
-        functionObject.execute(context);
-        stats.endFunctionExecution(startExecution, functionObject.hasResult());
-      } catch (FunctionException e) {
-        stats.endFunctionExecutionWithException(startExecution, functionObject.hasResult());
-        throw e;
-      } catch (Exception e) {
-        stats.endFunctionExecutionWithException(startExecution, functionObject.hasResult());
-        throw new FunctionException(e);
-      } finally {
-        handshake.setClientReadTimeout(earlierClientReadTimeout);
-      }
-
-    } catch (IOException e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      String message = "Server could not send the reply";
-      sendException(hasResult, clientMessage, message, serverConnection, e);
-
-    } catch (InternalFunctionInvocationTargetException e) {
-      /*
-       * TRAC #44709: InternalFunctionInvocationTargetException should not be logged
-       * Fix for #44709: User should not be aware of InternalFunctionInvocationTargetException. No
-       * instance is giving useful information to user to take any corrective action hence logging
-       * this at fine level logging. May occur when:
-       * 1> When bucket is moved
-       * 2> In case of HA FunctionInvocationTargetException thrown. Since it is HA, function will
-       * be re-executed on right node
-       * 3> Multiple target nodes found for single hop operation
-       * 4> in case of HA member departed
-       */
-      if (logger.isDebugEnabled()) {
-        logger.debug("Exception on server while executing function: {}", function, e);
-      }
-      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
-
-    } catch (Exception e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
-    }
-  }
-
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection, Throwable e) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseException(msg, MessageType.EXCEPTION, serverConnection, e);
-      serverConnection.setAsTrue(RESPONDED);
-    }
-  }
-
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR, message,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-    }
-  }
-
-  interface ServerToClientFunctionResultSenderFactory {
-    ServerToClientFunctionResultSender create(ChunkedMessage msg, int messageType,
-        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext);
-  }
-
-  interface FunctionContextImplFactory {
-    FunctionContextImpl create(Cache cache, String functionId, Object args,
-        ResultSender resultSender);
-  }
-
-  private static class DefaultServerToClientFunctionResultSenderFactory
-      implements ServerToClientFunctionResultSenderFactory {
-    @Override
-    public ServerToClientFunctionResultSender create(ChunkedMessage msg, int messageType,
-        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext) {
-      return new ServerToClientFunctionResultSender(msg, messageType, sc, function, authzContext);
-    }
-  }
-
-  private static class DefaultFunctionContextImplFactory implements FunctionContextImplFactory {
-    @Override
-    public FunctionContextImpl create(Cache cache, String functionId, Object args,
-        ResultSender resultSender) {
-      return new FunctionContextImpl(cache, functionId, args, resultSender);
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction65.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction65.java
deleted file mode 100644
index a252c4a..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction65.java
+++ /dev/null
@@ -1,302 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.annotations.VisibleForTesting;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.LowMemoryException;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.ResultSender;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.execute.AbstractExecution;
-import org.apache.geode.internal.cache.execute.FunctionContextImpl;
-import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
-import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
-import org.apache.geode.internal.cache.execute.InternalFunctionService;
-import org.apache.geode.internal.cache.execute.MemberMappedArgument;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
-import org.apache.geode.internal.cache.execute.metrics.FunctionStats;
-import org.apache.geode.internal.cache.execute.metrics.FunctionStatsManager;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- * @since GemFire 6.5
- */
-public class ExecuteFunction65 extends BaseCommand {
-
-  @Immutable
-  private static final ExecuteFunction65 singleton = new ExecuteFunction65();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private final InternalFunctionExecutionService internalFunctionExecutionService;
-  private final ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory;
-  private final FunctionContextImplFactory functionContextImplFactory;
-
-  private ExecuteFunction65() {
-    this(InternalFunctionService.getInternalFunctionExecutionService(),
-        new DefaultServerToClientFunctionResultSender65Factory(),
-        new DefaultFunctionContextImplFactory());
-  }
-
-  @VisibleForTesting
-  ExecuteFunction65(InternalFunctionExecutionService internalFunctionExecutionService,
-      ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory,
-      FunctionContextImplFactory functionContextImplFactory) {
-    this.internalFunctionExecutionService = internalFunctionExecutionService;
-    this.serverToClientFunctionResultSender65Factory = serverToClientFunctionResultSender65Factory;
-    this.functionContextImplFactory = functionContextImplFactory;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    Object function = null;
-    Object args = null;
-    MemberMappedArgument memberMappedArg = null;
-    byte hasResult = 0;
-    byte functionState = 0;
-    boolean isReexecute = false;
-
-    try {
-      functionState = clientMessage.getPart(0).getSerializedForm()[0];
-
-      if (functionState == AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE_REEXECUTE) {
-        functionState = AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE;
-        isReexecute = true;
-      } else if (functionState == AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE_REEXECUTE) {
-        functionState = AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE;
-        isReexecute = true;
-      }
-
-      if (functionState != 1) {
-        hasResult = (byte) ((functionState & 2) - 1);
-      } else {
-        hasResult = functionState;
-      }
-      if (hasResult == 1) {
-        serverConnection.setAsTrue(REQUIRES_RESPONSE);
-        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-      }
-      function = clientMessage.getPart(1).getStringOrObject();
-      args = clientMessage.getPart(2).getObject();
-
-      Part part = clientMessage.getPart(3);
-      if (part != null) {
-        memberMappedArg = (MemberMappedArgument) part.getObject();
-      }
-    } catch (ClassNotFoundException e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      if (hasResult == 1) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    if (function == null) {
-      String message = "The input function for the execute function request is null";
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    // Execute function on the cache
-    try {
-      Function<?> functionObject;
-      if (function instanceof String) {
-        functionObject = internalFunctionExecutionService.getFunction((String) function);
-        if (functionObject == null) {
-          String message = String.format("Function named %s is not registered to FunctionService",
-              function);
-          logger.warn("{}: {}", serverConnection.getName(), message);
-          sendError(hasResult, clientMessage, message, serverConnection);
-          return;
-        } else {
-          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(),
-              functionObject.hasResult(), functionObject.optimizeForWrite());
-          if (logger.isDebugEnabled()) {
-            logger.debug("Function State on server side: {} on client: {}",
-                functionStateOnServerSide, functionState);
-          }
-          if (functionStateOnServerSide != functionState) {
-            String message =
-                "Function attributes at client and server don't match";
-            logger.warn("{}: {}", serverConnection.getName(), message);
-            sendError(hasResult, clientMessage, message, serverConnection);
-            return;
-          }
-        }
-      } else {
-        functionObject = (Function) function;
-      }
-
-      FunctionStats stats = FunctionStatsManager.getFunctionStats(functionObject.getId());
-
-      // check if the caller is authorized to do this operation on server
-      functionObject.getRequiredPermissions(null, args).forEach(securityService::authorize);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      ExecuteFunctionOperationContext executeContext = null;
-      if (authzRequest != null) {
-        executeContext = authzRequest.executeFunctionAuthorize(functionObject.getId(), null, null,
-            args, functionObject.optimizeForWrite());
-      }
-
-      ChunkedMessage chunkedMessage = serverConnection.getFunctionResponseMessage();
-      chunkedMessage.setTransactionId(clientMessage.getTransactionId());
-      ResultSender resultSender = serverToClientFunctionResultSender65Factory.create(chunkedMessage,
-          MessageType.EXECUTE_FUNCTION_RESULT, serverConnection, functionObject, executeContext);
-
-      FunctionContext context;
-      InternalCache cache = serverConnection.getCache();
-      InternalDistributedMember localVM =
-          (InternalDistributedMember) cache.getDistributedSystem().getDistributedMember();
-
-      if (memberMappedArg != null) {
-        context = functionContextImplFactory.create(cache, functionObject.getId(),
-            memberMappedArg.getArgumentsForMember(localVM.getId()), resultSender, isReexecute);
-      } else {
-        context = functionContextImplFactory.create(cache, functionObject.getId(), args,
-            resultSender, isReexecute);
-      }
-
-      ServerSideHandshake handshake = serverConnection.getHandshake();
-      int earlierClientReadTimeout = handshake.getClientReadTimeout();
-      handshake.setClientReadTimeout(0);
-
-      long startExecution = stats.startFunctionExecution(functionObject.hasResult());
-      try {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function on Server: {} with context: {}", serverConnection,
-              context);
-        }
-
-        cache.getInternalResourceManager().getHeapMonitor().createLowMemoryIfNeeded(null,
-            (DistributedMember) null);
-
-        LowMemoryException lowMemoryException = cache.getInternalResourceManager().getHeapMonitor()
-            .createLowMemoryIfNeeded(functionObject, cache.getMyId());
-        if (lowMemoryException != null) {
-          sendException(hasResult, clientMessage, lowMemoryException.getMessage(), serverConnection,
-              lowMemoryException);
-          return;
-        }
-        functionObject.execute(context);
-        if (!((ServerToClientFunctionResultSender65) resultSender).isLastResultReceived()
-            && functionObject.hasResult()) {
-          throw new FunctionException(String.format("The function, %s, did not send last result",
-              functionObject.getId()));
-        }
-        stats.endFunctionExecution(startExecution, functionObject.hasResult());
-      } catch (FunctionException e) {
-        stats.endFunctionExecutionWithException(startExecution, functionObject.hasResult());
-        throw e;
-      } catch (Exception e) {
-        stats.endFunctionExecutionWithException(startExecution, functionObject.hasResult());
-        throw new FunctionException(e);
-      } finally {
-        handshake.setClientReadTimeout(earlierClientReadTimeout);
-      }
-
-    } catch (IOException e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      String message = "Server could not send the reply";
-      sendException(hasResult, clientMessage, message, serverConnection, e);
-
-    } catch (InternalFunctionInvocationTargetException e) {
-      /*
-       * TRAC #44709: InternalFunctionInvocationTargetException should not be logged
-       * Fix for #44709: User should not be aware of InternalFunctionInvocationTargetException. No
-       * instance is giving useful information to user to take any corrective action hence logging
-       * this at fine level logging. May occur when:
-       * 1> In case of HA FunctionInvocationTargetException thrown. Since it is HA, function will
-       * be re-executed on right node
-       * 2> in case of HA member departed
-       */
-      if (logger.isDebugEnabled()) {
-        logger.debug("Exception on server while executing function: {}", function, e);
-      }
-      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
-
-    } catch (Exception e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
-    }
-  }
-
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection, Throwable e) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseException(msg, MessageType.EXCEPTION, serverConnection, e);
-      serverConnection.setAsTrue(RESPONDED);
-    }
-  }
-
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR, message,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-    }
-  }
-
-  interface ServerToClientFunctionResultSender65Factory {
-    ServerToClientFunctionResultSender65 create(ChunkedMessage msg, int messageType,
-        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext);
-  }
-
-  interface FunctionContextImplFactory {
-    FunctionContextImpl create(Cache cache, String functionId, Object args,
-        ResultSender resultSender, boolean isPossibleDuplicate);
-  }
-
-  private static class DefaultServerToClientFunctionResultSender65Factory
-      implements ServerToClientFunctionResultSender65Factory {
-    @Override
-    public ServerToClientFunctionResultSender65 create(ChunkedMessage msg, int messageType,
-        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext) {
-      return new ServerToClientFunctionResultSender65(msg, messageType, sc, function, authzContext);
-    }
-  }
-
-  private static class DefaultFunctionContextImplFactory implements FunctionContextImplFactory {
-    @Override
-    public FunctionContextImpl create(Cache cache, String functionId, Object args,
-        ResultSender resultSender, boolean isPossibleDuplicat) {
-      return new FunctionContextImpl(cache, functionId, args, resultSender, isPossibleDuplicat);
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction66.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction66.java
deleted file mode 100644
index bb68e04..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction66.java
+++ /dev/null
@@ -1,427 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.apache.geode.internal.cache.execute.ServerFunctionExecutor.DEFAULT_CLIENT_FUNCTION_TIMEOUT;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.geode.InternalGemFireError;
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.annotations.internal.MakeNotStatic;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.LowMemoryException;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.ResultSender;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.TXManagerImpl;
-import org.apache.geode.internal.cache.TXStateProxy;
-import org.apache.geode.internal.cache.execute.AbstractExecution;
-import org.apache.geode.internal.cache.execute.FunctionContextImpl;
-import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
-import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
-import org.apache.geode.internal.cache.execute.InternalFunctionService;
-import org.apache.geode.internal.cache.execute.MemberMappedArgument;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
-import org.apache.geode.internal.cache.execute.metrics.FunctionStats;
-import org.apache.geode.internal.cache.execute.metrics.FunctionStatsManager;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.logging.internal.executors.LoggingExecutors;
-
-/**
- * @since GemFire 6.6
- */
-public class ExecuteFunction66 extends BaseCommand {
-  @Immutable
-  private static final ExecuteFunction66 singleton = new ExecuteFunction66();
-
-  @MakeNotStatic
-  private static volatile boolean asyncTxWarningIssued;
-
-  @Immutable
-  private static final ExecutorService execService =
-      LoggingExecutors.newCachedThreadPool("Function Execution Thread-", true);
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private final InternalFunctionExecutionService internalFunctionExecutionService;
-  private final ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory;
-  private final FunctionContextImplFactory functionContextImplFactory;
-
-  ExecuteFunction66() {
-    this(InternalFunctionService.getInternalFunctionExecutionService(),
-        new DefaultServerToClientFunctionResultSender65Factory(),
-        new DefaultFunctionContextImplFactory());
-  }
-
-  ExecuteFunction66(InternalFunctionExecutionService internalFunctionExecutionService,
-      ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory,
-      FunctionContextImplFactory functionContextImplFactory) {
-    this.internalFunctionExecutionService = internalFunctionExecutionService;
-    this.serverToClientFunctionResultSender65Factory = serverToClientFunctionResultSender65Factory;
-    this.functionContextImplFactory = functionContextImplFactory;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    Object function = null;
-    Object args;
-    MemberMappedArgument memberMappedArg = null;
-    String[] groups;
-    byte hasResult = 0;
-    byte functionState;
-    boolean isReexecute = false;
-    boolean allMembers;
-    boolean ignoreFailedMembers;
-    int functionTimeout = DEFAULT_CLIENT_FUNCTION_TIMEOUT;
-
-    try {
-      byte[] bytes = clientMessage.getPart(0).getSerializedForm();
-      functionState = bytes[0];
-      if (bytes.length >= 5
-          && serverConnection.getClientVersion().ordinal() >= KnownVersion.GFE_8009.ordinal()) {
-        functionTimeout = Part.decodeInt(bytes, 1);
-      }
-
-      if (functionState == AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE_REEXECUTE) {
-        functionState = AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE;
-        isReexecute = true;
-      } else if (functionState == AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE_REEXECUTE) {
-        functionState = AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE;
-        isReexecute = true;
-      }
-
-      if (functionState != 1) {
-        hasResult = (byte) ((functionState & 2) - 1);
-      } else {
-        hasResult = functionState;
-      }
-      if (hasResult == 1) {
-        serverConnection.setAsTrue(REQUIRES_RESPONSE);
-        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-      }
-      function = clientMessage.getPart(1).getStringOrObject();
-      args = clientMessage.getPart(2).getObject();
-
-      Part part = clientMessage.getPart(3);
-      if (part != null) {
-        memberMappedArg = (MemberMappedArgument) part.getObject();
-      }
-
-      groups = getGroups(clientMessage);
-      allMembers = getAllMembers(clientMessage);
-      ignoreFailedMembers = getIgnoreFailedMembers(clientMessage);
-    } catch (ClassNotFoundException e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      if (hasResult == 1) {
-        writeChunkedException(clientMessage, e, serverConnection);
-      } else {
-        writeException(clientMessage, e, false, serverConnection);
-      }
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (function == null) {
-      String message = "The input function for the execute function request is null";
-      logger.warn("{} : {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    // Execute function on the cache
-    try {
-      Function<?> functionObject;
-      if (function instanceof String) {
-        functionObject = internalFunctionExecutionService.getFunction((String) function);
-        if (functionObject == null) {
-          String message = String
-              .format("Function named %s is not registered to FunctionService", function);
-          logger.warn("{}: {}", serverConnection.getName(), message);
-          sendError(hasResult, clientMessage, message, serverConnection);
-          return;
-        } else {
-          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(),
-              functionObject.hasResult(), functionObject.optimizeForWrite());
-          if (logger.isDebugEnabled()) {
-            logger.debug("Function State on server side: {} on client: {}",
-                functionStateOnServerSide, functionState);
-          }
-          if (functionStateOnServerSide != functionState) {
-            String message = String
-                .format("Function attributes at client and server don't match for %s", function);
-            logger.warn("{}: {}", serverConnection.getName(), message);
-            sendError(hasResult, clientMessage, message, serverConnection);
-            return;
-          }
-        }
-      } else {
-        functionObject = (Function) function;
-      }
-
-      FunctionStats stats = FunctionStatsManager.getFunctionStats(functionObject.getId());
-
-      // check if the caller is authorized to do this operation on server
-      functionObject.getRequiredPermissions(null, args).forEach(securityService::authorize);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      ExecuteFunctionOperationContext executeContext = null;
-      if (authzRequest != null) {
-        executeContext = authzRequest.executeFunctionAuthorize(functionObject.getId(), null, null,
-            args, functionObject.optimizeForWrite());
-      }
-
-      ChunkedMessage chunkedMessage = serverConnection.getFunctionResponseMessage();
-      chunkedMessage.setTransactionId(clientMessage.getTransactionId());
-      ServerToClientFunctionResultSender resultSender =
-          serverToClientFunctionResultSender65Factory.create(chunkedMessage,
-              MessageType.EXECUTE_FUNCTION_RESULT,
-              serverConnection, functionObject, executeContext);
-
-      FunctionContext context;
-      InternalCache cache = serverConnection.getCache();
-      InternalDistributedMember localVM =
-          (InternalDistributedMember) cache.getDistributedSystem().getDistributedMember();
-
-      if (memberMappedArg != null) {
-        context = functionContextImplFactory.create(cache, functionObject.getId(),
-            memberMappedArg.getArgumentsForMember(localVM.getId()), resultSender, isReexecute);
-      } else {
-        context = functionContextImplFactory.create(cache, functionObject.getId(), args,
-            resultSender, isReexecute);
-      }
-
-      ServerSideHandshake handshake = serverConnection.getHandshake();
-      int earlierClientReadTimeout = handshake.getClientReadTimeout();
-      handshake.setClientReadTimeout(functionTimeout);
-
-      try {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function on Server: {} with context: {}", serverConnection,
-              context);
-        }
-
-        LowMemoryException lowMemoryException = cache.getInternalResourceManager().getHeapMonitor()
-            .createLowMemoryIfNeeded(functionObject, cache.getMyId());
-        if (lowMemoryException != null) {
-          sendException(hasResult, clientMessage, lowMemoryException.getMessage(), serverConnection,
-              lowMemoryException);
-          return;
-        }
-
-        // cache is never null or the above invocations would have thrown NPE
-        DistributionManager dm = cache.getDistributionManager();
-        if (groups != null && groups.length > 0) {
-          executeFunctionOnGroups(function, args, groups, allMembers, functionObject, resultSender,
-              ignoreFailedMembers);
-        } else {
-          executeFunctionLocally(functionObject, context,
-              (ServerToClientFunctionResultSender65) resultSender, dm, stats);
-        }
-
-        if (!functionObject.hasResult()) {
-          writeReply(clientMessage, serverConnection);
-        }
-      } catch (FunctionException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new FunctionException(e);
-      } finally {
-        handshake.setClientReadTimeout(earlierClientReadTimeout);
-      }
-
-    } catch (IOException e) {
-      logger.warn("Exception on server while executing function: {}}", function, e);
-      String message = "Server could not send the reply";
-      sendException(hasResult, clientMessage, message, serverConnection, e);
-
-    } catch (InternalFunctionInvocationTargetException e) {
-      /*
-       * TRAC #44709: InternalFunctionInvocationTargetException should not be logged
-       * Fix for #44709: User should not be aware of InternalFunctionInvocationTargetException. No
-       * instance is giving useful information to user to take any corrective action hence logging
-       * this at fine level logging. May occur when:
-       * 1> When bucket is moved
-       * 2> In case of HA FunctionInvocationTargetException thrown. Since it is HA, function will
-       * be re-executed on right node
-       * 3> Multiple target nodes found for single hop operation
-       * 4> in case of HA member departed
-       */
-      if (logger.isDebugEnabled()) {
-        logger.debug("Exception on server while executing function: {}", function, e);
-      }
-      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
-
-    } catch (Exception e) {
-      logger.warn("Exception on server while executing function: {}", function, e);
-      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
-    }
-  }
-
-  protected boolean getIgnoreFailedMembers(Message msg) {
-    return false;
-  }
-
-  protected boolean getAllMembers(Message msg) {
-    return false;
-  }
-
-  protected void executeFunctionOnGroups(Object function, Object args, String[] groups,
-      boolean allMembers, Function functionObject, ServerToClientFunctionResultSender resultSender,
-      boolean ignoreFailedMembers) {
-    throw new InternalGemFireError();
-  }
-
-  protected String[] getGroups(Message msg) throws IOException, ClassNotFoundException {
-    return null;
-  }
-
-  private void executeFunctionLocally(final Function fn, final FunctionContext cx,
-      final ServerToClientFunctionResultSender65 sender, DistributionManager dm,
-      final FunctionStats stats) throws IOException {
-
-    if (fn.hasResult()) {
-      long startExecution = stats.startFunctionExecution(fn.hasResult());
-      try {
-        fn.execute(cx);
-        if (sender.isOkayToSendResult() && !sender.isLastResultReceived() && fn.hasResult()) {
-          throw new FunctionException(
-              String.format("The function, %s, did not send last result", fn.getId()));
-        }
-        stats.endFunctionExecution(startExecution, fn.hasResult());
-      } catch (Exception e) {
-        stats.endFunctionExecutionWithException(startExecution, fn.hasResult());
-        throw e;
-      }
-    } else {
-      /*
-       * if dm is null it mean cache is also null. Transactional function without cache cannot be
-       * executed.
-       */
-      TXStateProxy txState = TXManagerImpl.getCurrentTXState();
-      Runnable functionExecution = () -> {
-        InternalCache cache = null;
-        long startExecution = stats.startFunctionExecution(fn.hasResult());
-        try {
-          if (txState != null) {
-            cache = GemFireCacheImpl.getExisting("executing function");
-            cache.getTxManager().masqueradeAs(txState);
-            if (cache.getLogger().warningEnabled() && !asyncTxWarningIssued) {
-              asyncTxWarningIssued = true;
-              cache.getLogger().warning(
-                  "Function invoked within transactional context, but hasResults() is false; ordering of transactional operations cannot be guaranteed.  This message is only issued once by a server.");
-            }
-          }
-          fn.execute(cx);
-          stats.endFunctionExecution(startExecution, fn.hasResult());
-        } catch (InternalFunctionInvocationTargetException e) {
-          // TRAC #44709: InternalFunctionInvocationTargetException should not be logged
-          stats.endFunctionExecutionWithException(startExecution, fn.hasResult());
-          if (logger.isDebugEnabled()) {
-            logger.debug("Exception on server while executing function: {}", fn, e);
-          }
-        } catch (Exception e) {
-          stats.endFunctionExecutionWithException(startExecution, fn.hasResult());
-          logger.warn("Exception on server while executing function: {}", fn, e);
-        } finally {
-          if (txState != null && cache != null) {
-            cache.getTxManager().unmasquerade(txState);
-          }
-        }
-      };
-
-      if (dm == null) {
-        /*
-         * Executing the function in its own thread pool as FunctionExecution Thread pool of
-         * DistributionManager is not yet available.
-         */
-        execService.execute(functionExecution);
-      } else {
-        ClusterDistributionManager newDM = (ClusterDistributionManager) dm;
-        newDM.getExecutors().getFunctionExecutor().execute(functionExecution);
-      }
-    }
-  }
-
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection, Throwable e) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseException(msg, MessageType.EXCEPTION, serverConnection, e);
-    } else {
-      writeException(msg, e, false, serverConnection);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-  }
-
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR, message,
-          serverConnection);
-    } else {
-      writeErrorResponse(msg, MessageType.EXECUTE_FUNCTION_ERROR, message, serverConnection);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-  }
-
-  interface ServerToClientFunctionResultSender65Factory {
-    ServerToClientFunctionResultSender65 create(ChunkedMessage msg, int messageType,
-        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext);
-  }
-
-  interface FunctionContextImplFactory {
-    FunctionContextImpl create(Cache cache, String functionId, Object args,
-        ResultSender resultSender, boolean isPossibleDuplicate);
-  }
-
-  private static class DefaultServerToClientFunctionResultSender65Factory
-      implements ServerToClientFunctionResultSender65Factory {
-    @Override
-    public ServerToClientFunctionResultSender65 create(ChunkedMessage msg, int messageType,
-        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext) {
-      return new ServerToClientFunctionResultSender65(msg, messageType, sc, function, authzContext);
-    }
-  }
-
-  private static class DefaultFunctionContextImplFactory implements FunctionContextImplFactory {
-    @Override
-    public FunctionContextImpl create(Cache cache, String functionId, Object args,
-        ResultSender resultSender, boolean isPossibleDuplicat) {
-      return new FunctionContextImpl(cache, functionId, args, resultSender, isPossibleDuplicat);
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70.java
index 5ba98f7..e51ccad 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.tier.sockets.command;
 
+import static org.apache.geode.internal.cache.execute.ServerFunctionExecutor.DEFAULT_CLIENT_FUNCTION_TIMEOUT;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -21,67 +23,284 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
 
 import org.apache.geode.annotations.Immutable;
+import org.apache.geode.annotations.internal.MakeNotStatic;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.LowMemoryException;
 import org.apache.geode.cache.client.internal.ExecuteFunctionOp;
 import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionException;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.cache.TXStateProxy;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
+import org.apache.geode.internal.cache.execute.FunctionContextImpl;
 import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
 import org.apache.geode.internal.cache.execute.InternalFunctionExecutionServiceImpl;
+import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
+import org.apache.geode.internal.cache.execute.InternalFunctionService;
 import org.apache.geode.internal.cache.execute.MemberFunctionExecutor;
+import org.apache.geode.internal.cache.execute.MemberMappedArgument;
 import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
+import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
+import org.apache.geode.internal.cache.execute.metrics.FunctionStats;
+import org.apache.geode.internal.cache.execute.metrics.FunctionStatsManager;
 import org.apache.geode.internal.cache.tier.Command;
+import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.ServerSideHandshake;
+import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
+import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
 import org.apache.geode.internal.cache.tier.sockets.Message;
 import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
+import org.apache.geode.internal.security.AuthorizeRequest;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.logging.internal.executors.LoggingExecutors;
 
-public class ExecuteFunction70 extends ExecuteFunction66 {
+public class ExecuteFunction70 extends BaseCommand {
 
   @Immutable
   private static final ExecuteFunction70 singleton = new ExecuteFunction70();
+  @MakeNotStatic
+  private static volatile boolean asyncTxWarningIssued;
 
-  public static Command getCommand() {
-    return singleton;
-  }
+  @Immutable
+  private static final ExecutorService execService =
+      LoggingExecutors.newCachedThreadPool("Function Execution Thread-", true);
 
-  private ExecuteFunction70() {
-    // nothing
+  private final InternalFunctionExecutionService internalFunctionExecutionService;
+  private final ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory;
+  private final FunctionContextImplFactory functionContextImplFactory;
+
+  ExecuteFunction70() {
+    this(InternalFunctionService.getInternalFunctionExecutionService(),
+        new DefaultServerToClientFunctionResultSender65Factory(),
+        new DefaultFunctionContextImplFactory());
   }
 
   ExecuteFunction70(InternalFunctionExecutionService internalFunctionExecutionService,
       ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory,
       FunctionContextImplFactory functionContextImplFactory) {
-    super(internalFunctionExecutionService, serverToClientFunctionResultSender65Factory,
-        functionContextImplFactory);
+    this.internalFunctionExecutionService = internalFunctionExecutionService;
+    this.serverToClientFunctionResultSender65Factory = serverToClientFunctionResultSender65Factory;
+    this.functionContextImplFactory = functionContextImplFactory;
   }
 
   @Override
   public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
       final SecurityService securityService, long start) throws IOException {
-    super.cmdExecute(clientMessage, serverConnection, securityService, start);
-  }
+    Object function = null;
+    Object args;
+    MemberMappedArgument memberMappedArg = null;
+    String[] groups;
+    byte hasResult = 0;
+    byte functionState;
+    boolean isReexecute = false;
+    boolean allMembers;
+    boolean ignoreFailedMembers;
+    int functionTimeout = DEFAULT_CLIENT_FUNCTION_TIMEOUT;
 
-  @Override
-  protected String[] getGroups(Message msg) throws IOException, ClassNotFoundException {
-    String[] groups = null;
-    Part messagePart = msg.getPart(4);
-    if (messagePart != null) {
-      groups = (String[]) messagePart.getObject();
+    try {
+      byte[] bytes = clientMessage.getPart(0).getSerializedForm();
+      functionState = bytes[0];
+      if (bytes.length >= 5
+          && serverConnection.getClientVersion().ordinal() >= KnownVersion.GFE_8009.ordinal()) {
+        functionTimeout = Part.decodeInt(bytes, 1);
+      }
+
+      if (functionState == AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE_REEXECUTE) {
+        functionState = AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE;
+        isReexecute = true;
+      } else if (functionState == AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE_REEXECUTE) {
+        functionState = AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE;
+        isReexecute = true;
+      }
+
+      if (functionState != 1) {
+        hasResult = (byte) ((functionState & 2) - 1);
+      } else {
+        hasResult = functionState;
+      }
+      if (hasResult == 1) {
+        serverConnection.setAsTrue(REQUIRES_RESPONSE);
+        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
+      }
+      function = clientMessage.getPart(1).getStringOrObject();
+      args = clientMessage.getPart(2).getObject();
+
+      Part part = clientMessage.getPart(3);
+      if (part != null) {
+        memberMappedArg = (MemberMappedArgument) part.getObject();
+      }
+
+      groups = getGroups(clientMessage);
+      allMembers = getAllMembers(clientMessage);
+      ignoreFailedMembers = getIgnoreFailedMembers(clientMessage);
+    } catch (ClassNotFoundException e) {
+      logger.warn("Exception on server while executing function: {}", function, e);
+      if (hasResult == 1) {
+        writeChunkedException(clientMessage, e, serverConnection);
+      } else {
+        writeException(clientMessage, e, false, serverConnection);
+      }
+      serverConnection.setAsTrue(RESPONDED);
+      return;
     }
-    return groups;
+
+    if (function == null) {
+      String message = "The input function for the execute function request is null";
+      logger.warn("{} : {}", serverConnection.getName(), message);
+      sendError(hasResult, clientMessage, message, serverConnection);
+      return;
+    }
+
+    // Execute function on the cache
+    try {
+      Function<?> functionObject;
+      if (function instanceof String) {
+        functionObject = internalFunctionExecutionService.getFunction((String) function);
+        if (functionObject == null) {
+          String message = String
+              .format("Function named %s is not registered to FunctionService", function);
+          logger.warn("{}: {}", serverConnection.getName(), message);
+          sendError(hasResult, clientMessage, message, serverConnection);
+          return;
+        } else {
+          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(),
+              functionObject.hasResult(), functionObject.optimizeForWrite());
+          if (logger.isDebugEnabled()) {
+            logger.debug("Function State on server side: {} on client: {}",
+                functionStateOnServerSide, functionState);
+          }
+          if (functionStateOnServerSide != functionState) {
+            String message = String
+                .format("Function attributes at client and server don't match for %s", function);
+            logger.warn("{}: {}", serverConnection.getName(), message);
+            sendError(hasResult, clientMessage, message, serverConnection);
+            return;
+          }
+        }
+      } else {
+        functionObject = (Function) function;
+      }
+
+      FunctionStats stats = FunctionStatsManager.getFunctionStats(functionObject.getId());
+
+      // check if the caller is authorized to do this operation on server
+      functionObject.getRequiredPermissions(null, args).forEach(securityService::authorize);
+
+      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionObject.getId(), null, null,
+            args, functionObject.optimizeForWrite());
+      }
+
+      ChunkedMessage chunkedMessage = serverConnection.getFunctionResponseMessage();
+      chunkedMessage.setTransactionId(clientMessage.getTransactionId());
+      ServerToClientFunctionResultSender resultSender =
+          serverToClientFunctionResultSender65Factory.create(chunkedMessage,
+              MessageType.EXECUTE_FUNCTION_RESULT,
+              serverConnection, functionObject, executeContext);
+
+      FunctionContext context;
+      InternalCache cache = serverConnection.getCache();
+      InternalDistributedMember localVM =
+          (InternalDistributedMember) cache.getDistributedSystem().getDistributedMember();
+
+      if (memberMappedArg != null) {
+        context = functionContextImplFactory.create(cache, functionObject.getId(),
+            memberMappedArg.getArgumentsForMember(localVM.getId()), resultSender, isReexecute);
+      } else {
+        context = functionContextImplFactory.create(cache, functionObject.getId(), args,
+            resultSender, isReexecute);
+      }
+
+      ServerSideHandshake handshake = serverConnection.getHandshake();
+      int earlierClientReadTimeout = handshake.getClientReadTimeout();
+      handshake.setClientReadTimeout(functionTimeout);
+
+      try {
+        if (logger.isDebugEnabled()) {
+          logger.debug("Executing Function on Server: {} with context: {}", serverConnection,
+              context);
+        }
+
+        LowMemoryException lowMemoryException = cache.getInternalResourceManager().getHeapMonitor()
+            .createLowMemoryIfNeeded(functionObject, cache.getMyId());
+        if (lowMemoryException != null) {
+          sendException(hasResult, clientMessage, lowMemoryException.getMessage(), serverConnection,
+              lowMemoryException);
+          return;
+        }
+
+        // cache is never null or the above invocations would have thrown NPE
+        DistributionManager dm = cache.getDistributionManager();
+        if (groups != null && groups.length > 0) {
+          executeFunctionOnGroups(function, args, groups, allMembers, functionObject, resultSender,
+              ignoreFailedMembers);
+        } else {
+          executeFunctionLocally(functionObject, context,
+              (ServerToClientFunctionResultSender65) resultSender, dm, stats);
+        }
+
+        if (!functionObject.hasResult()) {
+          writeReply(clientMessage, serverConnection);
+        }
+      } catch (FunctionException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new FunctionException(e);
+      } finally {
+        handshake.setClientReadTimeout(earlierClientReadTimeout);
+      }
+
+    } catch (IOException e) {
+      logger.warn("Exception on server while executing function: {}}", function, e);
+      String message = "Server could not send the reply";
+      sendException(hasResult, clientMessage, message, serverConnection, e);
+
+    } catch (InternalFunctionInvocationTargetException e) {
+      /*
+       * TRAC #44709: InternalFunctionInvocationTargetException should not be logged
+       * Fix for #44709: User should not be aware of InternalFunctionInvocationTargetException. No
+       * instance is giving useful information to user to take any corrective action hence logging
+       * this at fine level logging. May occur when:
+       * 1> When bucket is moved
+       * 2> In case of HA FunctionInvocationTargetException thrown. Since it is HA, function will
+       * be re-executed on right node
+       * 3> Multiple target nodes found for single hop operation
+       * 4> in case of HA member departed
+       */
+      if (logger.isDebugEnabled()) {
+        logger.debug("Exception on server while executing function: {}", function, e);
+      }
+      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
+
+    } catch (Exception e) {
+      logger.warn("Exception on server while executing function: {}", function, e);
+      sendException(hasResult, clientMessage, e.getMessage(), serverConnection, e);
+    }
   }
 
-  @Override
   protected boolean getIgnoreFailedMembers(Message msg) {
     return isFlagSet(msg, ExecuteFunctionOp.IGNORE_FAILED_MEMBERS_INDEX);
   }
 
-  @Override
   protected boolean getAllMembers(Message msg) {
     return isFlagSet(msg, ExecuteFunctionOp.ALL_MEMBERS_INDEX);
   }
@@ -100,7 +319,6 @@
     return isSet;
   }
 
-  @Override
   protected void executeFunctionOnGroups(Object function, Object args, String[] groups,
       boolean allMembers, Function functionObject, ServerToClientFunctionResultSender resultSender,
       boolean ignoreFailedMembers) {
@@ -154,4 +372,132 @@
       execution.execute(functionObject).getResult();
     }
   }
+
+  public static Command getCommand() {
+    return singleton;
+  }
+
+  protected String[] getGroups(Message msg) throws IOException, ClassNotFoundException {
+    String[] groups = null;
+    Part messagePart = msg.getPart(4);
+    if (messagePart != null) {
+      groups = (String[]) messagePart.getObject();
+    }
+    return groups;
+  }
+
+  private void executeFunctionLocally(final Function fn, final FunctionContext cx,
+      final ServerToClientFunctionResultSender65 sender, DistributionManager dm,
+      final FunctionStats stats) throws IOException {
+
+    if (fn.hasResult()) {
+      long startExecution = stats.startFunctionExecution(fn.hasResult());
+      try {
+        fn.execute(cx);
+        if (sender.isOkayToSendResult() && !sender.isLastResultReceived() && fn.hasResult()) {
+          throw new FunctionException(
+              String.format("The function, %s, did not send last result", fn.getId()));
+        }
+        stats.endFunctionExecution(startExecution, fn.hasResult());
+      } catch (Exception e) {
+        stats.endFunctionExecutionWithException(startExecution, fn.hasResult());
+        throw e;
+      }
+    } else {
+      /*
+       * if dm is null it mean cache is also null. Transactional function without cache cannot be
+       * executed.
+       */
+      TXStateProxy txState = TXManagerImpl.getCurrentTXState();
+      Runnable functionExecution = () -> {
+        InternalCache cache = null;
+        long startExecution = stats.startFunctionExecution(fn.hasResult());
+        try {
+          if (txState != null) {
+            cache = GemFireCacheImpl.getExisting("executing function");
+            cache.getTxManager().masqueradeAs(txState);
+            if (cache.getLogger().warningEnabled() && !asyncTxWarningIssued) {
+              asyncTxWarningIssued = true;
+              cache.getLogger().warning(
+                  "Function invoked within transactional context, but hasResults() is false; ordering of transactional operations cannot be guaranteed.  This message is only issued once by a server.");
+            }
+          }
+          fn.execute(cx);
+          stats.endFunctionExecution(startExecution, fn.hasResult());
+        } catch (InternalFunctionInvocationTargetException e) {
+          // TRAC #44709: InternalFunctionInvocationTargetException should not be logged
+          stats.endFunctionExecutionWithException(startExecution, fn.hasResult());
+          if (logger.isDebugEnabled()) {
+            logger.debug("Exception on server while executing function: {}", fn, e);
+          }
+        } catch (Exception e) {
+          stats.endFunctionExecutionWithException(startExecution, fn.hasResult());
+          logger.warn("Exception on server while executing function: {}", fn, e);
+        } finally {
+          if (txState != null && cache != null) {
+            cache.getTxManager().unmasquerade(txState);
+          }
+        }
+      };
+
+      if (dm == null) {
+        /*
+         * Executing the function in its own thread pool as FunctionExecution Thread pool of
+         * DistributionManager is not yet available.
+         */
+        execService.execute(functionExecution);
+      } else {
+        ClusterDistributionManager newDM = (ClusterDistributionManager) dm;
+        newDM.getExecutors().getFunctionExecutor().execute(functionExecution);
+      }
+    }
+  }
+
+  private void sendException(byte hasResult, Message msg, String message,
+      ServerConnection serverConnection, Throwable e) throws IOException {
+    if (hasResult == 1) {
+      writeFunctionResponseException(msg, MessageType.EXCEPTION, serverConnection, e);
+    } else {
+      writeException(msg, e, false, serverConnection);
+    }
+    serverConnection.setAsTrue(RESPONDED);
+  }
+
+  private void sendError(byte hasResult, Message msg, String message,
+      ServerConnection serverConnection) throws IOException {
+    if (hasResult == 1) {
+      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR, message,
+          serverConnection);
+    } else {
+      writeErrorResponse(msg, MessageType.EXECUTE_FUNCTION_ERROR, message, serverConnection);
+    }
+    serverConnection.setAsTrue(RESPONDED);
+  }
+
+  interface ServerToClientFunctionResultSender65Factory {
+    ServerToClientFunctionResultSender65 create(ChunkedMessage msg, int messageType,
+        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext);
+  }
+
+  interface FunctionContextImplFactory {
+    FunctionContextImpl create(Cache cache, String functionId, Object args,
+        ResultSender resultSender, boolean isPossibleDuplicate);
+  }
+
+  private static class DefaultServerToClientFunctionResultSender65Factory
+      implements ServerToClientFunctionResultSender65Factory {
+    @Override
+    public ServerToClientFunctionResultSender65 create(ChunkedMessage msg, int messageType,
+        ServerConnection sc, Function function, ExecuteFunctionOperationContext authzContext) {
+      return new ServerToClientFunctionResultSender65(msg, messageType, sc, function, authzContext);
+    }
+  }
+
+  private static class DefaultFunctionContextImplFactory implements FunctionContextImplFactory {
+    @Override
+    public FunctionContextImpl create(Cache cache, String functionId, Object args,
+        ResultSender resultSender, boolean isPossibleDuplicat) {
+      return new FunctionContextImpl(cache, functionId, args, resultSender, isPossibleDuplicat);
+    }
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
deleted file mode 100755
index 8f98e35..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.internal.cache.DistributedRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.execute.AbstractExecution;
-import org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutor;
-import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
-import org.apache.geode.internal.cache.execute.MemberMappedArgument;
-import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionExecutor;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- * This is the base command which reads the parts for the MessageType.EXECUTE_REGION_FUNCTION and
- * executes the given function on server region.<br>
- * If the hasResult byte is 1, then this command send back the result after the execution to the
- * client else do not send the reply back to the client
- *
- * @since GemFire 5.8LA
- */
-public class ExecuteRegionFunction extends BaseCommand {
-
-  @Immutable
-  private static final ExecuteRegionFunction singleton = new ExecuteRegionFunction();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private ExecuteRegionFunction() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    String regionName = null;
-    Object function = null;
-    Object args = null;
-    MemberMappedArgument memberMappedArg = null;
-    Set filter = null;
-    byte hasResult = 0;
-    int filterSize = 0, partNumber = 0;
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    try {
-      hasResult = clientMessage.getPart(0).getSerializedForm()[0];
-      if (hasResult == 1) {
-        serverConnection.setAsTrue(REQUIRES_RESPONSE);
-        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-      }
-      regionName = clientMessage.getPart(1).getCachedString();
-      function = clientMessage.getPart(2).getStringOrObject();
-      args = clientMessage.getPart(3).getObject();
-      Part part = clientMessage.getPart(4);
-      if (part != null) {
-        Object obj = part.getObject();
-        if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument) obj;
-        }
-      }
-      filterSize = clientMessage.getPart(5).getInt();
-      if (filterSize != 0) {
-        filter = new HashSet();
-        partNumber = 6;
-        for (int i = 0; i < filterSize; i++) {
-          filter.add(clientMessage.getPart(partNumber + i).getStringOrObject());
-        }
-      }
-
-    } catch (ClassNotFoundException exception) {
-      logger.warn(String.format("Exception on server while executing function : %s",
-          function),
-          exception);
-      if (hasResult == 1) {
-        writeChunkedException(clientMessage, exception, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    if (function == null || regionName == null) {
-      String message = null;
-      if (function == null) {
-        message =
-            String.format("The input %s for the execute function request is null",
-                "function");
-      }
-      if (regionName == null) {
-        message =
-            String.format("The input %s for the execute function request is null",
-                "region");
-      }
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    Region region = crHelper.getRegion(regionName);
-    if (region == null) {
-      String message =
-          String.format("The region named %s was not found during execute Function request.",
-              regionName);
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    ServerSideHandshake handshake = serverConnection.getHandshake();
-    int earlierClientReadTimeout = handshake.getClientReadTimeout();
-    handshake.setClientReadTimeout(0);
-    ServerToClientFunctionResultSender resultSender = null;
-    Function<?> functionObject = null;
-    try {
-      if (function instanceof String) {
-        functionObject = FunctionService.getFunction((String) function);
-        if (functionObject == null) {
-          String message =
-              String.format("The function, %s, has not been registered",
-                  function);
-          logger.warn("{}: {}", serverConnection.getName(), message);
-          sendError(hasResult, clientMessage, message, serverConnection);
-          return;
-        }
-      } else {
-        functionObject = (Function) function;
-      }
-
-      // check if the caller is authorized to do this operation on server
-      functionObject.getRequiredPermissions(regionName, args).forEach(securityService::authorize);
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      final String functionName = functionObject.getId();
-      final String regionPath = region.getFullPath();
-      ExecuteFunctionOperationContext executeContext = null;
-      if (authzRequest != null) {
-        executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter,
-            args, functionObject.optimizeForWrite());
-      }
-
-      // Construct execution
-      AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
-      ChunkedMessage m = serverConnection.getFunctionResponseMessage();
-      m.setTransactionId(clientMessage.getTransactionId());
-      resultSender =
-          new ServerToClientFunctionResultSender(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT,
-              serverConnection, functionObject, executeContext);
-
-      if (execution instanceof PartitionedRegionFunctionExecutor) {
-        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter, args,
-            memberMappedArg, resultSender, null, false);
-      } else {
-        execution = new DistributedRegionFunctionExecutor((DistributedRegion) region, filter, args,
-            memberMappedArg, resultSender);
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug("Executing Function: {} on Server: {} with Execution: {}",
-            functionObject.getId(), serverConnection, execution);
-      }
-      if (hasResult == 1) {
-        if (function instanceof String) {
-          execution.execute((String) function).getResult();
-        } else {
-          execution.execute(functionObject).getResult();
-        }
-      } else {
-        if (function instanceof String) {
-          execution.execute((String) function);
-        } else {
-          execution.execute(functionObject);
-        }
-      }
-    } catch (IOException ioe) {
-      logger.warn(String.format("Exception on server while executing function : %s",
-          function),
-          ioe);
-      final String message = "Server could not send the reply";
-      sendException(hasResult, clientMessage, message, serverConnection, ioe);
-    } catch (InternalFunctionInvocationTargetException internalfunctionException) {
-      // Fix for #44709: User should not be aware of
-      // InternalFunctionInvocationTargetException. No instance of
-      // InternalFunctionInvocationTargetException is giving useful
-      // information to user to take any corrective action hence logging
-      // this at fine level logging
-      // 1> When bucket is moved
-      // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-      // it is HA, fucntion will be reexecuted on right node
-      // 3> Multiple target nodes found for single hop operation
-      // 4> in case of HA member departed
-      if (logger.isDebugEnabled()) {
-        logger.debug(String.format("Exception on server while executing function: %s",
-            new Object[] {function}),
-            internalfunctionException);
-      }
-      final String message = internalfunctionException.getMessage();
-      sendException(hasResult, clientMessage, message, serverConnection, internalfunctionException);
-    } catch (FunctionException fe) {
-      logger.warn(String.format("Exception on server while executing function : %s",
-          function),
-          fe);
-      String message = fe.getMessage();
-
-      sendException(hasResult, clientMessage, message, serverConnection, fe);
-    } catch (Exception e) {
-      logger.warn(String.format("Exception on server while executing function : %s",
-          function),
-          e);
-      String message = e.getMessage();
-      sendException(hasResult, clientMessage, message, serverConnection, e);
-    } finally {
-      handshake.setClientReadTimeout(earlierClientReadTimeout);
-    }
-  }
-
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection, Throwable e) throws IOException {
-    synchronized (msg) {
-      if (hasResult == 1) {
-        writeFunctionResponseException(msg, MessageType.EXCEPTION, serverConnection, e);
-        serverConnection.setAsTrue(RESPONDED);
-      }
-    }
-  }
-
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection) throws IOException {
-    synchronized (msg) {
-      if (hasResult == 1) {
-        writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message,
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      }
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction61.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction61.java
deleted file mode 100755
index 36417dd..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction61.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.FunctionInvocationTargetException;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.internal.cache.DistributedRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.execute.AbstractExecution;
-import org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutor;
-import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
-import org.apache.geode.internal.cache.execute.MemberMappedArgument;
-import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionExecutor;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- *
- *
- * @since GemFire 6.1
- */
-public class ExecuteRegionFunction61 extends BaseCommand {
-
-  @Immutable
-  private static final ExecuteRegionFunction61 singleton = new ExecuteRegionFunction61();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private ExecuteRegionFunction61() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    String regionName = null;
-    Object function = null;
-    Object args = null;
-    MemberMappedArgument memberMappedArg = null;
-    byte isReExecute = 0;
-    Set filter = null;
-    byte hasResult = 0;
-    int removedNodesSize = 0;
-    Set removedNodesSet = null;
-    int filterSize = 0, partNumber = 0;
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    try {
-      hasResult = clientMessage.getPart(0).getSerializedForm()[0];
-      if (hasResult == 1) {
-        serverConnection.setAsTrue(REQUIRES_RESPONSE);
-        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-      }
-      regionName = clientMessage.getPart(1).getCachedString();
-      function = clientMessage.getPart(2).getStringOrObject();
-      args = clientMessage.getPart(3).getObject();
-      Part part = clientMessage.getPart(4);
-      if (part != null) {
-        Object obj = part.getObject();
-        if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument) obj;
-        }
-      }
-      isReExecute = clientMessage.getPart(5).getSerializedForm()[0];
-      filterSize = clientMessage.getPart(6).getInt();
-      if (filterSize != 0) {
-        filter = new HashSet();
-        partNumber = 7;
-        for (int i = 0; i < filterSize; i++) {
-          filter.add(clientMessage.getPart(partNumber + i).getStringOrObject());
-        }
-      }
-
-      partNumber = 7 + filterSize;
-      removedNodesSize = clientMessage.getPart(partNumber).getInt();
-
-      if (removedNodesSize != 0) {
-        removedNodesSet = new HashSet();
-        partNumber = partNumber + 1;
-
-        for (int i = 0; i < removedNodesSize; i++) {
-          removedNodesSet.add(clientMessage.getPart(partNumber + i).getStringOrObject());
-        }
-      }
-
-    } catch (ClassNotFoundException exception) {
-      logger.warn(String.format("Exception on server while executing function : %s",
-          function),
-          exception);
-      if (hasResult == 1) {
-        writeChunkedException(clientMessage, exception, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    if (function == null || regionName == null) {
-      String message = null;
-      if (function == null) {
-        message =
-            String.format("The input %s for the execute function request is null",
-                "function");
-      }
-      if (regionName == null) {
-        message =
-            String.format("The input %s for the execute function request is null",
-                "region");
-      }
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    } else {
-      Region region = crHelper.getRegion(regionName);
-      if (region == null) {
-        String message =
-            String.format("The region named %s was not found during execute Function request.",
-                regionName);
-        logger.warn("{}: {}", serverConnection.getName(), message);
-        sendError(hasResult, clientMessage, message, serverConnection);
-        return;
-      }
-      ServerSideHandshake handshake = serverConnection.getHandshake();
-      int earlierClientReadTimeout = handshake.getClientReadTimeout();
-      handshake.setClientReadTimeout(0);
-      ServerToClientFunctionResultSender resultSender = null;
-      Function<?> functionObject = null;
-      try {
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String) function);
-          if (functionObject == null) {
-            String message =
-                String.format("The function, %s, has not been registered",
-                    function);
-            logger.warn("{}: {}", serverConnection.getName(), message);
-            sendError(hasResult, clientMessage, message, serverConnection);
-            return;
-          }
-        } else {
-          functionObject = (Function) function;
-        }
-        // check if the caller is authorized to do this operation on server
-        functionObject.getRequiredPermissions(regionName, args).forEach(securityService::authorize);
-
-        AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-        final String functionName = functionObject.getId();
-        final String regionPath = region.getFullPath();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter,
-              args, functionObject.optimizeForWrite());
-        }
-
-        // Construct execution
-        AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
-        ChunkedMessage m = serverConnection.getFunctionResponseMessage();
-        m.setTransactionId(clientMessage.getTransactionId());
-        resultSender =
-            new ServerToClientFunctionResultSender(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT,
-                serverConnection, functionObject, executeContext);
-
-
-        if (execution instanceof PartitionedRegionFunctionExecutor) {
-          execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter,
-              args, memberMappedArg, resultSender, removedNodesSet, false);
-        } else {
-          execution = new DistributedRegionFunctionExecutor((DistributedRegion) region, filter,
-              args, memberMappedArg, resultSender);
-        }
-        if (isReExecute == 1) {
-          execution.setIsReExecute();
-        }
-
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function: {} on Server: {} with Execution: {}",
-              functionObject.getId(), serverConnection, execution);
-        }
-        if (hasResult == 1) {
-          if (function instanceof String) {
-            execution.execute((String) function).getResult();
-          } else {
-            execution.execute(functionObject).getResult();
-          }
-        } else {
-          if (function instanceof String) {
-            execution.execute((String) function);
-          } else {
-            execution.execute(functionObject);
-          }
-        }
-      } catch (IOException ioe) {
-        logger.warn(String.format("Exception on server while executing function : %s",
-            function),
-            ioe);
-        final String message = "Server could not send the reply";
-        sendException(hasResult, clientMessage, message, serverConnection, ioe);
-      } catch (FunctionException fe) {
-        String message = fe.getMessage();
-
-        if (fe.getCause() instanceof FunctionInvocationTargetException) {
-          if (fe.getCause() instanceof InternalFunctionInvocationTargetException) {
-            // Fix for #44709: User should not be aware of
-            // InternalFunctionInvocationTargetException. No instance of
-            // InternalFunctionInvocationTargetException is giving useful
-            // information to user to take any corrective action hence logging
-            // this at fine level logging
-            // 1> When bucket is moved
-            // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-            // it is HA, fucntion will be reexecuted on right node
-            // 3> Multiple target nodes found for single hop operation
-            // 4> in case of HA member departed
-            if (logger.isDebugEnabled()) {
-              logger.debug(String.format("Exception on server while executing function: %s",
-                  new Object[] {function}),
-                  fe);
-            }
-          } else if (functionObject.isHA()) {
-            logger.warn("Exception on server while executing function : " +
-                function + " :" + message);
-          } else {
-            logger.warn(String.format("Exception on server while executing function : %s",
-                function),
-                fe);
-          }
-          resultSender.setException(fe);
-        } else {
-          if (setLastResultReceived(resultSender)) {
-            logger.warn(String.format("Exception on server while executing function : %s",
-                function),
-                fe);
-            sendException(hasResult, clientMessage, message, serverConnection, fe);
-          }
-        }
-
-      } catch (Exception e) {
-        if (setLastResultReceived(resultSender)) {
-          logger.warn(String.format("Exception on server while executing function : %s",
-              function),
-              e);
-          String message = e.getMessage();
-          sendException(hasResult, clientMessage, message, serverConnection, e);
-        }
-      }
-
-      finally {
-        handshake.setClientReadTimeout(earlierClientReadTimeout);
-      }
-    }
-  }
-
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection, Throwable e) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseException(msg, MessageType.EXCEPTION, serverConnection, e);
-      serverConnection.setAsTrue(RESPONDED);
-    }
-  }
-
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection) throws IOException {
-    if (hasResult == 1) {
-      writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
deleted file mode 100755
index aa1c8f4..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
+++ /dev/null
@@ -1,402 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.FunctionInvocationTargetException;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.internal.cache.DistributedRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.execute.AbstractExecution;
-import org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutor;
-import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
-import org.apache.geode.internal.cache.execute.MemberMappedArgument;
-import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionExecutor;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- * @since GemFire 6.5
- */
-public class ExecuteRegionFunction65 extends BaseCommand {
-
-  @Immutable
-  private static final ExecuteRegionFunction65 singleton = new ExecuteRegionFunction65();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private ExecuteRegionFunction65() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    String regionName = null;
-    Object function = null;
-    Object args = null;
-    MemberMappedArgument memberMappedArg = null;
-    byte isReExecute = 0;
-    Set<Object> filter = null;
-    byte hasResult = 0;
-    int removedNodesSize = 0;
-    Set<Object> removedNodesSet = null;
-    int filterSize = 0, partNumber = 0;
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    byte functionState = 0;
-    try {
-      functionState = clientMessage.getPart(0).getSerializedForm()[0];
-      if (functionState != 1) {
-        hasResult = (byte) ((functionState & 2) - 1);
-      } else {
-        hasResult = functionState;
-      }
-      if (hasResult == 1) {
-        serverConnection.setAsTrue(REQUIRES_RESPONSE);
-        serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-      }
-      regionName = clientMessage.getPart(1).getCachedString();
-      function = clientMessage.getPart(2).getStringOrObject();
-      args = clientMessage.getPart(3).getObject();
-      Part part = clientMessage.getPart(4);
-      if (part != null) {
-        Object obj = part.getObject();
-        if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument) obj;
-        }
-      }
-      isReExecute = clientMessage.getPart(5).getSerializedForm()[0];
-      filterSize = clientMessage.getPart(6).getInt();
-      if (filterSize != 0) {
-        filter = new HashSet<Object>();
-        partNumber = 7;
-        for (int i = 0; i < filterSize; i++) {
-          filter.add(clientMessage.getPart(partNumber + i).getStringOrObject());
-        }
-      }
-
-      partNumber = 7 + filterSize;
-      removedNodesSize = clientMessage.getPart(partNumber).getInt();
-
-      if (removedNodesSize != 0) {
-        removedNodesSet = new HashSet<Object>();
-        partNumber = partNumber + 1;
-
-        for (int i = 0; i < removedNodesSize; i++) {
-          removedNodesSet.add(clientMessage.getPart(partNumber + i).getStringOrObject());
-        }
-      }
-
-    } catch (ClassNotFoundException exception) {
-      logger.warn(
-          String.format("Exception on server while executing function : %s",
-              function),
-          exception);
-      if (hasResult == 1) {
-        writeChunkedException(clientMessage, exception, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    if (function == null || regionName == null) {
-      String message = null;
-      if (function == null) {
-        message =
-            String.format("The input %s for the execute function request is null",
-                "function");
-      }
-      if (regionName == null) {
-        message =
-            String.format("The input %s for the execute function request is null",
-                "region");
-      }
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    Region region = crHelper.getRegion(regionName);
-    if (region == null) {
-      String message =
-          String.format("The region named %s was not found during execute Function request.",
-              regionName);
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      sendError(hasResult, clientMessage, message, serverConnection);
-      return;
-    }
-
-    ServerSideHandshake handshake = serverConnection.getHandshake();
-    int earlierClientReadTimeout = handshake.getClientReadTimeout();
-    handshake.setClientReadTimeout(0);
-    ServerToClientFunctionResultSender resultSender = null;
-    Function<?> functionObject = null;
-    try {
-      if (function instanceof String) {
-        functionObject = FunctionService.getFunction((String) function);
-        if (functionObject == null) {
-          String message =
-              String.format("The function, %s, has not been registered",
-                  function);
-          logger.warn("{}: {}", serverConnection.getName(), message);
-          sendError(hasResult, clientMessage, message, serverConnection);
-          return;
-        } else {
-          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(),
-              functionObject.hasResult(), functionObject.optimizeForWrite());
-          if (logger.isDebugEnabled()) {
-            logger.debug("Function State on server side: {} on client: {}",
-                functionStateOnServerSide, functionState);
-          }
-          if (functionStateOnServerSide != functionState) {
-            String message =
-                "Function attributes at client and server don't match";
-            logger.warn("{}: {}", serverConnection.getName(), message);
-            sendError(hasResult, clientMessage, message, serverConnection);
-            return;
-          }
-        }
-      } else {
-        functionObject = (Function) function;
-      }
-
-      // check if the caller is authorized to do this operation on server
-      functionObject.getRequiredPermissions(regionName, args).forEach(securityService::authorize);
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      final String functionName = functionObject.getId();
-      final String regionPath = region.getFullPath();
-      ExecuteFunctionOperationContext executeContext = null;
-      if (authzRequest != null) {
-        executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter,
-            args, functionObject.optimizeForWrite());
-      }
-
-      // Construct execution
-      AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
-      ChunkedMessage m = serverConnection.getFunctionResponseMessage();
-      m.setTransactionId(clientMessage.getTransactionId());
-      resultSender =
-          new ServerToClientFunctionResultSender65(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT,
-              serverConnection, functionObject, executeContext);
-
-
-      if (execution instanceof PartitionedRegionFunctionExecutor) {
-        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter, args,
-            memberMappedArg, resultSender, removedNodesSet, false);
-      } else {
-        execution = new DistributedRegionFunctionExecutor((DistributedRegion) region, filter, args,
-            memberMappedArg, resultSender);
-      }
-      if (isReExecute == 1) {
-        execution = execution.setIsReExecute();
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Executing Function: {} on Server: {} with Execution: {} functionState={} reexecute={} hasResult={}",
-            functionObject.getId(), serverConnection, execution, functionState, isReExecute,
-            hasResult);
-      }
-      if (hasResult == 1) {
-        if (function instanceof String) {
-          switch (functionState) {
-            case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-              execution.execute((String) function).getResult();
-              break;
-            case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-              execution.execute((String) function).getResult();
-              break;
-            case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
-              execution.execute((String) function).getResult();
-              break;
-            case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
-              execution.execute((String) function).getResult();
-              break;
-          }
-        } else {
-          execution.execute(functionObject).getResult();
-        }
-      } else {
-        if (function instanceof String) {
-          switch (functionState) {
-            case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
-              execution.execute((String) function);
-              break;
-            case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
-              execution.execute((String) function);
-              break;
-          }
-        } else {
-          execution.execute(functionObject);
-        }
-      }
-    } catch (IOException ioe) {
-      logger.warn(
-          String.format("Exception on server while executing function : %s",
-              function),
-          ioe);
-      final String message = "Server could not send the reply";
-      sendException(hasResult, clientMessage, message, serverConnection, ioe);
-    } catch (FunctionException fe) {
-      String message = fe.getMessage();
-
-      if (fe.getCause() instanceof FunctionInvocationTargetException) {
-        if (fe.getCause() instanceof InternalFunctionInvocationTargetException) {
-          // Fix for #44709: User should not be aware of
-          // InternalFunctionInvocationTargetException. No instance of
-          // InternalFunctionInvocationTargetException is giving useful
-          // information to user to take any corrective action hence logging
-          // this at fine level logging
-          // 1> When bucket is moved
-          // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-          // it is HA, fucntion will be reexecuted on right node
-          // 3> Multiple target nodes found for single hop operation
-          // 4> in case of HA member departed
-          if (logger.isDebugEnabled()) {
-            logger.debug(
-                String.format("Exception on server while executing function: %s",
-                    new Object[] {function}),
-                fe);
-          }
-        } else if (functionObject.isHA()) {
-          logger.warn("Exception on server while executing function : {}",
-              function + " :" + message);
-        } else {
-          logger.warn(
-              String.format("Exception on server while executing function : %s",
-                  function),
-              fe);
-        }
-
-        resultSender.setException(fe);
-      } else {
-        if (setLastResultReceived(resultSender)) {
-          logger.warn(
-              String.format("Exception on server while executing function : %s",
-                  function),
-              fe);
-          sendException(hasResult, clientMessage, message, serverConnection, fe);
-        }
-      }
-
-    } catch (Exception e) {
-      if (setLastResultReceived(resultSender)) {
-        logger.warn(
-            String.format("Exception on server while executing function : %s",
-                function),
-            e);
-        String message = e.getMessage();
-        sendException(hasResult, clientMessage, message, serverConnection, e);
-      }
-    } finally {
-      handshake.setClientReadTimeout(earlierClientReadTimeout);
-    }
-  }
-
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection, Throwable e) throws IOException {
-    synchronized (msg) {
-      if (hasResult == 1) {
-        writeFunctionResponseException(msg, MessageType.EXCEPTION, message, serverConnection, e);
-        serverConnection.setAsTrue(RESPONDED);
-      }
-    }
-  }
-
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection serverConnection) throws IOException {
-    synchronized (msg) {
-      if (hasResult == 1) {
-        writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message,
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      }
-    }
-  }
-
-  protected static void writeFunctionResponseException(Message origMsg, int messageType,
-      String message, ServerConnection serverConnection, Throwable e) throws IOException {
-    ChunkedMessage functionResponseMsg = serverConnection.getFunctionResponseMessage();
-    ChunkedMessage chunkedResponseMsg = serverConnection.getChunkedResponseMessage();
-    int numParts = 0;
-    if (functionResponseMsg.headerHasBeenSent()) {
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
-        functionResponseMsg.setNumberOfParts(3);
-        functionResponseMsg.addObjPart(e);
-        functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe =
-            (InternalFunctionInvocationTargetException) e.getCause();
-        functionResponseMsg.addObjPart(fe.getFailedNodeSet());
-        numParts = 3;
-      } else {
-        functionResponseMsg.setNumberOfParts(2);
-        functionResponseMsg.addObjPart(e);
-        functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        numParts = 2;
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Sending exception chunk while reply in progress: ",
-            serverConnection.getName(), e);
-      }
-      functionResponseMsg.setServerConnection(serverConnection);
-      functionResponseMsg.setLastChunkAndNumParts(true, numParts);
-      // functionResponseMsg.setLastChunk(true);
-      functionResponseMsg.sendChunk(serverConnection);
-    } else {
-      chunkedResponseMsg.setMessageType(messageType);
-      chunkedResponseMsg.setTransactionId(origMsg.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
-        chunkedResponseMsg.setNumberOfParts(3);
-        chunkedResponseMsg.addObjPart(e);
-        chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe =
-            (InternalFunctionInvocationTargetException) e.getCause();
-        chunkedResponseMsg.addObjPart(fe.getFailedNodeSet());
-        numParts = 3;
-      } else {
-        chunkedResponseMsg.setNumberOfParts(2);
-        chunkedResponseMsg.addObjPart(e);
-        chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        numParts = 2;
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Sending exception chunk: ", serverConnection.getName(), e);
-      }
-      chunkedResponseMsg.setServerConnection(serverConnection);
-      chunkedResponseMsg.setLastChunkAndNumParts(true, numParts);
-      chunkedResponseMsg.sendChunk(serverConnection);
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll.java
deleted file mode 100755
index 01293c0..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.operations.GetOperationContext;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.ObjectPartList;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.AuthorizeRequestPP;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class GetAll extends BaseCommand {
-
-  @Immutable
-  private static final GetAll singleton = new GetAll();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart = null, keysPart = null;
-    String regionName = null;
-    Object[] keys = null;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-
-    // Retrieve the region name from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    regionName = regionNamePart.getCachedString();
-
-    // Retrieve the keys array from the message parts
-    keysPart = clientMessage.getPart(1);
-    try {
-      keys = (Object[]) keysPart.getObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (logger.isDebugEnabled()) {
-      StringBuilder buffer = new StringBuilder();
-      buffer.append(serverConnection.getName()).append(": Received getAll request (")
-          .append(clientMessage.getPayloadLength()).append(" bytes) from ")
-          .append(serverConnection.getSocketString()).append(" for region ").append(regionName)
-          .append(" keys ");
-      if (keys != null) {
-        for (int i = 0; i < keys.length; i++) {
-          buffer.append(keys[i]).append(" ");
-        }
-      } else {
-        buffer.append("NULL");
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug(buffer.toString());
-      }
-    }
-
-    // Process the getAll request
-    if (regionName == null) {
-      String message = null;
-      message = "The input region name for the getAll request is null";
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      writeChunkedErrorResponse(clientMessage, MessageType.GET_ALL_DATA_ERROR, message,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during getAll request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    // Send header
-    ChunkedMessage chunkedResponseMsg = serverConnection.getChunkedResponseMessage();
-    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-    chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-    chunkedResponseMsg.sendHeader();
-
-    // Send chunk response
-    try {
-      fillAndSendGetAllResponseChunks(region, regionName, keys, serverConnection, securityService);
-      serverConnection.setAsTrue(RESPONDED);
-    } catch (Exception e) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, e);
-
-      // Otherwise, write an exception message and continue
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-  }
-
-  private void fillAndSendGetAllResponseChunks(Region region, String regionName, Object[] keys,
-      ServerConnection servConn, SecurityService securityService) throws IOException {
-
-    // Interpret null keys object as a request to get all key,value entry pairs
-    // of the region; otherwise iterate each key and perform the get behavior.
-    Iterator allKeysIter;
-    int numKeys;
-    if (keys != null) {
-      allKeysIter = null;
-      numKeys = keys.length;
-    } else {
-      Set allKeys = region.keySet();
-      allKeysIter = allKeys.iterator();
-      numKeys = allKeys.size();
-    }
-
-    ObjectPartList values = new ObjectPartList(MAXIMUM_CHUNK_SIZE, keys == null);
-    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-    AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
-    Request request = (Request) Request.getCommand();
-    Object[] valueAndIsObject = new Object[3];
-    for (int i = 0; i < numKeys; i++) {
-      // Send the intermediate chunk if necessary
-      if (values.size() == MAXIMUM_CHUNK_SIZE) {
-        // Send the chunk and clear the list
-        sendGetAllResponseChunk(region, values, false, servConn);
-        values.clear();
-      }
-
-      Object key;
-      if (keys != null) {
-        key = keys[i];
-      } else {
-        key = allKeysIter.next();
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Getting value for key={}", servConn.getName(), key);
-      }
-      // Determine if the user authorized to get this key
-      GetOperationContext getContext = null;
-      if (authzRequest != null) {
-        try {
-          getContext = authzRequest.getAuthorize(regionName, key, null);
-          if (logger.isDebugEnabled()) {
-            logger.debug("{}: Passed GET pre-authorization for key={}", servConn.getName(), key);
-          }
-        } catch (NotAuthorizedException ex) {
-          logger.warn(
-              String.format("%s: Caught the following exception attempting to get value for key=%s",
-                  new Object[] {servConn.getName(), key}),
-              ex);
-          values.addExceptionPart(key, ex);
-          continue;
-        }
-      }
-
-      try {
-        securityService.authorize(Resource.DATA, Operation.READ, regionName, key);
-      } catch (NotAuthorizedException ex) {
-        logger.warn(
-            String.format("%s: Caught the following exception attempting to get value for key=%s",
-                new Object[] {servConn.getName(), key}),
-            ex);
-        values.addExceptionPart(key, ex);
-        continue;
-      }
-
-      // Get the value and update the statistics. Do not deserialize
-      // the value if it is a byte[].
-      // Getting a value in serialized form is pretty nasty. I split this out
-      // so the logic can be re-used by the CacheClientProxy.
-      request.getValueAndIsObject(region, key, null, servConn, valueAndIsObject);
-      Object value = valueAndIsObject[0];
-      boolean isObject = ((Boolean) valueAndIsObject[1]).booleanValue();
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Retrieved value for key={}: {}", servConn.getName(), key, value);
-      }
-
-      if (postAuthzRequest != null) {
-        try {
-          getContext = postAuthzRequest.getAuthorize(regionName, key, value, isObject, getContext);
-          byte[] serializedValue = getContext.getSerializedValue();
-          if (serializedValue == null) {
-            value = getContext.getObject();
-          } else {
-            value = serializedValue;
-          }
-          isObject = getContext.isObject();
-          if (logger.isDebugEnabled()) {
-            logger.debug("{}: Passed GET post-authorization for key={}: {}", servConn.getName(),
-                key, value);
-          }
-        } catch (NotAuthorizedException ex) {
-          logger.warn(
-              String.format("%s: Caught the following exception attempting to get value for key=%s",
-                  new Object[] {servConn.getName(), key}),
-              ex);
-          values.addExceptionPart(key, ex);
-          continue;
-        }
-      }
-
-      // post process
-      value = securityService.postProcess(regionName, key, value, isObject);
-
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Returning value for key={}: {}", servConn.getName(), key, value);
-      }
-
-      // Add the value to the list of values
-      values.addObjectPart(key, value, isObject, null);
-    }
-
-    // Send the last chunk even if the list is of zero size.
-    sendGetAllResponseChunk(region, values, true, servConn);
-    servConn.setAsTrue(RESPONDED);
-  }
-
-  private static void sendGetAllResponseChunk(Region region, ObjectPartList list, boolean lastChunk,
-      ServerConnection servConn) throws IOException {
-    ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
-    chunkedResponseMsg.setNumberOfParts(1);
-    chunkedResponseMsg.setLastChunk(lastChunk);
-    chunkedResponseMsg.addObjPart(list, false);
-
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending {} getAll response chunk for region={} values={} chunk=<{}>",
-          servConn.getName(), (lastChunk ? " last " : " "), region.getFullPath(), list,
-          chunkedResponseMsg);
-    }
-
-    chunkedResponseMsg.sendChunk(servConn);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll651.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll651.java
deleted file mode 100644
index 75dbc70..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll651.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.operations.GetOperationContext;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.ObjectPartList651;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.AuthorizeRequestPP;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class GetAll651 extends BaseCommand {
-
-  @Immutable
-  private static final GetAll651 singleton = new GetAll651();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart = null, keysPart = null;
-    String regionName = null;
-    Object[] keys = null;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-
-    // Retrieve the region name from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    regionName = regionNamePart.getCachedString();
-
-    // Retrieve the keys array from the message parts
-    keysPart = clientMessage.getPart(1);
-    try {
-      keys = (Object[]) keysPart.getObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (logger.isDebugEnabled()) {
-      StringBuilder buffer = new StringBuilder();
-      buffer.append(serverConnection.getName()).append(": Received getAll request (")
-          .append(clientMessage.getPayloadLength()).append(" bytes) from ")
-          .append(serverConnection.getSocketString()).append(" for region ").append(regionName)
-          .append(" keys ");
-      if (keys != null) {
-        for (int i = 0; i < keys.length; i++) {
-          buffer.append(keys[i]).append(" ");
-        }
-      } else {
-        buffer.append("NULL");
-      }
-      logger.debug(buffer.toString());
-    }
-
-    // Process the getAll request
-    if (regionName == null) {
-      String message = null;
-      // if (regionName == null) (can only be null)
-      {
-        message = "The input region name for the getAll request is null";
-      }
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      writeChunkedErrorResponse(clientMessage, MessageType.GET_ALL_DATA_ERROR, message,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during getAll request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    // Send header
-    ChunkedMessage chunkedResponseMsg = serverConnection.getChunkedResponseMessage();
-    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-    chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-    chunkedResponseMsg.sendHeader();
-
-    // Send chunk response
-    try {
-      fillAndSendGetAllResponseChunks(region, regionName, keys, serverConnection, securityService);
-      serverConnection.setAsTrue(RESPONDED);
-    } catch (Exception e) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, e);
-
-      // Otherwise, write an exception message and continue
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-  }
-
-  private void fillAndSendGetAllResponseChunks(Region region, String regionName, Object[] keys,
-      ServerConnection servConn, SecurityService securityService) throws IOException {
-
-    // Interpret null keys object as a request to get all key,value entry pairs
-    // of the region; otherwise iterate each key and perform the get behavior.
-    Iterator allKeysIter;
-    int numKeys;
-    if (keys != null) {
-      allKeysIter = null;
-      numKeys = keys.length;
-    } else {
-      Set allKeys = region.keySet();
-      allKeysIter = allKeys.iterator();
-      numKeys = allKeys.size();
-    }
-    ObjectPartList651 values = getObjectPartsList(keys == null);
-    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-    AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
-    Request request = (Request) Request.getCommand();
-    Object[] valueAndIsObject = new Object[3];
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    for (int i = 0; i < numKeys; i++) {
-      // Send the intermediate chunk if necessary
-      if (values.size() == MAXIMUM_CHUNK_SIZE) {
-        // Send the chunk and clear the list
-        sendGetAllResponseChunk(region, values, false, servConn);
-        values.clear();
-      }
-
-      Object key;
-      boolean keyNotPresent = false;
-      if (keys != null) {
-        key = keys[i];
-      } else {
-        key = allKeysIter.next();
-      }
-      if (isDebugEnabled) {
-        logger.debug("{}: Getting value for key={}", servConn.getName(), key);
-      }
-      // Determine if the user authorized to get this key
-      GetOperationContext getContext = null;
-      if (authzRequest != null) {
-        try {
-          getContext = authzRequest.getAuthorize(regionName, key, null);
-          if (isDebugEnabled) {
-            logger.debug("{}: Passed GET pre-authorization for key={}", servConn.getName(), key);
-          }
-        } catch (NotAuthorizedException ex) {
-          logger.warn(
-              String.format("%s: Caught the following exception attempting to get value for key=%s",
-                  new Object[] {servConn.getName(), key}),
-              ex);
-          values.addExceptionPart(key, ex);
-          continue;
-        }
-      }
-
-      try {
-        securityService.authorize(Resource.DATA, Operation.READ, regionName, key);
-      } catch (NotAuthorizedException ex) {
-        logger.warn(
-            String.format("%s: Caught the following exception attempting to get value for key=%s",
-                new Object[] {servConn.getName(), key}),
-            ex);
-        values.addExceptionPart(key, ex);
-        continue;
-      }
-
-      // Get the value and update the statistics. Do not deserialize
-      // the value if it is a byte[].
-      // Getting a value in serialized form is pretty nasty. I split this out
-      // so the logic can be re-used by the CacheClientProxy.
-      request.getValueAndIsObject(region, key, null, servConn, valueAndIsObject);
-      Object value = valueAndIsObject[0];
-      boolean isObject = ((Boolean) valueAndIsObject[1]).booleanValue();
-      keyNotPresent = ((Boolean) valueAndIsObject[2]).booleanValue();
-      if (isDebugEnabled) {
-        logger.debug("{}: Retrieved value for key={}: {}", servConn.getName(), key, value);
-      }
-
-      if (postAuthzRequest != null) {
-        try {
-          getContext = postAuthzRequest.getAuthorize(regionName, key, value, isObject, getContext);
-          byte[] serializedValue = getContext.getSerializedValue();
-          if (serializedValue == null) {
-            value = getContext.getObject();
-          } else {
-            value = serializedValue;
-          }
-          isObject = getContext.isObject();
-          if (isDebugEnabled) {
-            logger.debug("{}: Passed GET post-authorization for key={}: {}", servConn.getName(),
-                key, value);
-          }
-        } catch (NotAuthorizedException ex) {
-          logger.warn(
-              String.format("%s: Caught the following exception attempting to get value for key=%s",
-                  new Object[] {servConn.getName(), key}),
-              ex);
-          values.addExceptionPart(key, ex);
-          continue;
-        }
-      }
-      value = securityService.postProcess(regionName, key, value, isObject);
-
-      if (isDebugEnabled) {
-        logger.debug("{}: Returning value for key={}: {}", servConn.getName(), key, value);
-      }
-
-      // Add the value to the list of values
-      if (keyNotPresent) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: key={} is not present on server.", servConn.getName(), key);
-        }
-        values.addObjectPartForAbsentKey(key, value);
-      } else {
-        values.addObjectPart(key, value, isObject, null);
-      }
-    }
-
-    // Send the last chunk even if the list is of zero size.
-    sendGetAllResponseChunk(region, values, true, servConn);
-    servConn.setAsTrue(RESPONDED);
-  }
-
-  /**
-   * This method is protected so it can be overriden in GetAllForRI.
-   *
-   * @param includeKeys if the part list should include the keys
-   */
-  protected ObjectPartList651 getObjectPartsList(boolean includeKeys) {
-    ObjectPartList651 values = new ObjectPartList651(MAXIMUM_CHUNK_SIZE, includeKeys);
-    return values;
-  }
-
-  private static void sendGetAllResponseChunk(Region region, ObjectPartList651 list,
-      boolean lastChunk, ServerConnection servConn) throws IOException {
-    ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
-    chunkedResponseMsg.setNumberOfParts(1);
-    chunkedResponseMsg.setLastChunk(lastChunk);
-    chunkedResponseMsg.addObjPart(list, false);
-
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending {} getAll response chunk for region={} values={} chunk=<{}>",
-          servConn.getName(), (lastChunk ? " last " : " "), region.getFullPath(), list,
-          chunkedResponseMsg);
-    }
-
-    chunkedResponseMsg.sendChunk(servConn);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAllForRI.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAllForRI.java
deleted file mode 100644
index 3fdd268..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetAllForRI.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.sockets.ObjectPartList651;
-import org.apache.geode.internal.cache.tier.sockets.SerializedObjectPartList;
-
-/**
- * A version of GetAll which in which the response contains the objects in serialized byte array
- * form, so that they can be separated into individual values without being deserialized. The
- * standard GetAll requires us to deserialize the value of every object.
- *
- * [bruce] this class is superseded by GetAll70, which merges GetAll651 and GetAllForRI
- *
- *
- */
-public class GetAllForRI extends GetAll651 {
-  @Immutable
-  private static final GetAllForRI singleton = new GetAllForRI();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  protected GetAllForRI() {}
-
-  @Override
-  protected ObjectPartList651 getObjectPartsList(boolean includeKeys) {
-    return new SerializedObjectPartList(MAXIMUM_CHUNK_SIZE, includeKeys);
-  }
-
-
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPRMetadataCommand.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPRMetadataCommand.java
deleted file mode 100755
index ec2f381..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPRMetadataCommand.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.BucketServerLocation;
-import org.apache.geode.internal.cache.BucketServerLocation66;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- * {@link Command} for {@link GetClientPRMetadataCommand}
- *
- *
- * @since GemFire 6.5
- */
-public class GetClientPRMetadataCommand extends BaseCommand {
-
-  @Immutable
-  private static final GetClientPRMetadataCommand singleton = new GetClientPRMetadataCommand();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private GetClientPRMetadataCommand() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start)
-      throws IOException, ClassNotFoundException, InterruptedException {
-    String regionFullPath = null;
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    regionFullPath = clientMessage.getPart(0).getCachedString();
-    String errMessage = "";
-    if (regionFullPath == null) {
-      logger.warn("The input region path for the GetClientPRMetadata request is null");
-      errMessage =
-          "The input region path for the GetClientPRMetadata request is null";
-      writeErrorResponse(clientMessage, MessageType.GET_CLIENT_PR_METADATA_ERROR,
-          errMessage.toString(), serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-    } else {
-      Region region = crHelper.getRegion(regionFullPath);
-      if (region == null) {
-        logger.warn("Region was not found during GetClientPRMetadata request for region path : {}",
-            regionFullPath);
-        errMessage = "Region was not found during GetClientPRMetadata request for region path : "
-            + regionFullPath;
-        writeErrorResponse(clientMessage, MessageType.GET_CLIENT_PR_METADATA_ERROR,
-            errMessage.toString(), serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      } else {
-        try {
-          Message responseMsg = serverConnection.getResponseMessage();
-          responseMsg.setTransactionId(clientMessage.getTransactionId());
-          responseMsg.setMessageType(MessageType.RESPONSE_CLIENT_PR_METADATA);
-
-          PartitionedRegion prRgion = (PartitionedRegion) region;
-          Map<Integer, List<BucketServerLocation66>> bucketToServerLocations =
-              prRgion.getRegionAdvisor().getAllClientBucketProfiles();
-          responseMsg.setNumberOfParts(bucketToServerLocations.size());
-          for (List<BucketServerLocation66> serverLocations : bucketToServerLocations.values()) {
-            List<BucketServerLocation> oldServerLocations = new ArrayList<BucketServerLocation>();
-            for (BucketServerLocation66 bs : serverLocations) {
-              oldServerLocations.add(new BucketServerLocation(bs.getBucketId(), bs.getPort(),
-                  bs.getHostName(), bs.isPrimary(), bs.getVersion()));
-              responseMsg.addObjPart(oldServerLocations);
-            }
-          }
-          responseMsg.send();
-          clientMessage.clearParts();
-        } catch (Exception e) {
-          writeException(clientMessage, e, false, serverConnection);
-        } finally {
-          serverConnection.setAsTrue(Command.RESPONDED);
-        }
-      }
-    }
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPartitionAttributesCommand.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPartitionAttributesCommand.java
deleted file mode 100755
index c6f9e3c..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPartitionAttributesCommand.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.PartitionResolver;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.client.internal.GetClientPartitionAttributesOp;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.SecurityService;
-
-/**
- * {@link Command} for {@link GetClientPartitionAttributesOp} operation
- *
- * @since GemFire 6.5
- */
-public class GetClientPartitionAttributesCommand extends BaseCommand {
-
-  @Immutable
-  private static final GetClientPartitionAttributesCommand singleton =
-      new GetClientPartitionAttributesCommand();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start)
-      throws IOException, ClassNotFoundException, InterruptedException {
-    String regionFullPath = null;
-    regionFullPath = clientMessage.getPart(0).getCachedString();
-    String errMessage = "";
-    if (regionFullPath == null) {
-      errMessage = "The input region path for the GetClientPartitionAttributes request is null";
-      logger.warn(errMessage);
-      writeErrorResponse(clientMessage, MessageType.GET_CLIENT_PARTITION_ATTRIBUTES_ERROR,
-          errMessage, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    Region region = serverConnection.getCache().getRegion(regionFullPath);
-    if (region == null) {
-      logger.warn(
-          "Region was not found during GetClientPartitionAttributes request for region path : {}",
-          regionFullPath);
-      errMessage =
-          "Region was not found during GetClientPartitionAttributes request for region path : "
-              + regionFullPath;
-      writeErrorResponse(clientMessage, MessageType.GET_CLIENT_PARTITION_ATTRIBUTES_ERROR,
-          errMessage.toString(), serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    try {
-      Message responseMsg = serverConnection.getResponseMessage();
-      responseMsg.setTransactionId(clientMessage.getTransactionId());
-      responseMsg.setMessageType(MessageType.RESPONSE_CLIENT_PARTITION_ATTRIBUTES);
-
-      PartitionedRegion prRgion = (PartitionedRegion) region;
-
-      PartitionResolver partitionResolver = prRgion.getPartitionResolver();
-      int numParts = 2; // MINUMUM PARTS
-      if (partitionResolver != null) {
-        numParts++;
-      }
-      responseMsg.setNumberOfParts(numParts);
-      // PART 1
-      responseMsg.addObjPart(prRgion.getTotalNumberOfBuckets());
-
-      // PART 2
-      if (partitionResolver != null) {
-        responseMsg.addObjPart(partitionResolver.getClass().toString().substring(6));
-      }
-
-      // PART 3
-      String leaderRegionPath = null;
-      PartitionedRegion leaderRegion = null;
-      String leaderRegionName = prRgion.getColocatedWith();
-      if (leaderRegionName != null) {
-        Cache cache = prRgion.getCache();
-        while (leaderRegionName != null) {
-          leaderRegion = (PartitionedRegion) cache.getRegion(leaderRegionName);
-          if (leaderRegion.getColocatedWith() == null) {
-            leaderRegionPath = leaderRegion.getFullPath();
-            break;
-          } else {
-            leaderRegionName = leaderRegion.getColocatedWith();
-          }
-        }
-      }
-      responseMsg.addObjPart(leaderRegionPath);
-      responseMsg.send();
-      clientMessage.clearParts();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-    } finally {
-      serverConnection.setAsTrue(Command.RESPONDED);
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetEntryCommand.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetEntryCommand.java
deleted file mode 100644
index 5da7bec..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetEntryCommand.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.Region.Entry;
-import org.apache.geode.internal.cache.EntrySnapshot;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.NonLocalRegionEntry;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-
-/**
- * getEntry(key) operation performed on server. Extends Request, and overrides getValueAndIsObject()
- * in Request so as to not invoke loader.
- *
- * @since GemFire 6.6
- */
-public class GetEntryCommand extends Request {
-
-  @Immutable
-  private static final GetEntryCommand singleton = new GetEntryCommand();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  protected GetEntryCommand() {}
-
-  @Override
-  public void getValueAndIsObject(Region p_region, Object key, Object callbackArg,
-      ServerConnection servConn, Object[] result) {
-    Object data = null;
-    LocalRegion region = (LocalRegion) p_region;
-    Entry entry = region.getEntry(key);
-    if (logger.isDebugEnabled()) {
-      logger.debug("GetEntryCommand: for key: {} returning entry: {}", key, entry);
-    }
-    if (entry != null) {
-      EntrySnapshot snap = new EntrySnapshot();
-      NonLocalRegionEntry re = new NonLocalRegionEntry(entry, region);
-      snap.setRegionEntry(re);
-      snap.setRegion(region);
-      data = snap;
-    }
-    result[0] = data;
-    result[1] = true; // isObject is true
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate.java
deleted file mode 100644
index 3aecc9f..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.EntryNotFoundException;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.operations.InvalidateOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.versions.VersionTag;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.util.Breadcrumbs;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class Invalidate extends BaseCommand {
-
-  @Immutable
-  private static final Invalidate singleton = new Invalidate();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart = null, keyPart = null, callbackArgPart = null;
-    String regionName = null;
-    Object callbackArg = null, key = null;
-    Part eventPart = null;
-    StringBuilder errMessage = new StringBuilder();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadInvalidateRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    keyPart = clientMessage.getPart(1);
-    eventPart = clientMessage.getPart(2);
-    // callbackArgPart = null; (redundant assignment)
-    if (clientMessage.getNumberOfParts() > 3) {
-      callbackArgPart = clientMessage.getPart(3);
-      try {
-        callbackArg = callbackArgPart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    regionName = regionNamePart.getCachedString();
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    if (logger.isDebugEnabled()) {
-      logger.debug(serverConnection.getName() + ": Received invalidate request ("
-          + clientMessage.getPayloadLength() + " bytes) from " + serverConnection.getSocketString()
-          + " for region " + regionName + " key " + key);
-    }
-
-    // Process the invalidate request
-    if (key == null || regionName == null) {
-      if (key == null) {
-        logger.warn("The input key for the invalidate request is null");
-        errMessage.append("The input key for the invalidate request is null");
-      }
-      if (regionName == null) {
-        logger.warn("The input region name for the invalidate request is null");
-        errMessage
-            .append("The input region name for the invalidate request is null");
-      }
-      writeErrorResponse(clientMessage, MessageType.DESTROY_DATA_ERROR, errMessage.toString(),
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during invalidate request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    // Invalidate the entry
-    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    EventID eventId =
-        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-
-    Breadcrumbs.setEventId(eventId);
-
-    VersionTag tag = null;
-
-    try {
-      // for integrated security
-      securityService.authorize(Resource.DATA, Operation.WRITE, regionName, key);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        InvalidateOperationContext invalidateContext =
-            authzRequest.invalidateAuthorize(regionName, key, callbackArg);
-        callbackArg = invalidateContext.getCallbackArg();
-      }
-      EventIDHolder clientEvent = new EventIDHolder(eventId);
-
-      // msg.isRetry might be set by v7.0 and later clients
-      if (clientMessage.isRetry()) {
-        // if (logger.isDebugEnabled()) {
-        // logger.debug("DEBUG: encountered isRetry in Invalidate");
-        // }
-        clientEvent.setPossibleDuplicate(true);
-        if (region.getAttributes().getConcurrencyChecksEnabled()) {
-          // recover the version tag from other servers
-          clientEvent.setRegion(region);
-          if (!recoverVersionTagForRetriedOperation(clientEvent)) {
-            clientEvent.setPossibleDuplicate(false); // no-one has seen this event
-          }
-        }
-      }
-
-      region.basicBridgeInvalidate(key, callbackArg, serverConnection.getProxyID(), true,
-          clientEvent);
-      tag = clientEvent.getVersionTag();
-      serverConnection.setModificationInfo(true, regionName, key);
-    } catch (EntryNotFoundException e) {
-      // Don't send an exception back to the client if this
-      // exception happens. Just log it and continue.
-      logger.info("During {} no entry was found for key {}",
-          new Object[] {"invalidate", key});
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (Exception e) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, e);
-
-      // If an exception occurs during the destroy, preserve the connection
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      if (e instanceof GemFireSecurityException) {
-        // Fine logging for security exceptions since these are already
-        // logged by the security logger
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), e);
-        }
-      } else {
-        logger.warn(String.format("%s: Unexpected Exception",
-            serverConnection.getName()), e);
-      }
-      return;
-    }
-
-    // Update the statistics and write the reply
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessInvalidateTime(start - oldStart);
-    }
-    if (region instanceof PartitionedRegion) {
-      PartitionedRegion pr = (PartitionedRegion) region;
-      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, pr.getNetworkHopType(),
-            tag);
-        pr.clearNetworkHopData();
-      } else {
-        writeReply(clientMessage, serverConnection, tag);
-      }
-    } else {
-      writeReply(clientMessage, serverConnection, tag);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sent invalidate response for region {} key {}", serverConnection.getName(),
-          regionName, key);
-    }
-    stats.incWriteInvalidateResponseTime(DistributionStats.getStatTime() - start);
-  }
-
-  protected void writeReply(Message origMsg, ServerConnection servConn, VersionTag tag)
-      throws IOException {
-    writeReply(origMsg, servConn);
-  }
-
-  protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection servConn,
-      PartitionedRegion pr, byte nwHop, VersionTag tag) throws IOException {
-    writeReplyWithRefreshMetadata(origMsg, servConn, pr, nwHop);
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70.java
index 320e0da..92eaad3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70.java
@@ -15,18 +15,34 @@
 package org.apache.geode.internal.cache.tier.sockets.command;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.geode.annotations.Immutable;
+import org.apache.geode.cache.EntryNotFoundException;
+import org.apache.geode.cache.RegionDestroyedException;
 import org.apache.geode.cache.client.internal.DestroyOp;
 import org.apache.geode.cache.client.internal.InvalidateOp;
+import org.apache.geode.cache.operations.InvalidateOperationContext;
+import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.internal.cache.EventID;
+import org.apache.geode.internal.cache.EventIDHolder;
+import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
+import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
 import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.cache.versions.VersionTag;
+import org.apache.geode.internal.security.AuthorizeRequest;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.util.Breadcrumbs;
+import org.apache.geode.security.GemFireSecurityException;
+import org.apache.geode.security.ResourcePermission;
 
-public class Invalidate70 extends Invalidate {
+public class Invalidate70 extends BaseCommand {
 
   @Immutable
   private static final Invalidate70 singleton = new Invalidate70();
@@ -38,6 +54,171 @@
   private Invalidate70() {}
 
   @Override
+  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
+      final SecurityService securityService, long start) throws IOException, InterruptedException {
+    Part regionNamePart = null, keyPart = null, callbackArgPart = null;
+    String regionName = null;
+    Object callbackArg = null, key = null;
+    Part eventPart = null;
+    StringBuilder errMessage = new StringBuilder();
+    CacheServerStats stats = serverConnection.getCacheServerStats();
+    serverConnection.setAsTrue(REQUIRES_RESPONSE);
+
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incReadInvalidateRequestTime(start - oldStart);
+    }
+    // Retrieve the data from the message parts
+    regionNamePart = clientMessage.getPart(0);
+    keyPart = clientMessage.getPart(1);
+    eventPart = clientMessage.getPart(2);
+    // callbackArgPart = null; (redundant assignment)
+    if (clientMessage.getNumberOfParts() > 3) {
+      callbackArgPart = clientMessage.getPart(3);
+      try {
+        callbackArg = callbackArgPart.getObject();
+      } catch (Exception e) {
+        writeException(clientMessage, e, false, serverConnection);
+        serverConnection.setAsTrue(RESPONDED);
+        return;
+      }
+    }
+    regionName = regionNamePart.getCachedString();
+    try {
+      key = keyPart.getStringOrObject();
+    } catch (Exception e) {
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug(serverConnection.getName() + ": Received invalidate request ("
+          + clientMessage.getPayloadLength() + " bytes) from " + serverConnection.getSocketString()
+          + " for region " + regionName + " key " + key);
+    }
+
+    // Process the invalidate request
+    if (key == null || regionName == null) {
+      if (key == null) {
+        logger.warn("The input key for the invalidate request is null");
+        errMessage.append("The input key for the invalidate request is null");
+      }
+      if (regionName == null) {
+        logger.warn("The input region name for the invalidate request is null");
+        errMessage
+            .append("The input region name for the invalidate request is null");
+      }
+      writeErrorResponse(clientMessage, MessageType.DESTROY_DATA_ERROR, errMessage.toString(),
+          serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during invalidate request";
+      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+    // Invalidate the entry
+    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    EventID eventId =
+        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
+
+    Breadcrumbs.setEventId(eventId);
+
+    VersionTag tag = null;
+
+    try {
+      // for integrated security
+      securityService.authorize(ResourcePermission.Resource.DATA,
+          ResourcePermission.Operation.WRITE, regionName, key);
+
+      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
+      if (authzRequest != null) {
+        InvalidateOperationContext invalidateContext =
+            authzRequest.invalidateAuthorize(regionName, key, callbackArg);
+        callbackArg = invalidateContext.getCallbackArg();
+      }
+      EventIDHolder clientEvent = new EventIDHolder(eventId);
+
+      // msg.isRetry might be set by v7.0 and later clients
+      if (clientMessage.isRetry()) {
+        // if (logger.isDebugEnabled()) {
+        // logger.debug("DEBUG: encountered isRetry in Invalidate");
+        // }
+        clientEvent.setPossibleDuplicate(true);
+        if (region.getAttributes().getConcurrencyChecksEnabled()) {
+          // recover the version tag from other servers
+          clientEvent.setRegion(region);
+          if (!recoverVersionTagForRetriedOperation(clientEvent)) {
+            clientEvent.setPossibleDuplicate(false); // no-one has seen this event
+          }
+        }
+      }
+
+      region.basicBridgeInvalidate(key, callbackArg, serverConnection.getProxyID(), true,
+          clientEvent);
+      tag = clientEvent.getVersionTag();
+      serverConnection.setModificationInfo(true, regionName, key);
+    } catch (EntryNotFoundException e) {
+      // Don't send an exception back to the client if this
+      // exception happens. Just log it and continue.
+      logger.info("During {} no entry was found for key {}",
+          new Object[] {"invalidate", key});
+    } catch (RegionDestroyedException rde) {
+      writeException(clientMessage, rde, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(serverConnection, e);
+
+      // If an exception occurs during the destroy, preserve the connection
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      if (e instanceof GemFireSecurityException) {
+        // Fine logging for security exceptions since these are already
+        // logged by the security logger
+        if (logger.isDebugEnabled()) {
+          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), e);
+        }
+      } else {
+        logger.warn(String.format("%s: Unexpected Exception",
+            serverConnection.getName()), e);
+      }
+      return;
+    }
+
+    // Update the statistics and write the reply
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessInvalidateTime(start - oldStart);
+    }
+    if (region instanceof PartitionedRegion) {
+      PartitionedRegion pr = (PartitionedRegion) region;
+      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
+        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, pr.getNetworkHopType(),
+            tag);
+        pr.clearNetworkHopData();
+      } else {
+        writeReply(clientMessage, serverConnection, tag);
+      }
+    } else {
+      writeReply(clientMessage, serverConnection, tag);
+    }
+    serverConnection.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent invalidate response for region {} key {}", serverConnection.getName(),
+          regionName, key);
+    }
+    stats.incWriteInvalidateResponseTime(DistributionStats.getStatTime() - start);
+  }
+
   protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection servConn,
       PartitionedRegion pr, byte nwHop, VersionTag versionTag) throws IOException {
     Message replyMsg = servConn.getReplyMessage();
@@ -64,7 +245,6 @@
     }
   }
 
-  @Override
   protected void writeReply(Message origMsg, ServerConnection servConn, VersionTag versionTag)
       throws IOException {
     Message replyMsg = servConn.getReplyMessage();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put.java
deleted file mode 100644
index 90ec9a4..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.ResourceException;
-import org.apache.geode.cache.operations.PutOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class Put extends BaseCommand {
-
-  @Immutable
-  private static final Put singleton = new Put();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart = null, keyPart = null, valuePart = null, callbackArgPart = null;
-    String regionName = null;
-    Object callbackArg = null, key = null;
-    Part eventPart = null;
-    String errMessage = "";
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-
-    // requiresResponse = true;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadPutRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    keyPart = clientMessage.getPart(1);
-    valuePart = clientMessage.getPart(2);
-    eventPart = clientMessage.getPart(3);
-    // callbackArgPart = null; (redundant assignment)
-    if (clientMessage.getNumberOfParts() > 4) {
-      callbackArgPart = clientMessage.getPart(4);
-      try {
-        callbackArg = callbackArgPart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    regionName = regionNamePart.getCachedString();
-
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: Received put request ({} bytes) from {} for region {} key {} value {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key, valuePart);
-    }
-
-    // Process the put request
-    if (key == null || regionName == null) {
-      if (key == null) {
-        logger.warn("{} The input key for the put request is null",
-            serverConnection.getName());
-        errMessage =
-            "The input key for the put request is null";
-      }
-      if (regionName == null) {
-        logger.warn("{} The input region name for the put request is null",
-            serverConnection.getName());
-        errMessage = "The input region name for the put request is null";
-      }
-      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during put request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (valuePart.isNull() && region.containsKey(key)) {
-      // Invalid to 'put' a null value in an existing key
-      logger.info("{}: Attempted to put a null value for existing key {}",
-          serverConnection.getName(), key);
-      errMessage =
-          "Attempted to put a null value for existing key %s";
-      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    EventID eventId =
-        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-
-    try {
-      byte[] value = valuePart.getSerializedForm();
-      boolean isObject = valuePart.isObject();
-
-      securityService.authorize(Resource.DATA, Operation.WRITE, regionName, key);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          authzRequest.createRegionAuthorize((String) key);
-        }
-        // Allow PUT operations on meta regions (bug #38961)
-        else if (!region.isUsedForMetaRegion()) {
-          PutOperationContext putContext =
-              authzRequest.putAuthorize(regionName, key, value, isObject, callbackArg);
-          value = putContext.getSerializedValue();
-          isObject = putContext.isObject();
-          callbackArg = putContext.getCallbackArg();
-        }
-      }
-      // If the value is 1 byte and the byte represents null,
-      // attempt to create the entry. This test needs to be
-      // moved to DataSerializer or DataSerializer.NULL needs
-      // to be publicly accessible.
-      boolean result = false;
-      if (value == null) {
-        // Create the null entry. Since the value is null, the value of the
-        // isObject
-        // the true after null doesn't matter and is not used.
-        result = region.basicBridgeCreate(key, null, true, callbackArg,
-            serverConnection.getProxyID(), true, new EventIDHolder(eventId), false);
-      } else {
-        // Put the entry
-        result = region.basicBridgePut(key, value, null, isObject, callbackArg,
-            serverConnection.getProxyID(), true, new EventIDHolder(eventId));
-      }
-      if (result) {
-        serverConnection.setModificationInfo(true, regionName, key);
-      } else {
-        String message = "%s: Failed to put entry for region %s key %s value %s";
-        String s = String.format(message, serverConnection.getName(), regionName, key, valuePart);
-        logger.info(s);
-        throw new Exception(s);
-      }
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (ResourceException re) {
-      writeException(clientMessage, re, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (Exception ce) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ce);
-
-      // If an exception occurs during the put, preserve the connection
-      writeException(clientMessage, ce, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      if (ce instanceof GemFireSecurityException) {
-        // Fine logging for security exceptions since these are already
-        // logged by the security logger
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), ce);
-        }
-      } else {
-        logger.warn(String.format("%s: Unexpected Exception",
-            serverConnection.getName()), ce);
-      }
-      return;
-    } finally {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessPutTime(start - oldStart);
-    }
-
-    // Increment statistics and write the reply
-    writeReply(clientMessage, serverConnection);
-
-    serverConnection.setAsTrue(RESPONDED);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sent put response back to {} for region {} key {} value {}",
-          serverConnection.getName(), serverConnection.getSocketString(), regionName, key,
-          valuePart);
-    }
-    stats.incWritePutResponseTime(DistributionStats.getStatTime() - start);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put61.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put61.java
deleted file mode 100644
index 946924a..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put61.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.geode.InvalidDeltaException;
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.ResourceException;
-import org.apache.geode.cache.operations.PutOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-/**
- * @since GemFire 6.1
- */
-public class Put61 extends BaseCommand {
-
-  @Immutable
-  private static final Put61 singleton = new Put61();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long p_start)
-      throws IOException, InterruptedException {
-    long start = p_start;
-    Part regionNamePart = null, keyPart = null, valuePart = null, callbackArgPart = null;
-    String regionName = null;
-    Object callbackArg = null, key = null;
-    Part eventPart = null;
-    StringBuilder errMessage = new StringBuilder();
-    boolean isDelta = false;
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-
-    // requiresResponse = true;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadPutRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    keyPart = clientMessage.getPart(1);
-    try {
-      isDelta = (Boolean) clientMessage.getPart(2).getObject();
-    } catch (Exception e) {
-      writeException(clientMessage, MessageType.PUT_DELTA_ERROR, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      // CachePerfStats not available here.
-      return;
-    }
-    valuePart = clientMessage.getPart(3);
-    eventPart = clientMessage.getPart(4);
-    if (clientMessage.getNumberOfParts() > 5) {
-      callbackArgPart = clientMessage.getPart(5);
-      try {
-        callbackArg = callbackArgPart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    regionName = regionNamePart.getCachedString();
-
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    if (isDebugEnabled) {
-      logger.debug("{}: Received 6.1{}put request ({} bytes) from {} for region {} key {}",
-          serverConnection.getName(), (isDelta ? " delta " : " "), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key);
-    }
-
-    // Process the put request
-    if (key == null || regionName == null) {
-      if (key == null) {
-        String putMsg = " The input key for the 6.1 put request is null";
-        if (isDebugEnabled) {
-          logger.debug("{}:{}", serverConnection.getName(), putMsg);
-        }
-        errMessage.append(putMsg);
-      }
-      if (regionName == null) {
-        String putMsg = " The input region name for the 6.1 put request is null";
-        if (isDebugEnabled) {
-          logger.debug("{}:{}", serverConnection.getName(), putMsg);
-        }
-        errMessage.append(putMsg);
-      }
-      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      String reason = " was not found during 6.1 put request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (valuePart.isNull() && region.containsKey(key)) {
-      // Invalid to 'put' a null value in an existing key
-      String putMsg = " Attempted to 6.1 put a null value for existing key " + key;
-      if (isDebugEnabled) {
-        logger.debug("{}:{}", serverConnection.getName(), putMsg);
-      }
-      errMessage.append(putMsg);
-      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    // try {
-    // this.eventId = (EventID)eventPart.getObject();
-    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    EventID eventId =
-        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-
-    try {
-      Object value = null;
-      if (!isDelta) {
-        value = valuePart.getSerializedForm();
-      }
-      boolean isObject = valuePart.isObject();
-      boolean isMetaRegion = region.isUsedForMetaRegion();
-      clientMessage.setMetaRegion(isMetaRegion);
-
-      securityService.authorize(Resource.DATA, Operation.WRITE, regionName, key);
-
-      AuthorizeRequest authzRequest = null;
-      if (!isMetaRegion) {
-        authzRequest = serverConnection.getAuthzRequest();
-      }
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          authzRequest.createRegionAuthorize((String) key);
-        }
-        // Allow PUT operations on meta regions (bug #38961)
-        else {
-          PutOperationContext putContext =
-              authzRequest.putAuthorize(regionName, key, value, isObject, callbackArg);
-          value = putContext.getValue();
-          isObject = putContext.isObject();
-          callbackArg = putContext.getCallbackArg();
-        }
-      }
-      // If the value is 1 byte and the byte represents null,
-      // attempt to create the entry. This test needs to be
-      // moved to DataSerializer or DataSerializer.NULL needs
-      // to be publicly accessible.
-      boolean result = false;
-      if (value == null && !isDelta) {
-        // Create the null entry. Since the value is null, the value of the
-        // isObject
-        // the true after null doesn't matter and is not used.
-        result = region.basicBridgeCreate(key, null, true, callbackArg,
-            serverConnection.getProxyID(), true, new EventIDHolder(eventId), false);
-      } else {
-        // Put the entry
-        byte[] delta = null;
-        if (isDelta) {
-          delta = valuePart.getSerializedForm();
-        }
-        result = region.basicBridgePut(key, value, delta, isObject, callbackArg,
-            serverConnection.getProxyID(), true, new EventIDHolder(eventId));
-      }
-      if (result) {
-        serverConnection.setModificationInfo(true, regionName, key);
-      } else {
-        String message = serverConnection.getName() + ": Failed to 6.1 put entry for region "
-            + regionName + " key " + key + " value " + valuePart;
-        if (isDebugEnabled) {
-          logger.debug(message);
-        }
-        throw new Exception(message);
-      }
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (ResourceException re) {
-      writeException(clientMessage, re, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (InvalidDeltaException ide) {
-      logger.info("Error applying delta for key {} of region {}: {}",
-          key, regionName, ide.getMessage());
-      writeException(clientMessage, MessageType.PUT_DELTA_ERROR, ide, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      region.getCachePerfStats().incDeltaFullValuesRequested();
-      return;
-
-    } catch (Exception ce) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ce);
-
-      // If an exception occurs during the put, preserve the connection
-      writeException(clientMessage, ce, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      if (ce instanceof GemFireSecurityException) {
-        // Fine logging for security exceptions since these are already
-        // logged by the security logger
-        if (isDebugEnabled) {
-          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), ce);
-        }
-      } else if (isDebugEnabled) {
-        logger.debug("{}: Unexpected Exception", serverConnection.getName(), ce);
-      }
-      return;
-    } finally {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessPutTime(start - oldStart);
-    }
-
-    // Increment statistics and write the reply
-    if (region instanceof PartitionedRegion) {
-      PartitionedRegion pr = (PartitionedRegion) region;
-      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, pr.getNetworkHopType());
-        pr.clearNetworkHopData();
-      } else {
-        writeReply(clientMessage, serverConnection);
-      }
-    } else {
-      writeReply(clientMessage, serverConnection);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    if (isDebugEnabled) {
-      logger.debug("{}: Sent 6.1 put response back to {} for region {} key {} value {}",
-          serverConnection.getName(), serverConnection.getSocketString(), regionName, key,
-          valuePart);
-    }
-    stats.incWritePutResponseTime(DistributionStats.getStatTime() - start);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put65.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put65.java
deleted file mode 100644
index db8ef00..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put65.java
+++ /dev/null
@@ -1,549 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.geode.InvalidDeltaException;
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.Operation;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.ResourceException;
-import org.apache.geode.cache.operations.PutOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.HeapDataOutputStream;
-import org.apache.geode.internal.InternalDataSerializer;
-import org.apache.geode.internal.cache.CachedDeserializable;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.TXManagerImpl;
-import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.versions.VersionTag;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.internal.util.Breadcrumbs;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-/**
- * @since GemFire 6.5
- */
-public class Put65 extends BaseCommand {
-
-  @Immutable
-  private static final Put65 singleton = new Put65();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long p_start)
-      throws IOException, InterruptedException {
-    long start = p_start;
-    final CacheServerStats stats = serverConnection.getCacheServerStats();
-
-    // requiresResponse = true;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadPutRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    int idx = 0;
-
-    final Part regionNamePart = clientMessage.getPart(idx++);
-
-    final Operation operation;
-    try {
-      final Part operationPart = clientMessage.getPart(idx++);
-
-      if (operationPart.isBytes()) {
-        final byte[] bytes = operationPart.getSerializedForm();
-        if (null == bytes || 0 == bytes.length) {
-          // older clients can send empty bytes for default operation.
-          operation = Operation.UPDATE;
-        } else {
-          operation = Operation.fromOrdinal(bytes[0]);
-        }
-      } else {
-
-        // Fallback for older clients.
-        if (operationPart.getObject() == null) {
-          // native clients may send a null since the op is java-serialized.
-          operation = Operation.UPDATE;
-        } else {
-          operation = (Operation) operationPart.getObject();
-        }
-      }
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final int flags = clientMessage.getPart(idx++).getInt();
-    final boolean requireOldValue = ((flags & 0x01) == 0x01);
-    final boolean haveExpectedOldValue = ((flags & 0x02) == 0x02);
-    final Object expectedOldValue;
-    if (haveExpectedOldValue) {
-      try {
-        expectedOldValue = clientMessage.getPart(idx++).getObject();
-      } catch (ClassNotFoundException e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    } else {
-      expectedOldValue = null;
-    }
-
-    final Part keyPart = clientMessage.getPart(idx++);
-
-    final boolean isDelta;
-    try {
-      isDelta = ((Boolean) clientMessage.getPart(idx).getObject());
-      idx += 1;
-    } catch (Exception e) {
-      writeException(clientMessage, MessageType.PUT_DELTA_ERROR, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      // CachePerfStats not available here.
-      return;
-    }
-
-    final Part valuePart = clientMessage.getPart(idx++);
-    final Part eventPart = clientMessage.getPart(idx++);
-
-    Object callbackArg = null;
-    if (clientMessage.getNumberOfParts() > idx) {
-      final Part callbackArgPart = clientMessage.getPart(idx++);
-      try {
-        callbackArg = callbackArgPart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    final Object key;
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final String regionName = regionNamePart.getCachedString();
-
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    if (isDebugEnabled) {
-      logger.debug(
-          "{}: Received {}put request ({} bytes) from {} for region {} key {} txId {} posdup: {}",
-          serverConnection.getName(), (isDelta ? " delta " : " "), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key, clientMessage.getTransactionId(),
-          clientMessage.isRetry());
-    }
-
-    // Process the put request
-    if (key == null || regionName == null) {
-      final StringBuilder errMessage = new StringBuilder();
-      if (key == null) {
-        final String putMsg = " The input key for the put request is null";
-        if (isDebugEnabled) {
-          logger.debug("{}:{}", serverConnection.getName(), putMsg);
-        }
-        errMessage.append(putMsg);
-      }
-      if (regionName == null) {
-        final String putMsg = " The input region name for the put request is null";
-        if (isDebugEnabled) {
-          logger.debug("{}:{}", serverConnection.getName(), putMsg);
-        }
-        errMessage.append(putMsg);
-      }
-      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      final String reason = " was not found during put request";
-      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    if (valuePart.isNull() && operation != Operation.PUT_IF_ABSENT && region.containsKey(key)) {
-      // Invalid to 'put' a null value in an existing key
-      final String putMsg = " Attempted to put a null value for existing key " + key;
-      if (isDebugEnabled) {
-        logger.debug("{}:{}", serverConnection.getName(), putMsg);
-      }
-      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, putMsg,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    final ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-    final long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-    final long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-
-    final EventIDHolder clientEvent = new EventIDHolder(
-        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId));
-
-    Breadcrumbs.setEventId(clientEvent.getEventId());
-
-    // msg.isRetry might be set by v7.0 and later clients
-    if (clientMessage.isRetry()) {
-      if (shouldSetPossibleDuplicate(region, clientEvent)) {
-        clientEvent.setPossibleDuplicate(true);
-      }
-    }
-
-    boolean result = false;
-    boolean sendOldValue = false;
-    boolean oldValueIsObject = true;
-    Object oldValue = null;
-
-    try {
-      Object value = null;
-      if (!isDelta) {
-        value = valuePart.getSerializedForm();
-      }
-      boolean isObject = valuePart.isObject();
-      boolean isMetaRegion = region.isUsedForMetaRegion();
-      clientMessage.setMetaRegion(isMetaRegion);
-
-      securityService.authorize(Resource.DATA, ResourcePermission.Operation.WRITE, regionName,
-          key);
-
-      AuthorizeRequest authzRequest = null;
-      if (!isMetaRegion) {
-        authzRequest = serverConnection.getAuthzRequest();
-      }
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          authzRequest.createRegionAuthorize((String) key);
-        }
-        // Allow PUT operations on meta regions (bug #38961)
-        else {
-          PutOperationContext putContext =
-              authzRequest.putAuthorize(regionName, key, value, isObject, callbackArg);
-          value = putContext.getValue();
-          isObject = putContext.isObject();
-          callbackArg = putContext.getCallbackArg();
-        }
-      }
-      if (isDebugEnabled) {
-        logger.debug("processing put65 with operation={}", operation);
-      }
-
-      // If the value is 1 byte and the byte represents null,
-      // attempt to create the entry. This test needs to be
-      // moved to DataSerializer or DataSerializer.NULL needs
-      // to be publicly accessible.
-      if (operation == Operation.PUT_IF_ABSENT) {
-        // try {
-        if (clientMessage.isRetry() && clientEvent.getVersionTag() != null) {
-          // bug #46590 the operation was successful the last time since it
-          // was applied to the cache, so return success and the recovered
-          // version tag
-          if (isDebugEnabled) {
-            logger.debug("putIfAbsent operation was successful last time with version {}",
-                clientEvent.getVersionTag());
-          }
-          // invoke basicBridgePutIfAbsent anyway to ensure that the event is distributed to all
-          // servers - bug #51664
-          region.basicBridgePutIfAbsent(key, value, isObject, callbackArg,
-              serverConnection.getProxyID(), true, clientEvent);
-          oldValue = null;
-        } else {
-          oldValue = region.basicBridgePutIfAbsent(key, value, isObject, callbackArg,
-              serverConnection.getProxyID(), true, clientEvent);
-        }
-        sendOldValue = true;
-        oldValueIsObject = true;
-        KnownVersion clientVersion = serverConnection.getClientVersion();
-        if (oldValue instanceof CachedDeserializable) {
-          oldValue = ((CachedDeserializable) oldValue).getSerializedValue();
-        } else if (oldValue instanceof byte[]) {
-          oldValueIsObject = false;
-        } else if ((oldValue instanceof Token)
-            && clientVersion.isNotNewerThan(KnownVersion.GFE_651)) {
-          // older clients don't know that Token is now a DSFID class, so we
-          // put the token in a serialized form they can consume
-          try (HeapDataOutputStream str = new HeapDataOutputStream(KnownVersion.CURRENT)) {
-            DataOutput dstr = new DataOutputStream(str);
-            InternalDataSerializer.writeSerializableObject(oldValue, dstr);
-            oldValue = str.toByteArray();
-          }
-        }
-        result = true;
-        // } catch (Exception e) {
-        // writeException(msg, e, false, servConn);
-        // servConn.setAsTrue(RESPONDED);
-        // return;
-        // }
-
-      } else if (operation == Operation.REPLACE) {
-        // try {
-        if (requireOldValue) { // <V> replace(<K>, <V>)
-          if (clientMessage.isRetry() && clientEvent.isConcurrencyConflict()
-              && clientEvent.getVersionTag() != null) {
-            if (isDebugEnabled) {
-              logger.debug("replace(k,v) operation was successful last time with version {}",
-                  clientEvent.getVersionTag());
-            }
-          }
-          oldValue = region.basicBridgeReplace(key, value, isObject, callbackArg,
-              serverConnection.getProxyID(), true, clientEvent);
-          sendOldValue = !clientEvent.isConcurrencyConflict();
-          oldValueIsObject = true;
-          KnownVersion clientVersion = serverConnection.getClientVersion();
-          if (oldValue instanceof CachedDeserializable) {
-            oldValue = ((CachedDeserializable) oldValue).getSerializedValue();
-          } else if (oldValue instanceof byte[]) {
-            oldValueIsObject = false;
-          } else if ((oldValue instanceof Token)
-              && clientVersion.isNotNewerThan(KnownVersion.GFE_651)) {
-            // older clients don't know that Token is now a DSFID class, so we
-            // put the token in a serialized form they can consume
-            try (HeapDataOutputStream str = new HeapDataOutputStream(KnownVersion.CURRENT)) {
-              DataOutput dstr = new DataOutputStream(str);
-              InternalDataSerializer.writeSerializableObject(oldValue, dstr);
-              oldValue = str.toByteArray();
-            }
-          }
-          if (isDebugEnabled) {
-            logger.debug("returning {} from replace(K,V)", oldValue);
-          }
-          result = true;
-        } else { // boolean replace(<K>, <V>, <V>) {
-          boolean didPut;
-          didPut = region.basicBridgeReplace(key, expectedOldValue, value, isObject, callbackArg,
-              serverConnection.getProxyID(), true, clientEvent);
-          if (clientMessage.isRetry() && clientEvent.getVersionTag() != null) {
-            if (isDebugEnabled) {
-              logger.debug("replace(k,v,v) operation was successful last time with version {}",
-                  clientEvent.getVersionTag());
-            }
-            didPut = true;
-          }
-          sendOldValue = true;
-          oldValueIsObject = true;
-          oldValue = didPut ? Boolean.TRUE : Boolean.FALSE;
-          if (isDebugEnabled) {
-            logger.debug("returning {} from replace(K,V,V)", oldValue);
-          }
-          result = true;
-        }
-        // } catch (Exception e) {
-        // writeException(msg, e, false, servConn);
-        // servConn.setAsTrue(RESPONDED);
-        // return;
-        // }
-
-      } else if (value == null && !isDelta) {
-        // Create the null entry. Since the value is null, the value of the
-        // isObject
-        // the true after null doesn't matter and is not used.
-        result = region.basicBridgeCreate(key, null, true, callbackArg,
-            serverConnection.getProxyID(), true, clientEvent, false);
-        if (clientMessage.isRetry() && clientEvent.isConcurrencyConflict()
-            && clientEvent.getVersionTag() != null) {
-          result = true;
-          if (isDebugEnabled) {
-            logger.debug("create(k,null) operation was successful last time with version {}",
-                clientEvent.getVersionTag());
-          }
-        }
-      } else {
-        // Put the entry
-        byte[] delta = null;
-        if (isDelta) {
-          delta = valuePart.getSerializedForm();
-        }
-        TXManagerImpl txMgr =
-            (TXManagerImpl) serverConnection.getCache().getCacheTransactionManager();
-        // bug 43068 - use create() if in a transaction and op is CREATE
-        if (txMgr.getTXState() != null && operation.isCreate()) {
-          result = region.basicBridgeCreate(key, (byte[]) value, isObject, callbackArg,
-              serverConnection.getProxyID(), true, clientEvent, true);
-        } else {
-          result = region.basicBridgePut(key, value, delta, isObject, callbackArg,
-              serverConnection.getProxyID(), true, clientEvent);
-        }
-        if (clientMessage.isRetry() && clientEvent.isConcurrencyConflict()
-            && clientEvent.getVersionTag() != null) {
-          if (isDebugEnabled) {
-            logger.debug("put(k,v) operation was successful last time with version {}",
-                clientEvent.getVersionTag());
-          }
-          result = true;
-        }
-      }
-      if (result) {
-        serverConnection.setModificationInfo(true, regionName, key);
-      } else {
-        String message = serverConnection.getName() + ": Failed to put entry for region "
-            + regionName + " key " + key + " value " + valuePart;
-        if (isDebugEnabled) {
-          logger.debug(message);
-        }
-        throw new Exception(message);
-      }
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (ResourceException re) {
-      writeException(clientMessage, re, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (InvalidDeltaException ide) {
-      logger.info("Error applying delta for key {} of region {}: {}",
-          new Object[] {key, regionName, ide.getMessage()});
-      writeException(clientMessage, MessageType.PUT_DELTA_ERROR, ide, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      region.getCachePerfStats().incDeltaFullValuesRequested();
-      return;
-    } catch (Exception ce) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ce);
-
-      // If an exception occurs during the put, preserve the connection
-      writeException(clientMessage, ce, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      if (ce instanceof GemFireSecurityException) {
-        // Fine logging for security exceptions since these are already
-        // logged by the security logger
-        if (isDebugEnabled) {
-          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), ce);
-        }
-      } else if (isDebugEnabled) {
-        logger.debug("{}: Unexpected Exception", serverConnection.getName(), ce);
-      }
-      return;
-    } finally {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessPutTime(start - oldStart);
-    }
-
-    // Increment statistics and write the reply
-    if (region instanceof PartitionedRegion) {
-      PartitionedRegion pr = (PartitionedRegion) region;
-      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, sendOldValue,
-            oldValueIsObject, oldValue, pr.getNetworkHopType(), clientEvent.getVersionTag());
-        pr.clearNetworkHopData();
-      } else {
-        writeReply(clientMessage, serverConnection, sendOldValue, oldValueIsObject, oldValue,
-            clientEvent.getVersionTag());
-      }
-    } else {
-      writeReply(clientMessage, serverConnection, sendOldValue, oldValueIsObject, oldValue,
-          clientEvent.getVersionTag());
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    if (isDebugEnabled) {
-      logger.debug("{}: Sent put response back to {} for region {} key {} value {}",
-          serverConnection.getName(), serverConnection.getSocketString(), regionName, key,
-          valuePart);
-    }
-    stats.incWritePutResponseTime(DistributionStats.getStatTime() - start);
-
-
-  }
-
-  boolean shouldSetPossibleDuplicate(LocalRegion region, EventIDHolder clientEvent) {
-    boolean shouldSetPossibleDuplicate = true;
-    if (region.getAttributes().getConcurrencyChecksEnabled()) {
-      // recover the version tag from other servers
-      clientEvent.setRegion(region);
-      boolean withPersistence = region.getAttributes().getDataPolicy().withPersistence();
-      if (!recoverVersionTagForRetriedOperation(clientEvent) && !withPersistence) {
-        // For persistent region, it is possible that all persistent copies went offline.
-        // Do not reset possible duplicate in this case, as persistent data
-        // can be recovered during the retry after recover of version tag failed.
-        shouldSetPossibleDuplicate = false; // no-one has seen this event
-      }
-    }
-    return shouldSetPossibleDuplicate;
-  }
-
-  protected void writeReply(Message origMsg, ServerConnection servConn, boolean sendOldValue,
-      boolean oldValueIsObject, Object oldValue, VersionTag tag) throws IOException {
-    Message replyMsg = servConn.getReplyMessage();
-    servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
-    replyMsg.setMessageType(MessageType.REPLY);
-    replyMsg.setNumberOfParts(sendOldValue ? 3 : 1);
-    replyMsg.setTransactionId(origMsg.getTransactionId());
-    replyMsg.addBytesPart(okBytes());
-    if (sendOldValue) {
-      replyMsg.addIntPart(oldValueIsObject ? 1 : 0);
-      replyMsg.addObjPart(oldValue);
-    }
-    replyMsg.send(servConn);
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: rpl tx: {} parts={}", servConn.getName(), origMsg.getTransactionId(),
-          replyMsg.getNumberOfParts());
-    }
-  }
-
-  protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection servConn,
-      PartitionedRegion pr, boolean sendOldValue, boolean oldValueIsObject, Object oldValue,
-      byte nwHopType, VersionTag tag) throws IOException {
-    Message replyMsg = servConn.getReplyMessage();
-    servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
-    replyMsg.setMessageType(MessageType.REPLY);
-    replyMsg.setNumberOfParts(sendOldValue ? 3 : 1);
-    replyMsg.setTransactionId(origMsg.getTransactionId());
-    replyMsg.addBytesPart(new byte[] {pr.getMetadataVersion(), nwHopType});
-    if (sendOldValue) {
-      replyMsg.addIntPart(oldValueIsObject ? 1 : 0);
-      replyMsg.addObjPart(oldValue);
-    }
-    replyMsg.send(servConn);
-    pr.getPrStats().incPRMetaDataSentCount();
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: rpl with REFRESH_METADATA tx: {} parts={}", servConn.getName(),
-          origMsg.getTransactionId(), replyMsg.getNumberOfParts());
-    }
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put70.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put70.java
index da63f79..4b96f75 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put70.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Put70.java
@@ -14,18 +14,45 @@
  */
 package org.apache.geode.internal.cache.tier.sockets.command;
 
+import java.io.DataOutput;
+import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
+import org.apache.geode.InvalidDeltaException;
 import org.apache.geode.annotations.Immutable;
+import org.apache.geode.cache.DynamicRegionFactory;
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.RegionDestroyedException;
+import org.apache.geode.cache.ResourceException;
 import org.apache.geode.cache.client.internal.PutOp;
+import org.apache.geode.cache.operations.PutOperationContext;
+import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.internal.HeapDataOutputStream;
+import org.apache.geode.internal.InternalDataSerializer;
+import org.apache.geode.internal.cache.CachedDeserializable;
+import org.apache.geode.internal.cache.EventID;
+import org.apache.geode.internal.cache.EventIDHolder;
+import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
+import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
 import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.cache.versions.VersionTag;
+import org.apache.geode.internal.security.AuthorizeRequest;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.util.Breadcrumbs;
+import org.apache.geode.security.GemFireSecurityException;
+import org.apache.geode.security.ResourcePermission;
 
-public class Put70 extends Put65 {
+public class Put70 extends BaseCommand {
 
   @Immutable
   private static final Put70 singleton = new Put70();
@@ -37,8 +64,446 @@
   private Put70() {}
 
   @Override
+  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
+      final SecurityService securityService, long p_start)
+      throws IOException, InterruptedException {
+    long start = p_start;
+    final CacheServerStats stats = serverConnection.getCacheServerStats();
+
+    // requiresResponse = true;
+    serverConnection.setAsTrue(REQUIRES_RESPONSE);
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incReadPutRequestTime(start - oldStart);
+    }
+    // Retrieve the data from the message parts
+    int idx = 0;
+
+    final Part regionNamePart = clientMessage.getPart(idx++);
+
+    final Operation operation;
+    try {
+      final Part operationPart = clientMessage.getPart(idx++);
+
+      if (operationPart.isBytes()) {
+        final byte[] bytes = operationPart.getSerializedForm();
+        if (null == bytes || 0 == bytes.length) {
+          // older clients can send empty bytes for default operation.
+          operation = Operation.UPDATE;
+        } else {
+          operation = Operation.fromOrdinal(bytes[0]);
+        }
+      } else {
+
+        // Fallback for older clients.
+        if (operationPart.getObject() == null) {
+          // native clients may send a null since the op is java-serialized.
+          operation = Operation.UPDATE;
+        } else {
+          operation = (Operation) operationPart.getObject();
+        }
+      }
+    } catch (Exception e) {
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    final int flags = clientMessage.getPart(idx++).getInt();
+    final boolean requireOldValue = ((flags & 0x01) == 0x01);
+    final boolean haveExpectedOldValue = ((flags & 0x02) == 0x02);
+    final Object expectedOldValue;
+    if (haveExpectedOldValue) {
+      try {
+        expectedOldValue = clientMessage.getPart(idx++).getObject();
+      } catch (ClassNotFoundException e) {
+        writeException(clientMessage, e, false, serverConnection);
+        serverConnection.setAsTrue(RESPONDED);
+        return;
+      }
+    } else {
+      expectedOldValue = null;
+    }
+
+    final Part keyPart = clientMessage.getPart(idx++);
+
+    final boolean isDelta;
+    try {
+      isDelta = ((Boolean) clientMessage.getPart(idx).getObject());
+      idx += 1;
+    } catch (Exception e) {
+      writeException(clientMessage, MessageType.PUT_DELTA_ERROR, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      // CachePerfStats not available here.
+      return;
+    }
+
+    final Part valuePart = clientMessage.getPart(idx++);
+    final Part eventPart = clientMessage.getPart(idx++);
+
+    Object callbackArg = null;
+    if (clientMessage.getNumberOfParts() > idx) {
+      final Part callbackArgPart = clientMessage.getPart(idx++);
+      try {
+        callbackArg = callbackArgPart.getObject();
+      } catch (Exception e) {
+        writeException(clientMessage, e, false, serverConnection);
+        serverConnection.setAsTrue(RESPONDED);
+        return;
+      }
+    }
+
+    final Object key;
+    try {
+      key = keyPart.getStringOrObject();
+    } catch (Exception e) {
+      writeException(clientMessage, e, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    final String regionName = regionNamePart.getCachedString();
+
+    final boolean isDebugEnabled = logger.isDebugEnabled();
+    if (isDebugEnabled) {
+      logger.debug(
+          "{}: Received {}put request ({} bytes) from {} for region {} key {} txId {} posdup: {}",
+          serverConnection.getName(), (isDelta ? " delta " : " "), clientMessage.getPayloadLength(),
+          serverConnection.getSocketString(), regionName, key, clientMessage.getTransactionId(),
+          clientMessage.isRetry());
+    }
+
+    // Process the put request
+    if (key == null || regionName == null) {
+      final StringBuilder errMessage = new StringBuilder();
+      if (key == null) {
+        final String putMsg = " The input key for the put request is null";
+        if (isDebugEnabled) {
+          logger.debug("{}:{}", serverConnection.getName(), putMsg);
+        }
+        errMessage.append(putMsg);
+      }
+      if (regionName == null) {
+        final String putMsg = " The input region name for the put request is null";
+        if (isDebugEnabled) {
+          logger.debug("{}:{}", serverConnection.getName(), putMsg);
+        }
+        errMessage.append(putMsg);
+      }
+      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
+          serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    final LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
+    if (region == null) {
+      final String reason = " was not found during put request";
+      writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    if (valuePart.isNull() && operation != Operation.PUT_IF_ABSENT && region.containsKey(key)) {
+      // Invalid to 'put' a null value in an existing key
+      final String putMsg = " Attempted to put a null value for existing key " + key;
+      if (isDebugEnabled) {
+        logger.debug("{}:{}", serverConnection.getName(), putMsg);
+      }
+      writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, putMsg,
+          serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    }
+
+    final ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    final long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    final long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+
+    final EventIDHolder clientEvent = new EventIDHolder(
+        new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId));
+
+    Breadcrumbs.setEventId(clientEvent.getEventId());
+
+    // msg.isRetry might be set by v7.0 and later clients
+    if (clientMessage.isRetry()) {
+      if (shouldSetPossibleDuplicate(region, clientEvent)) {
+        clientEvent.setPossibleDuplicate(true);
+      }
+    }
+
+    boolean sendOldValue = false;
+    boolean oldValueIsObject = true;
+    Object oldValue = null;
+
+    try {
+      Object value = null;
+      if (!isDelta) {
+        value = valuePart.getSerializedForm();
+      }
+      boolean isObject = valuePart.isObject();
+      boolean isMetaRegion = region.isUsedForMetaRegion();
+      clientMessage.setMetaRegion(isMetaRegion);
+
+      securityService.authorize(ResourcePermission.Resource.DATA,
+          ResourcePermission.Operation.WRITE, regionName,
+          key);
+
+      AuthorizeRequest authzRequest = null;
+      if (!isMetaRegion) {
+        authzRequest = serverConnection.getAuthzRequest();
+      }
+      if (authzRequest != null) {
+        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          authzRequest.createRegionAuthorize((String) key);
+        }
+        // Allow PUT operations on meta regions (bug #38961)
+        else {
+          PutOperationContext putContext =
+              authzRequest.putAuthorize(regionName, key, value, isObject, callbackArg);
+          value = putContext.getValue();
+          isObject = putContext.isObject();
+          callbackArg = putContext.getCallbackArg();
+        }
+      }
+      if (isDebugEnabled) {
+        logger.debug("processing put65 with operation={}", operation);
+      }
+
+      // If the value is 1 byte and the byte represents null,
+      // attempt to create the entry. This test needs to be
+      // moved to DataSerializer or DataSerializer.NULL needs
+      // to be publicly accessible.
+      boolean result;
+      if (operation == Operation.PUT_IF_ABSENT) {
+        // try {
+        if (clientMessage.isRetry() && clientEvent.getVersionTag() != null) {
+          // bug #46590 the operation was successful the last time since it
+          // was applied to the cache, so return success and the recovered
+          // version tag
+          if (isDebugEnabled) {
+            logger.debug("putIfAbsent operation was successful last time with version {}",
+                clientEvent.getVersionTag());
+          }
+          // invoke basicBridgePutIfAbsent anyway to ensure that the event is distributed to all
+          // servers - bug #51664
+          region.basicBridgePutIfAbsent(key, value, isObject, callbackArg,
+              serverConnection.getProxyID(), true, clientEvent);
+          oldValue = null;
+        } else {
+          oldValue = region.basicBridgePutIfAbsent(key, value, isObject, callbackArg,
+              serverConnection.getProxyID(), true, clientEvent);
+        }
+        sendOldValue = true;
+        oldValueIsObject = true;
+        KnownVersion clientVersion = serverConnection.getClientVersion();
+        if (oldValue instanceof CachedDeserializable) {
+          oldValue = ((CachedDeserializable) oldValue).getSerializedValue();
+        } else if (oldValue instanceof byte[]) {
+          oldValueIsObject = false;
+        } else if ((oldValue instanceof Token)
+            && clientVersion.isNotNewerThan(KnownVersion.GFE_651)) {
+          // older clients don't know that Token is now a DSFID class, so we
+          // put the token in a serialized form they can consume
+          try (HeapDataOutputStream str = new HeapDataOutputStream(KnownVersion.CURRENT)) {
+            DataOutput dstr = new DataOutputStream(str);
+            InternalDataSerializer.writeSerializableObject(oldValue, dstr);
+            oldValue = str.toByteArray();
+          }
+        }
+        result = true;
+        // } catch (Exception e) {
+        // writeException(msg, e, false, servConn);
+        // servConn.setAsTrue(RESPONDED);
+        // return;
+        // }
+
+      } else if (operation == Operation.REPLACE) {
+        // try {
+        if (requireOldValue) { // <V> replace(<K>, <V>)
+          if (clientMessage.isRetry() && clientEvent.isConcurrencyConflict()
+              && clientEvent.getVersionTag() != null) {
+            if (isDebugEnabled) {
+              logger.debug("replace(k,v) operation was successful last time with version {}",
+                  clientEvent.getVersionTag());
+            }
+          }
+          oldValue = region.basicBridgeReplace(key, value, isObject, callbackArg,
+              serverConnection.getProxyID(), true, clientEvent);
+          sendOldValue = !clientEvent.isConcurrencyConflict();
+          oldValueIsObject = true;
+          KnownVersion clientVersion = serverConnection.getClientVersion();
+          if (oldValue instanceof CachedDeserializable) {
+            oldValue = ((CachedDeserializable) oldValue).getSerializedValue();
+          } else if (oldValue instanceof byte[]) {
+            oldValueIsObject = false;
+          } else if ((oldValue instanceof Token)
+              && clientVersion.isNotNewerThan(KnownVersion.GFE_651)) {
+            // older clients don't know that Token is now a DSFID class, so we
+            // put the token in a serialized form they can consume
+            try (HeapDataOutputStream str = new HeapDataOutputStream(KnownVersion.CURRENT)) {
+              DataOutput dstr = new DataOutputStream(str);
+              InternalDataSerializer.writeSerializableObject(oldValue, dstr);
+              oldValue = str.toByteArray();
+            }
+          }
+          if (isDebugEnabled) {
+            logger.debug("returning {} from replace(K,V)", oldValue);
+          }
+          result = true;
+        } else { // boolean replace(<K>, <V>, <V>) {
+          boolean didPut;
+          didPut = region.basicBridgeReplace(key, expectedOldValue, value, isObject, callbackArg,
+              serverConnection.getProxyID(), true, clientEvent);
+          if (clientMessage.isRetry() && clientEvent.getVersionTag() != null) {
+            if (isDebugEnabled) {
+              logger.debug("replace(k,v,v) operation was successful last time with version {}",
+                  clientEvent.getVersionTag());
+            }
+            didPut = true;
+          }
+          sendOldValue = true;
+          oldValueIsObject = true;
+          oldValue = didPut ? Boolean.TRUE : Boolean.FALSE;
+          if (isDebugEnabled) {
+            logger.debug("returning {} from replace(K,V,V)", oldValue);
+          }
+          result = true;
+        }
+        // } catch (Exception e) {
+        // writeException(msg, e, false, servConn);
+        // servConn.setAsTrue(RESPONDED);
+        // return;
+        // }
+
+      } else if (value == null && !isDelta) {
+        // Create the null entry. Since the value is null, the value of the
+        // isObject
+        // the true after null doesn't matter and is not used.
+        result = region.basicBridgeCreate(key, null, true, callbackArg,
+            serverConnection.getProxyID(), true, clientEvent, false);
+        if (clientMessage.isRetry() && clientEvent.isConcurrencyConflict()
+            && clientEvent.getVersionTag() != null) {
+          result = true;
+          if (isDebugEnabled) {
+            logger.debug("create(k,null) operation was successful last time with version {}",
+                clientEvent.getVersionTag());
+          }
+        }
+      } else {
+        // Put the entry
+        byte[] delta = null;
+        if (isDelta) {
+          delta = valuePart.getSerializedForm();
+        }
+        TXManagerImpl txMgr =
+            (TXManagerImpl) serverConnection.getCache().getCacheTransactionManager();
+        // bug 43068 - use create() if in a transaction and op is CREATE
+        if (txMgr.getTXState() != null && operation.isCreate()) {
+          result = region.basicBridgeCreate(key, (byte[]) value, isObject, callbackArg,
+              serverConnection.getProxyID(), true, clientEvent, true);
+        } else {
+          result = region.basicBridgePut(key, value, delta, isObject, callbackArg,
+              serverConnection.getProxyID(), true, clientEvent);
+        }
+        if (clientMessage.isRetry() && clientEvent.isConcurrencyConflict()
+            && clientEvent.getVersionTag() != null) {
+          if (isDebugEnabled) {
+            logger.debug("put(k,v) operation was successful last time with version {}",
+                clientEvent.getVersionTag());
+          }
+          result = true;
+        }
+      }
+      if (result) {
+        serverConnection.setModificationInfo(true, regionName, key);
+      } else {
+        String message = serverConnection.getName() + ": Failed to put entry for region "
+            + regionName + " key " + key + " value " + valuePart;
+        if (isDebugEnabled) {
+          logger.debug(message);
+        }
+        throw new Exception(message);
+      }
+    } catch (RegionDestroyedException | ResourceException rde) {
+      writeException(clientMessage, rde, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      return;
+    } catch (InvalidDeltaException ide) {
+      logger.info("Error applying delta for key {} of region {}: {}",
+          new Object[] {key, regionName, ide.getMessage()});
+      writeException(clientMessage, MessageType.PUT_DELTA_ERROR, ide, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      region.getCachePerfStats().incDeltaFullValuesRequested();
+      return;
+    } catch (Exception ce) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(serverConnection, ce);
+
+      // If an exception occurs during the put, preserve the connection
+      writeException(clientMessage, ce, false, serverConnection);
+      serverConnection.setAsTrue(RESPONDED);
+      if (ce instanceof GemFireSecurityException) {
+        // Fine logging for security exceptions since these are already
+        // logged by the security logger
+        if (isDebugEnabled) {
+          logger.debug("{}: Unexpected Security exception", serverConnection.getName(), ce);
+        }
+      } else if (isDebugEnabled) {
+        logger.debug("{}: Unexpected Exception", serverConnection.getName(), ce);
+      }
+      return;
+    } finally {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessPutTime(start - oldStart);
+    }
+
+    // Increment statistics and write the reply
+    if (region instanceof PartitionedRegion) {
+      PartitionedRegion pr = (PartitionedRegion) region;
+      if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
+        writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr, sendOldValue,
+            oldValueIsObject, oldValue, pr.getNetworkHopType(), clientEvent.getVersionTag());
+        pr.clearNetworkHopData();
+      } else {
+        writeReply(clientMessage, serverConnection, sendOldValue, oldValueIsObject, oldValue,
+            clientEvent.getVersionTag());
+      }
+    } else {
+      writeReply(clientMessage, serverConnection, sendOldValue, oldValueIsObject, oldValue,
+          clientEvent.getVersionTag());
+    }
+    serverConnection.setAsTrue(RESPONDED);
+    if (isDebugEnabled) {
+      logger.debug("{}: Sent put response back to {} for region {} key {} value {}",
+          serverConnection.getName(), serverConnection.getSocketString(), regionName, key,
+          valuePart);
+    }
+    stats.incWritePutResponseTime(DistributionStats.getStatTime() - start);
+
+
+  }
+
+  boolean shouldSetPossibleDuplicate(LocalRegion region, EventIDHolder clientEvent) {
+    boolean shouldSetPossibleDuplicate = true;
+    if (region.getAttributes().getConcurrencyChecksEnabled()) {
+      // recover the version tag from other servers
+      clientEvent.setRegion(region);
+      boolean withPersistence = region.getAttributes().getDataPolicy().withPersistence();
+      if (!recoverVersionTagForRetriedOperation(clientEvent) && !withPersistence) {
+        // For persistent region, it is possible that all persistent copies went offline.
+        // Do not reset possible duplicate in this case, as persistent data
+        // can be recovered during the retry after recover of version tag failed.
+        shouldSetPossibleDuplicate = false; // no-one has seen this event
+      }
+    }
+    return shouldSetPossibleDuplicate;
+  }
+
   protected void writeReply(Message origMsg, ServerConnection servConn, boolean sendOldValue,
-      boolean oldValueIsObject, Object oldValue, VersionTag versionTag) throws IOException {
+      boolean oldValueIsObject, Object oldValue, VersionTag<?> versionTag) throws IOException {
     Message replyMsg = servConn.getReplyMessage();
     servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
     replyMsg.setMessageType(MessageType.REPLY);
@@ -72,10 +537,9 @@
     }
   }
 
-  @Override
   protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection servConn,
       PartitionedRegion pr, boolean sendOldValue, boolean oldValueIsObject, Object oldValue,
-      byte nwHopType, VersionTag versionTag) throws IOException {
+      byte nwHopType, VersionTag<?> versionTag) throws IOException {
     Message replyMsg = servConn.getReplyMessage();
     servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
     replyMsg.setMessageType(MessageType.REPLY);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/PutAll.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/PutAll.java
deleted file mode 100644
index 11805a0..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/PutAll.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.ResourceException;
-import org.apache.geode.cache.operations.PutAllOperationContext;
-import org.apache.geode.cache.operations.internal.UpdateOnlyMap;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.CachedDeserializableFactory;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.PutAllPartialResultException;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.versions.VersionTag;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class PutAll extends BaseCommand {
-
-  @Immutable
-  private static final PutAll singleton = new PutAll();
-
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private PutAll() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart = null, numberOfKeysPart = null, keyPart = null, valuePart = null;
-    String regionName = null;
-    int numberOfKeys = 0;
-    Object key = null;
-    Part eventPart = null;
-    StringBuilder errMessage = new StringBuilder();
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-    boolean replyWithMetaData = false;
-
-    // requiresResponse = true;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadPutAllRequestTime(start - oldStart);
-    }
-
-    try {
-      // Retrieve the data from the message parts
-      // part 0: region name
-      regionNamePart = clientMessage.getPart(0);
-      regionName = regionNamePart.getCachedString();
-
-      if (regionName == null) {
-        String putAllMsg =
-            "The input region name for the putAll request is null";
-        logger.warn("{}: {}", serverConnection.getName(), putAllMsg);
-        errMessage.append(putAllMsg);
-        writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-      LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during put request";
-        writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-
-      // part 1: eventID
-      eventPart = clientMessage.getPart(1);
-      ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-      long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-      long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-      EventID eventId =
-          new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-
-      // part 2: number of keys
-      numberOfKeysPart = clientMessage.getPart(2);
-      numberOfKeys = numberOfKeysPart.getInt();
-
-      // building the map
-      Map map = new LinkedHashMap();
-      // Map isObjectMap = new LinkedHashMap();
-      for (int i = 0; i < numberOfKeys; i++) {
-        keyPart = clientMessage.getPart(3 + i * 2);
-        key = keyPart.getStringOrObject();
-        if (key == null) {
-          String putAllMsg =
-              "One of the input keys for the putAll request is null";
-          logger.warn("{}: {}", serverConnection.getName(), putAllMsg);
-          errMessage.append(putAllMsg);
-          writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-              serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-
-        valuePart = clientMessage.getPart(3 + i * 2 + 1);
-        if (valuePart.isNull()) {
-          String putAllMsg =
-              "One of the input values for the putAll request is null";
-          logger.warn("{}: {}", serverConnection.getName(), putAllMsg);
-          errMessage.append(putAllMsg);
-          writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-              serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-
-        // byte[] value = valuePart.getSerializedForm();
-        Object value;
-        if (valuePart.isObject()) {
-          value =
-              CachedDeserializableFactory.create(valuePart.getSerializedForm(), region.getCache());
-        } else {
-          value = valuePart.getSerializedForm();
-        }
-        // put serializedform for auth. It will be modified with auth callback
-        map.put(key, value);
-        // isObjectMap.put(key, new Boolean(isObject));
-      } // for
-
-      if (clientMessage.getNumberOfParts() == (3 + 2 * numberOfKeys + 1)) {// it means optional
-                                                                           // timeout has
-        // been added
-        int timeout = clientMessage.getPart(3 + 2 * numberOfKeys).getInt();
-        serverConnection.setRequestSpecificTimeout(timeout);
-      }
-
-      securityService.authorize(Resource.DATA, Operation.WRITE, regionName);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          authzRequest.createRegionAuthorize(regionName);
-        } else {
-          PutAllOperationContext putAllContext =
-              authzRequest.putAllAuthorize(regionName, map, null);
-          map = putAllContext.getMap();
-          if (map instanceof UpdateOnlyMap) {
-            map = ((UpdateOnlyMap) map).getInternalMap();
-          }
-        }
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Received putAll request ({} bytes) from {} for region {}",
-            serverConnection.getName(), clientMessage.getPayloadLength(),
-            serverConnection.getSocketString(), regionName);
-      }
-
-      region.basicBridgePutAll(map, Collections.<Object, VersionTag>emptyMap(),
-          serverConnection.getProxyID(), eventId, false, null);
-
-      if (region instanceof PartitionedRegion) {
-        PartitionedRegion pr = (PartitionedRegion) region;
-        if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-          writeReplyWithRefreshMetadata(clientMessage, serverConnection, pr,
-              pr.getNetworkHopType());
-          pr.clearNetworkHopData();
-          replyWithMetaData = true;
-        }
-      }
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (ResourceException re) {
-      writeException(clientMessage, re, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (PutAllPartialResultException pre) {
-      writeException(clientMessage, pre, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (Exception ce) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ce);
-
-      // If an exception occurs during the put, preserve the connection
-      writeException(clientMessage, ce, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      logger.warn(String.format("%s: Unexpected Exception",
-          serverConnection.getName()), ce);
-      return;
-    } finally {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessPutAllTime(start - oldStart);
-    }
-
-    // Increment statistics and write the reply
-    if (!replyWithMetaData) {
-      writeReply(clientMessage, serverConnection);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sent putAll response back to {} for region {}", serverConnection.getName(),
-          serverConnection.getSocketString(), regionName);
-    }
-    stats.incWritePutAllResponseTime(DistributionStats.getStatTime() - start);
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/PutAll70.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/PutAll70.java
deleted file mode 100644
index db19802..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/PutAll70.java
+++ /dev/null
@@ -1,356 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.ResourceException;
-import org.apache.geode.cache.operations.PutAllOperationContext;
-import org.apache.geode.cache.operations.internal.UpdateOnlyMap;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.CachedDeserializableFactory;
-import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.PutAllPartialResultException;
-import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.ha.ThreadIdentifier;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
-import org.apache.geode.internal.cache.versions.VersionTag;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class PutAll70 extends BaseCommand {
-
-  @Immutable
-  private static final PutAll70 singleton = new PutAll70();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private PutAll70() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long startp) throws IOException, InterruptedException {
-    long start = startp; // copy this since we need to modify it
-    Part regionNamePart = null, numberOfKeysPart = null, keyPart = null, valuePart = null;
-    String regionName = null;
-    int numberOfKeys = 0;
-    Object key = null;
-    Part eventPart = null;
-    boolean replyWithMetaData = false;
-    VersionedObjectList response = null;
-
-    StringBuilder errMessage = new StringBuilder();
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-
-    // requiresResponse = true;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadPutAllRequestTime(start - oldStart);
-    }
-
-    try {
-      // Retrieve the data from the message parts
-      // part 0: region name
-      regionNamePart = clientMessage.getPart(0);
-      regionName = regionNamePart.getCachedString();
-
-      if (regionName == null) {
-        String putAllMsg =
-            "The input region name for the putAll request is null";
-        logger.warn("{}: {}", serverConnection.getName(), putAllMsg);
-        errMessage.append(putAllMsg);
-        writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-      LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during put request";
-        writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-
-      // part 1: eventID
-      eventPart = clientMessage.getPart(1);
-      ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-      long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-      long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-      EventID eventId =
-          new EventID(serverConnection.getEventMemberIDByteArray(), threadId, sequenceId);
-
-      // part 2: invoke callbacks (used by import)
-      Part callbacksPart = clientMessage.getPart(2);
-      boolean skipCallbacks = callbacksPart.getInt() == 1 ? true : false;
-
-      // part 3: number of keys
-      numberOfKeysPart = clientMessage.getPart(3);
-      numberOfKeys = numberOfKeysPart.getInt();
-
-      // building the map
-      Map map = new LinkedHashMap();
-      Map<Object, VersionTag> retryVersions = new LinkedHashMap<Object, VersionTag>();
-      // Map isObjectMap = new LinkedHashMap();
-      for (int i = 0; i < numberOfKeys; i++) {
-        keyPart = clientMessage.getPart(4 + i * 2);
-        key = keyPart.getStringOrObject();
-        if (key == null) {
-          String putAllMsg =
-              "One of the input keys for the putAll request is null";
-          logger.warn("{}: {}", serverConnection.getName(), putAllMsg);
-          errMessage.append(putAllMsg);
-          writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-              serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-
-        valuePart = clientMessage.getPart(4 + i * 2 + 1);
-        if (valuePart.isNull()) {
-          String putAllMsg =
-              "One of the input values for the putAll request is null";
-          logger.warn("{}: {}", serverConnection.getName(), putAllMsg);
-          errMessage.append(putAllMsg);
-          writeErrorResponse(clientMessage, MessageType.PUT_DATA_ERROR, errMessage.toString(),
-              serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-
-        // byte[] value = valuePart.getSerializedForm();
-        Object value;
-        if (valuePart.isObject()) {
-          // We're shoe-horning support for invalidated entries into putAll
-          // here...however Token.INVALID cannot be wrapped in a DataSerializable.
-          // Also, this code is using skipCallbacks as an import flag. If we make
-          // skipCallbacks configurable this code will need to be updated.
-          if (skipCallbacks && Token.INVALID.isSerializedValue(valuePart.getSerializedForm())) {
-            value = Token.INVALID;
-          } else {
-            value = CachedDeserializableFactory.create(valuePart.getSerializedForm(),
-                region.getCache());
-          }
-        } else {
-          value = valuePart.getSerializedForm();
-        }
-        // put serializedform for auth. It will be modified with auth callback
-        if (clientMessage.isRetry()) {
-          // Constuct the thread id/sequence id information for this element in the
-          // put all map
-
-          // The sequence id is constructed from the base sequence id and the offset
-          EventID entryEventId = new EventID(eventId, i);
-
-          // For PRs, the thread id assigned as a fake thread id.
-          if (region instanceof PartitionedRegion) {
-            PartitionedRegion pr = (PartitionedRegion) region;
-            int bucketId = pr.getKeyInfo(key).getBucketId();
-            long entryThreadId =
-                ThreadIdentifier.createFakeThreadIDForBulkOp(bucketId, entryEventId.getThreadID());
-            entryEventId = new EventID(entryEventId.getMembershipID(), entryThreadId,
-                entryEventId.getSequenceID());
-          }
-
-          VersionTag tag = findVersionTagsForRetriedBulkOp(region, entryEventId);
-          if (tag != null) {
-            retryVersions.put(key, tag);
-          }
-          // FIND THE VERSION TAG FOR THIS KEY - but how? all we have is the
-          // putAll eventId, not individual eventIds for entries, right?
-        }
-        map.put(key, value);
-        // isObjectMap.put(key, new Boolean(isObject));
-      } // for
-
-      if (clientMessage.getNumberOfParts() == (4 + 2 * numberOfKeys + 1)) {// it means optional
-                                                                           // timeout has
-        // been added
-        int timeout = clientMessage.getPart(4 + 2 * numberOfKeys).getInt();
-        serverConnection.setRequestSpecificTimeout(timeout);
-      }
-
-      securityService.authorize(Resource.DATA, Operation.WRITE, regionName);
-
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          authzRequest.createRegionAuthorize(regionName);
-        } else {
-          PutAllOperationContext putAllContext =
-              authzRequest.putAllAuthorize(regionName, map, null);
-          map = putAllContext.getMap();
-          if (map instanceof UpdateOnlyMap) {
-            map = ((UpdateOnlyMap) map).getInternalMap();
-          }
-        }
-      } else {
-        // no auth, so update the map based on isObjectMap here
-        /*
-         * Collection entries = map.entrySet(); Iterator iterator = entries.iterator(); Map.Entry
-         * mapEntry = null; while (iterator.hasNext()) { mapEntry = (Map.Entry)iterator.next();
-         * Object currkey = mapEntry.getKey(); byte[] serializedValue = (byte[])mapEntry.getValue();
-         * boolean isObject = ((Boolean)isObjectMap.get(currkey)).booleanValue(); if (isObject) {
-         * map.put(currkey, CachedDeserializableFactory.create(serializedValue)); } }
-         */
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Received putAll request ({} bytes) from {} for region {}",
-            serverConnection.getName(), clientMessage.getPayloadLength(),
-            serverConnection.getSocketString(), regionName);
-      }
-
-      response = region.basicBridgePutAll(map, retryVersions, serverConnection.getProxyID(),
-          eventId, skipCallbacks, null);
-      if (!region.getConcurrencyChecksEnabled()) {
-        // the client only needs this if versioning is being used
-        response = null;
-      }
-
-      if (region instanceof PartitionedRegion) {
-        PartitionedRegion pr = (PartitionedRegion) region;
-        if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-          writeReplyWithRefreshMetadata(clientMessage, response, serverConnection, pr,
-              pr.getNetworkHopType());
-          pr.clearNetworkHopData();
-          replyWithMetaData = true;
-        }
-      }
-    } catch (RegionDestroyedException rde) {
-      writeException(clientMessage, rde, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (ResourceException re) {
-      writeException(clientMessage, re, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (PutAllPartialResultException pre) {
-      writeException(clientMessage, pre, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    } catch (Exception ce) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ce);
-
-      // If an exception occurs during the put, preserve the connection
-      writeException(clientMessage, ce, false, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      // if (logger.fineEnabled()) {
-      logger.warn(String.format("%s: Unexpected Exception",
-          serverConnection.getName()), ce);
-      // }
-      return;
-    } finally {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incProcessPutAllTime(start - oldStart);
-    }
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending putAll70 response back to {} for region {}: {}",
-          serverConnection.getName(), serverConnection.getSocketString(), regionName, response);
-    }
-    // Starting in 7.0.1 we do not send the keys back
-    if (response != null
-        && KnownVersion.GFE_70.compareTo(serverConnection.getClientVersion()) < 0) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("setting putAll keys to null");
-      }
-      response.setKeys(null);
-    }
-
-    // Increment statistics and write the reply
-    if (!replyWithMetaData) {
-      writeReply(clientMessage, response, serverConnection);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-    stats.incWritePutAllResponseTime(DistributionStats.getStatTime() - start);
-  }
-
-  @Override
-  protected void writeReply(Message origMsg, ServerConnection serverConnection) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-
-  protected void writeReply(Message origMsg, VersionedObjectList response,
-      ServerConnection servConn) throws IOException {
-    Message replyMsg = servConn.getReplyMessage();
-    servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
-    replyMsg.setMessageType(MessageType.REPLY);
-    replyMsg.setNumberOfParts(2);
-    replyMsg.setTransactionId(origMsg.getTransactionId());
-    replyMsg.addBytesPart(okBytes());
-    if (response != null) {
-      response.clearObjects();
-      replyMsg.addObjPart(response);
-    }
-    replyMsg.send(servConn);
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: rpl tx: {}", servConn.getName(), origMsg.getTransactionId());
-    }
-  }
-
-  @Override
-  protected void writeReplyWithRefreshMetadata(Message origMsg, ServerConnection serverConnection,
-      PartitionedRegion pr, byte nwHop) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  private void writeReplyWithRefreshMetadata(Message origMsg, VersionedObjectList response,
-      ServerConnection servConn, PartitionedRegion pr, byte nwHop) throws IOException {
-    Message replyMsg = servConn.getReplyMessage();
-    servConn.getCache().getCancelCriterion().checkCancelInProgress(null);
-    replyMsg.setMessageType(MessageType.REPLY);
-    replyMsg.setNumberOfParts(2);
-    replyMsg.setTransactionId(origMsg.getTransactionId());
-    replyMsg.addBytesPart(new byte[] {pr.getMetadataVersion(), nwHop});
-    if (response != null) {
-      response.clearObjects();
-      replyMsg.addObjPart(response);
-    }
-    replyMsg.send(servConn);
-    pr.getPrStats().incPRMetaDataSentCount();
-    if (logger.isTraceEnabled()) {
-      logger.trace("{}: rpl with REFRESH_METADATA tx: {}", servConn.getName(),
-          origMsg.getTransactionId());
-    }
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterest.java
deleted file mode 100644
index cd4e0ad..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterest.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.InterestResultPolicy;
-import org.apache.geode.cache.operations.RegisterInterestOperationContext;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.InterestType;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class RegisterInterest extends BaseCommand {
-
-  @Immutable
-  private static final RegisterInterest singleton = new RegisterInterest();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  @Override
-  public void cmdExecute(Message clientMessage, ServerConnection serverConnection,
-      SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart, keyPart;
-    String regionName;
-    Object key;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-
-    // Retrieve the data from the message parts
-    regionNamePart = clientMessage.getPart(0);
-    InterestResultPolicy policy;
-    // Retrieve the interest type
-    int interestType = clientMessage.getPart(1).getInt();
-
-    // Retrieve the InterestResultPolicy
-    try {
-      policy = (InterestResultPolicy) clientMessage.getPart(2).getObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    boolean isDurable;
-    try {
-      Part durablePart = clientMessage.getPart(3);
-      byte[] durablePartBytes = (byte[]) durablePart.getObject();
-      isDurable = durablePartBytes[0] == 0x01;
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    // Retrieve the key
-    keyPart = clientMessage.getPart(4);
-    regionName = regionNamePart.getCachedString();
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    boolean sendUpdatesAsInvalidates = false;
-
-    // VJR: Check for a sixth part for client version 6.0.3 onwards for the
-    // time being until refactoring into a new command version.
-    if (clientMessage.getNumberOfParts() > 5) {
-      try {
-        Part notifyPart = clientMessage.getPart(5);
-        byte[] notifyPartBytes = (byte[]) notifyPart.getObject();
-        sendUpdatesAsInvalidates = notifyPartBytes[0] == 0x01;
-      } catch (Exception e) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received register interest request ({} bytes) from {} for region {} key {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key);
-    }
-
-    // Process the register interest request
-    if (key == null || regionName == null) {
-      String message = null;
-      if (key == null) {
-        message =
-            "The input key for the register interest request is null";
-      }
-      if (regionName == null) {
-        message =
-            "The input region name for the register interest request is null.";
-      }
-      logger.warn("{}: {}", serverConnection.getName(), message);
-      writeChunkedErrorResponse(clientMessage, MessageType.REGISTER_INTEREST_DATA_ERROR,
-          message, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    // input key not null
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      logger.info("{}: Region named {} was not found during register interest request.",
-          new Object[] {serverConnection.getName(), regionName});
-    }
-    try {
-      if (interestType == InterestType.REGULAR_EXPRESSION) {
-        securityService.authorize(Resource.DATA, Operation.READ, regionName);
-      } else {
-        securityService.authorize(Resource.DATA, Operation.READ, regionName, key);
-      }
-      AuthorizeRequest authorizeRequest = serverConnection.getAuthzRequest();
-      if (authorizeRequest != null) {
-        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          RegisterInterestOperationContext registerContext =
-              authorizeRequest.registerInterestAuthorize(regionName, key, interestType, policy);
-          key = registerContext.getKey();
-        }
-      }
-      serverConnection.getAcceptor().getCacheClientNotifier().registerClientInterest(regionName,
-          key, serverConnection.getProxyID(), interestType, isDurable, sendUpdatesAsInvalidates,
-          false, 0, true);
-    } catch (Exception e) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, e);
-      // Otherwise, write an exception message and continue
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    CacheClientProxy ccp = serverConnection.getAcceptor().getCacheClientNotifier()
-        .getClientProxy(serverConnection.getProxyID());
-    if (ccp == null) {
-      // fix for 37593
-      IOException ioException = new IOException(
-          "CacheClientProxy for this client is no longer on the server");
-      writeChunkedException(clientMessage, ioException, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    boolean isPrimary = ccp.isPrimary();
-    ChunkedMessage chunkedResponseMsg = serverConnection.getRegisterInterestResponseMessage();
-    if (!isPrimary) {
-      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
-      chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-      chunkedResponseMsg.setLastChunk(true);
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>",
-            serverConnection.getName(), regionName, key, chunkedResponseMsg);
-      }
-      chunkedResponseMsg.sendChunk(serverConnection);
-    } else { // isPrimary
-
-      // Send header which describes how many chunks will follow
-      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
-      chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-
-      // Send chunk response
-      try {
-        fillAndSendRegisterInterestResponseChunks(region, key, interestType, policy,
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      } catch (Exception e) {
-        writeChunkedException(clientMessage, e, serverConnection, chunkedResponseMsg);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "{}: Sent register interest response for region {} key {}",
-            serverConnection.getName(), regionName, key);
-      }
-    } // isPrimary
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList.java
deleted file mode 100644
index ce92bb7..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.InterestResultPolicy;
-import org.apache.geode.cache.operations.RegisterInterestOperationContext;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.InterestType;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class RegisterInterestList extends BaseCommand {
-
-  @Immutable
-  private static final RegisterInterestList singleton = new RegisterInterestList();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  RegisterInterestList() {}
-
-  @Override
-  public void cmdExecute(Message clientMessage, ServerConnection serverConnection,
-      SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart, keyPart, numberOfKeysPart;
-    String regionName;
-    Object key = null;
-    InterestResultPolicy policy;
-    List<Object> keys;
-    int numberOfKeys, partNumber;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-    ChunkedMessage chunkedResponseMsg = serverConnection.getRegisterInterestResponseMessage();
-
-    regionNamePart = clientMessage.getPart(0);
-    regionName = regionNamePart.getCachedString();
-
-    // Retrieve the InterestResultPolicy
-    try {
-      policy = (InterestResultPolicy) clientMessage.getPart(1).getObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    boolean isDurable;
-    try {
-      Part durablePart = clientMessage.getPart(2);
-      byte[] durablePartBytes = (byte[]) durablePart.getObject();
-      isDurable = durablePartBytes[0] == 0x01;
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    numberOfKeysPart = clientMessage.getPart(3);
-    numberOfKeys = numberOfKeysPart.getInt();
-
-    partNumber = 4;
-    keys = new ArrayList<>();
-    for (int i = 0; i < numberOfKeys; i++) {
-      keyPart = clientMessage.getPart(partNumber + i);
-      try {
-        key = keyPart.getStringOrObject();
-      } catch (Exception e) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-      keys.add(key);
-    }
-
-    boolean sendUpdatesAsInvalidates = false;
-
-    // VJR: Check for an extra part for client version 6.0.3 onwards for the
-    // time being until refactoring into a new command version.
-    if (clientMessage.getNumberOfParts() > (numberOfKeys + partNumber)) {
-      try {
-        Part notifyPart = clientMessage.getPart(numberOfKeys + partNumber);
-        byte[] notifyPartBytes = (byte[]) notifyPart.getObject();
-        sendUpdatesAsInvalidates = notifyPartBytes[0] == 0x01;
-      } catch (Exception e) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    if (logger.isDebugEnabled()) {
-      logger.debug(
-          "{}: Received register interest request ({} bytes) from {} for the following {} keys in region {}: {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), numberOfKeys, regionName, keys);
-    }
-
-    // Process the register interest request
-    if (keys.isEmpty() || regionName == null) {
-      String errMessage = null;
-      if (keys.isEmpty() && regionName == null) {
-        errMessage =
-            "The input list of keys is empty and the input region name is null for the register interest request.";
-      } else if (keys.isEmpty()) {
-        errMessage =
-            "The input list of keys for the register interest request is empty.";
-      } else if (regionName == null) {
-        errMessage =
-            "The input region name for the register interest request is null.";
-      }
-      logger.warn("{}: {}", serverConnection.getName(), errMessage);
-      writeChunkedErrorResponse(clientMessage, MessageType.REGISTER_INTEREST_DATA_ERROR, errMessage,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      logger.info("{}: Region named {} was not found during register interest list request.",
-          new Object[] {serverConnection.getName(), regionName});
-    }
-    try {
-      securityService.authorize(Resource.DATA, Operation.READ, regionName);
-      AuthorizeRequest authorizeRequest = serverConnection.getAuthzRequest();
-      if (authorizeRequest != null) {
-        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          RegisterInterestOperationContext registerContext =
-              authorizeRequest.registerInterestListAuthorize(regionName, keys, policy);
-          keys = (List<Object>) registerContext.getKey();
-        }
-      }
-      // Register interest
-      serverConnection.getAcceptor().getCacheClientNotifier().registerClientInterest(regionName,
-          keys, serverConnection.getProxyID(), isDurable, sendUpdatesAsInvalidates, false, 0, true);
-    } catch (Exception ex) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ex);
-      // Otherwise, write an exception message and continue
-      writeChunkedException(clientMessage, ex, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-
-    CacheClientProxy ccp = serverConnection.getAcceptor().getCacheClientNotifier()
-        .getClientProxy(serverConnection.getProxyID());
-
-    if (ccp == null) {
-      IOException ioException = new IOException(
-          "CacheClientProxy for this client is no longer on the server");
-      writeChunkedException(clientMessage, ioException, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    boolean isPrimary = ccp.isPrimary();
-
-    if (!isPrimary) {
-      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
-      chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-      chunkedResponseMsg.setLastChunk(true);
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>",
-            serverConnection.getName(), regionName, key, chunkedResponseMsg);
-      }
-      chunkedResponseMsg.sendChunk(serverConnection);
-    } else { // isPrimary
-      // Send header which describes how many chunks will follow
-      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
-      chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-
-      // Send chunk response
-      try {
-        fillAndSendRegisterInterestResponseChunks(region, keys, InterestType.KEY, policy,
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      } catch (Exception e) {
-        checkForInterrupt(serverConnection, e);
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "{}: Sent register interest response for the following {} keys in region {}: {}",
-            serverConnection.getName(), numberOfKeys, regionName, keys);
-      }
-
-    } // isPrimary
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList61.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList61.java
deleted file mode 100644
index 67f9f76..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList61.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.DynamicRegionFactory;
-import org.apache.geode.cache.InterestResultPolicy;
-import org.apache.geode.cache.operations.RegisterInterestOperationContext;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.InterestType;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-/**
- * @since GemFire 6.1
- */
-public class RegisterInterestList61 extends BaseCommand {
-
-  @Immutable
-  private static final RegisterInterestList61 singleton = new RegisterInterestList61();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  RegisterInterestList61() {}
-
-  @Override
-  public void cmdExecute(Message clientMessage, ServerConnection serverConnection,
-      SecurityService securityService, long start) throws IOException, InterruptedException {
-    Part regionNamePart, keyPart, numberOfKeysPart;
-    String regionName;
-    Object key = null;
-    InterestResultPolicy policy;
-    List<Object> keys;
-    int numberOfKeys, partNumber;
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-    ChunkedMessage chunkedResponseMsg = serverConnection.getRegisterInterestResponseMessage();
-
-    regionNamePart = clientMessage.getPart(0);
-    regionName = regionNamePart.getCachedString();
-
-    // Retrieve the InterestResultPolicy
-    try {
-      policy = (InterestResultPolicy) clientMessage.getPart(1).getObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    boolean isDurable;
-    try {
-      Part durablePart = clientMessage.getPart(2);
-      byte[] durablePartBytes = (byte[]) durablePart.getObject();
-      isDurable = durablePartBytes[0] == 0x01;
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    // region data policy
-    byte[] regionDataPolicyPartBytes;
-    try {
-      Part regionDataPolicyPart = clientMessage.getPart(clientMessage.getNumberOfParts() - 1);
-      regionDataPolicyPartBytes = (byte[]) regionDataPolicyPart.getObject();
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    numberOfKeysPart = clientMessage.getPart(3);
-    numberOfKeys = numberOfKeysPart.getInt();
-
-    partNumber = 4;
-    keys = new ArrayList<>();
-    for (int i = 0; i < numberOfKeys; i++) {
-      keyPart = clientMessage.getPart(partNumber + i);
-      try {
-        key = keyPart.getStringOrObject();
-      } catch (Exception e) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-      keys.add(key);
-    }
-
-    boolean sendUpdatesAsInvalidates = false;
-
-    // VJR: Check for an extra part for client version 6.0.3 onwards for the
-    // time being until refactoring into a new command version.
-    if (clientMessage.getNumberOfParts() > (numberOfKeys + partNumber)) {
-      try {
-        Part notifyPart = clientMessage.getPart(numberOfKeys + partNumber);
-        byte[] notifyPartBytes = (byte[]) notifyPart.getObject();
-        sendUpdatesAsInvalidates = notifyPartBytes[0] == 0x01;
-      } catch (Exception e) {
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-
-    if (logger.isDebugEnabled()) {
-      logger.debug(
-          "{}: Received register interest 61 request ({} bytes) from {} for the following {} keys in region {}: {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), numberOfKeys, regionName, keys);
-    }
-
-    // Process the register interest request
-    if (keys.isEmpty() || regionName == null) {
-      String errMessage = null;
-      if (keys.isEmpty() && regionName == null) {
-        errMessage =
-            "The input list of keys is empty and the input region name is null for the register interest request.";
-      } else if (keys.isEmpty()) {
-        errMessage =
-            "The input list of keys for the register interest request is empty.";
-      } else if (regionName == null) {
-        errMessage =
-            "The input region name for the register interest request is null.";
-      }
-      logger.warn("{}: {}", serverConnection.getName(), errMessage);
-      writeChunkedErrorResponse(clientMessage, MessageType.REGISTER_INTEREST_DATA_ERROR, errMessage,
-          serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    LocalRegion region = (LocalRegion) serverConnection.getCache().getRegion(regionName);
-    if (region == null) {
-      logger.info("{}: Region named {} was not found during register interest list request.",
-          new Object[] {serverConnection.getName(), regionName});
-    }
-    try {
-      securityService.authorize(Resource.DATA, Operation.READ, regionName);
-      AuthorizeRequest authorizeRequest = serverConnection.getAuthzRequest();
-      if (authorizeRequest != null) {
-        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          RegisterInterestOperationContext registerContext =
-              authorizeRequest.registerInterestListAuthorize(regionName, keys, policy);
-          keys = (List<Object>) registerContext.getKey();
-        }
-      }
-      // Register interest
-      serverConnection.getAcceptor().getCacheClientNotifier().registerClientInterest(regionName,
-          keys, serverConnection.getProxyID(), isDurable, sendUpdatesAsInvalidates, true,
-          regionDataPolicyPartBytes[0], true);
-    } catch (Exception ex) {
-      // If an interrupted exception is thrown , rethrow it
-      checkForInterrupt(serverConnection, ex);
-      // Otherwise, write an exception message and continue
-      writeChunkedException(clientMessage, ex, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-
-    CacheClientProxy ccp = serverConnection.getAcceptor().getCacheClientNotifier()
-        .getClientProxy(serverConnection.getProxyID());
-
-    if (ccp == null) {
-      IOException ioException = new IOException(
-          "CacheClientProxy for this client is no longer on the server");
-      writeChunkedException(clientMessage, ioException, serverConnection);
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-
-    boolean isPrimary = ccp.isPrimary();
-
-    if (!isPrimary) {
-      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
-      chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-      chunkedResponseMsg.setLastChunk(true);
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>",
-            serverConnection.getName(), regionName, key, chunkedResponseMsg);
-      }
-      chunkedResponseMsg.sendChunk(serverConnection);
-    } else { // isPrimary
-      // Send header which describes how many chunks will follow
-      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
-      chunkedResponseMsg.setTransactionId(clientMessage.getTransactionId());
-      chunkedResponseMsg.sendHeader();
-
-      // Send chunk response
-      try {
-        fillAndSendRegisterInterestResponseChunks(region, keys, InterestType.KEY, policy,
-            serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      } catch (Exception e) {
-        checkForInterrupt(serverConnection, e);
-        writeChunkedException(clientMessage, e, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "{}: Sent register interest response for the following {} keys in region {}: {}",
-            serverConnection.getName(), numberOfKeys, regionName, keys);
-      }
-
-    } // isPrimary
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList66.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList66.java
index 1ab7417..c95a433 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList66.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList66.java
@@ -39,7 +39,7 @@
 
 /**
  * All keys of the register interest list are being sent as a single part since 6.6. There is no
- * need to send no keys as a separate part.In earlier versions {@link RegisterInterestList61} number
+ * need to send no keys as a separate part. In earlier versions the number
  * of keys & each individual key was sent as a separate part.
  *
  * @since GemFire 6.6
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Request.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Request.java
deleted file mode 100644
index d21d50a..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/Request.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import java.io.IOException;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.operations.GetOperationContext;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.CachedDeserializable;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.AuthorizeRequestPP;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class Request extends BaseCommand {
-
-  @Immutable
-  private static final Request singleton = new Request();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  Request() {}
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException {
-    Part regionNamePart = null, keyPart = null, valuePart = null;
-    String regionName = null;
-    Object callbackArg = null, key = null;
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-    String errMessage = null;
-
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    // requiresResponse = true;
-    {
-      long oldStart = start;
-      start = DistributionStats.getStatTime();
-      stats.incReadGetRequestTime(start - oldStart);
-    }
-    // Retrieve the data from the message parts
-    int parts = clientMessage.getNumberOfParts();
-    regionNamePart = clientMessage.getPart(0);
-    keyPart = clientMessage.getPart(1);
-    // valuePart = null; (redundant assignment)
-    if (parts > 2) {
-      valuePart = clientMessage.getPart(2);
-      try {
-        callbackArg = valuePart.getObject();
-      } catch (Exception e) {
-        writeException(clientMessage, e, false, serverConnection);
-        // responded = true;
-        serverConnection.setAsTrue(RESPONDED);
-        return;
-      }
-    }
-    regionName = regionNamePart.getCachedString();
-    try {
-      key = keyPart.getStringOrObject();
-    } catch (Exception e) {
-      writeException(clientMessage, e, false, serverConnection);
-      // responded = true;
-      serverConnection.setAsTrue(RESPONDED);
-      return;
-    }
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received get request ({} bytes) from {} for region {} key {} txId {}",
-          serverConnection.getName(), clientMessage.getPayloadLength(),
-          serverConnection.getSocketString(), regionName, key, clientMessage.getTransactionId());
-    }
-
-    // Process the get request
-    if (key == null || regionName == null) {
-      if ((key == null) && (regionName == null)) {
-        errMessage =
-            "The input region name and key for the get request are null.";
-      } else if (key == null) {
-        errMessage = "The input key for the get request is null.";
-      } else if (regionName == null) {
-        errMessage = "The input region name for the get request is null.";
-      }
-      logger.warn("{}: {}", serverConnection.getName(), errMessage);
-      writeErrorResponse(clientMessage, MessageType.REQUESTDATAERROR, errMessage, serverConnection);
-      // responded = true;
-      serverConnection.setAsTrue(RESPONDED);
-    } else {
-      Region region = serverConnection.getCache().getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during get request";
-        writeRegionDestroyedEx(clientMessage, regionName, reason, serverConnection);
-        serverConnection.setAsTrue(RESPONDED);
-      } else {
-
-        GetOperationContext getContext = null;
-
-        try {
-          securityService.authorize(Resource.DATA, Operation.READ, regionName, key);
-          AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-          if (authzRequest != null) {
-            getContext = authzRequest.getAuthorize(regionName, key, callbackArg);
-            callbackArg = getContext.getCallbackArg();
-          }
-        } catch (NotAuthorizedException ex) {
-          writeException(clientMessage, ex, false, serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-
-        // Get the value and update the statistics. Do not deserialize
-        // the value if it is a byte[].
-        Object[] valueAndIsObject = new Object[3];
-        try {
-          getValueAndIsObject(region, key, callbackArg, serverConnection, valueAndIsObject);
-        } catch (Exception e) {
-          writeException(clientMessage, e, false, serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-
-        Object data = valueAndIsObject[0];
-        boolean isObject = ((Boolean) valueAndIsObject[1]).booleanValue();
-
-
-
-        try {
-          AuthorizeRequestPP postAuthzRequest = serverConnection.getPostAuthzRequest();
-          if (postAuthzRequest != null) {
-            getContext = postAuthzRequest.getAuthorize(regionName, key, data, isObject, getContext);
-            byte[] serializedValue = getContext.getSerializedValue();
-            if (serializedValue == null) {
-              data = getContext.getObject();
-            } else {
-              data = serializedValue;
-            }
-            isObject = getContext.isObject();
-          }
-        } catch (NotAuthorizedException ex) {
-          writeException(clientMessage, ex, false, serverConnection);
-          serverConnection.setAsTrue(RESPONDED);
-          return;
-        }
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessGetTime(start - oldStart);
-        }
-
-        if (region instanceof PartitionedRegion) {
-          PartitionedRegion pr = (PartitionedRegion) region;
-          if (pr.getNetworkHopType() != PartitionedRegion.NETWORK_HOP_NONE) {
-            writeResponseWithRefreshMetadata(data, callbackArg, clientMessage, isObject,
-                serverConnection, pr, pr.getNetworkHopType());
-            pr.clearNetworkHopData();
-          } else {
-            writeResponse(data, callbackArg, clientMessage, isObject, serverConnection);
-          }
-        } else {
-          writeResponse(data, callbackArg, clientMessage, isObject, serverConnection);
-        }
-
-        serverConnection.setAsTrue(RESPONDED);
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Wrote get response back to {} for region {} key {} value: {}",
-              serverConnection.getName(), serverConnection.getSocketString(), regionName, key,
-              data);
-        }
-        stats.incWriteGetResponseTime(DistributionStats.getStatTime() - start);
-      }
-    }
-  }
-
-  // take the result 2 element "result" as argument instead of
-  // returning as the result to avoid creating the array repeatedly
-  // for large number of entries like in getAll
-  public void getValueAndIsObject(Region region, Object key, Object callbackArg,
-      ServerConnection servConn, Object[] result) {
-
-    String regionName = region.getFullPath();
-    if (servConn != null) {
-      servConn.setModificationInfo(true, regionName, key);
-    }
-
-    boolean isObject = true;
-    ClientProxyMembershipID id = servConn == null ? null : servConn.getProxyID();
-    Object data = ((LocalRegion) region).get(key, callbackArg, true, true, true, id, null, false);
-
-    // If the value in the VM is a CachedDeserializable,
-    // get its value. If it is Token.REMOVED, Token.DESTROYED,
-    // Token.INVALID, or Token.LOCAL_INVALID
-    // set it to null. If it is NOT_AVAILABLE, get the value from
-    // disk. If it is already a byte[], set isObject to false.
-    if (data instanceof CachedDeserializable) {
-      CachedDeserializable cd = (CachedDeserializable) data;
-      if (!cd.isSerialized()) {
-        // it is a byte[]
-        isObject = false;
-        data = cd.getDeserializedForReading();
-      } else {
-        data = cd.getValue();
-      }
-    } else if (data == Token.REMOVED_PHASE1 || data == Token.REMOVED_PHASE2
-        || data == Token.TOMBSTONE || data == Token.DESTROYED) {
-      data = null;
-    } else if (data == Token.INVALID || data == Token.LOCAL_INVALID) {
-      data = null; // fix for bug 35884
-    } else if (data instanceof byte[]) {
-      isObject = false;
-    }
-
-    result[0] = data;
-    result[1] = Boolean.valueOf(isObject);
-    result[2] = (data == null);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/cache/execute/FunctionAdapterJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/execute/FunctionAdapterJUnitTest.java
index e67ebaf..23f674f 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/execute/FunctionAdapterJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/execute/FunctionAdapterJUnitTest.java
@@ -16,6 +16,7 @@
 package org.apache.geode.cache.execute;
 
 import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -30,9 +31,9 @@
 import org.apache.geode.internal.serialization.KnownVersion;
 import org.apache.geode.internal.serialization.VersionedDataInputStream;
 
+@SuppressWarnings("deprecation") // Intentionally testing deprecated class.
 public class FunctionAdapterJUnitTest {
 
-  private static final long serialVersionUID = 1L;
   private FunctionAdapter adapter;
 
   @Before
@@ -64,7 +65,7 @@
   private static class MyFunctionAdapter extends FunctionAdapter {
 
     @Override
-    public void execute(final FunctionContext context) {}
+    public void execute(@SuppressWarnings("rawtypes") final FunctionContext context) {}
 
   }
 
@@ -77,18 +78,20 @@
                 + "serializedFunctionAdapterWithDifferentSerialVersionUID.ser").getAbsolutePath());
 
     DataInputStream dis =
-        new VersionedDataInputStream(new DataInputStream(fis), KnownVersion.GFE_82);
+        new VersionedDataInputStream(new DataInputStream(fis), KnownVersion.GFE_81);
     Object o = InternalDataSerializer.basicReadObject(dis);
     assertTrue(o instanceof FunctionAdapter);
   }
 
+  @SuppressWarnings("unused") // Used via deserialization
   private static class SomeFunction extends FunctionAdapter {
 
     private static final long serialVersionUID = -6417837315839543937L;
 
     @Override
-    public void execute(FunctionContext context) {
-      context.getResultSender().lastResult("S");
+    public void execute(@SuppressWarnings("rawtypes") FunctionContext context) {
+      final ResultSender<String> stringResultSender = uncheckedCast(context.getResultSender());
+      stringResultSender.lastResult("S");
     }
 
     @Override
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/BaseCommandJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/BaseCommandJUnitTest.java
index 719ae1a..354a2e4 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/BaseCommandJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/BaseCommandJUnitTest.java
@@ -21,31 +21,17 @@
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction61;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction65;
 import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction66;
 
-@RunWith(JUnitParamsRunner.class)
 public class BaseCommandJUnitTest {
 
-  public BaseCommand[] getCommands() {
-    return new BaseCommand[] {(BaseCommand) ExecuteRegionFunction61.getCommand(),
-        (BaseCommand) ExecuteRegionFunction65
-            .getCommand(),
-        (BaseCommand) ExecuteRegionFunction66.getCommand()};
-  }
+  private BaseCommand baseCommand = (BaseCommand) ExecuteRegionFunction66.getCommand();
 
   @Test
-  @Parameters(method = "getCommands")
-  public void whenLastReceivedIsSetThenCheckAndSetLastResultSentIfValidMustReturnTrue(
-      BaseCommand baseCommand) {
-
+  public void whenLastReceivedIsSetThenCheckAndSetLastResultSentIfValidMustReturnTrue() {
     ServerToClientFunctionResultSender resultSender =
         mock(ServerToClientFunctionResultSender.class);
     when(resultSender.isLastResultReceived()).thenReturn(true);
@@ -54,10 +40,7 @@
   }
 
   @Test
-  @Parameters(method = "getCommands")
-  public void whenLastReceivedIsNotSetThenCheckAndSetLastResultSentIfValidMustReturnFalse(
-      BaseCommand baseCommand) {
-
+  public void whenLastReceivedIsNotSetThenCheckAndSetLastResultSentIfValidMustReturnFalse() {
     ServerToClientFunctionResultSender resultSender =
         mock(ServerToClientFunctionResultSender.class);
     when(resultSender.isLastResultReceived()).thenReturn(false);
@@ -66,10 +49,7 @@
   }
 
   @Test
-  @Parameters(method = "getCommands")
-  public void whenLastReceivedIsNotSetThenCheckAndSetLastResultSentIfValidMustSetIt(
-      BaseCommand baseCommand) {
-
+  public void whenLastReceivedIsNotSetThenCheckAndSetLastResultSentIfValidMustSetIt() {
     ServerToClientFunctionResultSender resultSender =
         mock(ServerToClientFunctionResultSender.class);
     when(resultSender.isLastResultReceived()).thenReturn(false);
@@ -79,10 +59,7 @@
   }
 
   @Test
-  @Parameters(method = "getCommands")
-  public void whenLastReceivedIsSetThenCheckAndSetLastResultSentIfValidMustNotSetIt(
-      BaseCommand baseCommand) {
-
+  public void whenLastReceivedIsSetThenCheckAndSetLastResultSentIfValidMustNotSetIt() {
     ServerToClientFunctionResultSender resultSender =
         mock(ServerToClientFunctionResultSender.class);
     when(resultSender.isLastResultReceived()).thenReturn(true);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializerJUnitTest.java
deleted file mode 100644
index cf549a8..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializerJUnitTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets;
-
-import org.junit.Test;
-
-import org.apache.geode.internal.serialization.KnownVersion;
-
-public class CommandInitializerJUnitTest {
-
-  @Test
-  public void testCommandMapContainsAllVersions() {
-    for (KnownVersion version : KnownVersion.getAllVersions()) {
-      org.junit.Assert.assertNotNull(
-          "Please add a command set for " + version + " of Geode to CommandInitializer",
-          CommandInitializer.getCommands(version));
-    }
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializerTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializerTest.java
new file mode 100644
index 0000000..9fcbeb4
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CommandInitializerTest.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.internal.cache.tier.sockets;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import org.apache.geode.internal.cache.tier.Command;
+import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.sockets.command.AddPdxEnum;
+import org.apache.geode.internal.cache.tier.sockets.command.AddPdxType;
+import org.apache.geode.internal.cache.tier.sockets.command.ClearRegion;
+import org.apache.geode.internal.cache.tier.sockets.command.ClientReady;
+import org.apache.geode.internal.cache.tier.sockets.command.CloseConnection;
+import org.apache.geode.internal.cache.tier.sockets.command.CommitCommand;
+import org.apache.geode.internal.cache.tier.sockets.command.ContainsKey66;
+import org.apache.geode.internal.cache.tier.sockets.command.CreateRegion;
+import org.apache.geode.internal.cache.tier.sockets.command.Destroy70;
+import org.apache.geode.internal.cache.tier.sockets.command.DestroyRegion;
+import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction70;
+import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunction66;
+import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunctionGeode18;
+import org.apache.geode.internal.cache.tier.sockets.command.ExecuteRegionFunctionSingleHop;
+import org.apache.geode.internal.cache.tier.sockets.command.GatewayReceiverCommand;
+import org.apache.geode.internal.cache.tier.sockets.command.Get70;
+import org.apache.geode.internal.cache.tier.sockets.command.GetAll70;
+import org.apache.geode.internal.cache.tier.sockets.command.GetAllWithCallback;
+import org.apache.geode.internal.cache.tier.sockets.command.GetClientPRMetadataCommand66;
+import org.apache.geode.internal.cache.tier.sockets.command.GetClientPartitionAttributesCommand66;
+import org.apache.geode.internal.cache.tier.sockets.command.GetEntry70;
+import org.apache.geode.internal.cache.tier.sockets.command.GetFunctionAttribute;
+import org.apache.geode.internal.cache.tier.sockets.command.GetPDXEnumById;
+import org.apache.geode.internal.cache.tier.sockets.command.GetPDXIdForEnum;
+import org.apache.geode.internal.cache.tier.sockets.command.GetPDXIdForType;
+import org.apache.geode.internal.cache.tier.sockets.command.GetPDXTypeById;
+import org.apache.geode.internal.cache.tier.sockets.command.GetPdxEnums70;
+import org.apache.geode.internal.cache.tier.sockets.command.GetPdxTypes70;
+import org.apache.geode.internal.cache.tier.sockets.command.Invalid;
+import org.apache.geode.internal.cache.tier.sockets.command.Invalidate70;
+import org.apache.geode.internal.cache.tier.sockets.command.KeySet;
+import org.apache.geode.internal.cache.tier.sockets.command.MakePrimary;
+import org.apache.geode.internal.cache.tier.sockets.command.PeriodicAck;
+import org.apache.geode.internal.cache.tier.sockets.command.Ping;
+import org.apache.geode.internal.cache.tier.sockets.command.Put70;
+import org.apache.geode.internal.cache.tier.sockets.command.PutAll80;
+import org.apache.geode.internal.cache.tier.sockets.command.PutAllWithCallback;
+import org.apache.geode.internal.cache.tier.sockets.command.PutUserCredentials;
+import org.apache.geode.internal.cache.tier.sockets.command.Query651;
+import org.apache.geode.internal.cache.tier.sockets.command.QueryGeode10;
+import org.apache.geode.internal.cache.tier.sockets.command.QueryWithParametersGeode10;
+import org.apache.geode.internal.cache.tier.sockets.command.RegisterDataSerializers;
+import org.apache.geode.internal.cache.tier.sockets.command.RegisterInstantiators;
+import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterest61;
+import org.apache.geode.internal.cache.tier.sockets.command.RegisterInterestList66;
+import org.apache.geode.internal.cache.tier.sockets.command.RemoveAll;
+import org.apache.geode.internal.cache.tier.sockets.command.RemoveUserAuth;
+import org.apache.geode.internal.cache.tier.sockets.command.RequestEventValue;
+import org.apache.geode.internal.cache.tier.sockets.command.RollbackCommand;
+import org.apache.geode.internal.cache.tier.sockets.command.Size;
+import org.apache.geode.internal.cache.tier.sockets.command.TXFailoverCommand;
+import org.apache.geode.internal.cache.tier.sockets.command.TXSynchronizationCommand;
+import org.apache.geode.internal.cache.tier.sockets.command.UnregisterInterest;
+import org.apache.geode.internal.cache.tier.sockets.command.UnregisterInterestList;
+import org.apache.geode.internal.cache.tier.sockets.command.UpdateClientNotification;
+import org.apache.geode.internal.serialization.KnownVersion;
+
+public class CommandInitializerTest {
+
+  @Test
+  public void testCommandMapContainsAllVersions() {
+    for (KnownVersion version : KnownVersion.getAllVersions()) {
+      if (version.isNotOlderThan(KnownVersion.OLDEST)) {
+        org.junit.Assert.assertNotNull(
+            "Please add a command set for " + version + " of Geode to CommandInitializer",
+            CommandInitializer.getDefaultInstance().get(version));
+      }
+    }
+  }
+
+  @Test
+  public void initializeGeode18Commands() {
+    @SuppressWarnings("unchecked")
+    final Map<Integer, Command> commands = mock(Map.class);
+
+    CommandInitializer.initializeGeode18Commands(commands);
+
+    verify(commands).put(MessageType.EXECUTE_REGION_FUNCTION,
+        ExecuteRegionFunctionGeode18.getCommand());
+
+    verifyNoMoreInteractions(commands);
+  }
+
+  @Test
+  public void initializeGfe90Commands() {
+    @SuppressWarnings("unchecked")
+    final Map<Integer, Command> commands = mock(Map.class);
+
+    CommandInitializer.initializeGfe90Commands(commands);
+
+    verify(commands).put(MessageType.QUERY_WITH_PARAMETERS,
+        QueryWithParametersGeode10.getCommand());
+    verify(commands).put(MessageType.QUERY, QueryGeode10.getCommand());
+
+    verifyNoMoreInteractions(commands);
+  }
+
+  @Test
+  public void initializeGfe82Commands() {
+    @SuppressWarnings("unchecked")
+    final Map<Integer, Command> commands = mock(Map.class);
+
+    CommandInitializer.initializeGfe81Commands(commands);
+
+    verify(commands).put(MessageType.PING, Ping.getCommand());
+    verify(commands).put(MessageType.QUERY,
+        org.apache.geode.internal.cache.tier.sockets.command.Query.getCommand());
+    verify(commands).put(MessageType.CLEAR_REGION, ClearRegion.getCommand());
+    verify(commands).put(MessageType.DESTROY_REGION, DestroyRegion.getCommand());
+    verify(commands).put(MessageType.UNREGISTER_INTEREST, UnregisterInterest.getCommand());
+    verify(commands).put(MessageType.UNREGISTER_INTEREST_LIST, UnregisterInterestList.getCommand());
+    verify(commands).put(MessageType.KEY_SET, KeySet.getCommand());
+    verify(commands).put(MessageType.CREATE_REGION, CreateRegion.getCommand());
+    verify(commands).put(MessageType.MAKE_PRIMARY, MakePrimary.getCommand());
+    verify(commands).put(MessageType.PERIODIC_ACK, PeriodicAck.getCommand());
+    verify(commands).put(MessageType.REGISTER_INSTANTIATORS, RegisterInstantiators.getCommand());
+    verify(commands).put(MessageType.UPDATE_CLIENT_NOTIFICATION,
+        UpdateClientNotification.getCommand());
+    verify(commands).put(MessageType.CLOSE_CONNECTION, CloseConnection.getCommand());
+    verify(commands).put(MessageType.CLIENT_READY, ClientReady.getCommand());
+    verify(commands).put(MessageType.INVALID, Invalid.getCommand());
+
+    verify(commands).put(MessageType.REGISTER_INTEREST, RegisterInterest61.getCommand());
+    verify(commands).put(MessageType.REQUEST_EVENT_VALUE, RequestEventValue.getCommand());
+    verify(commands).put(MessageType.REGISTER_DATASERIALIZERS,
+        RegisterDataSerializers.getCommand());
+
+    verify(commands).put(MessageType.USER_CREDENTIAL_MESSAGE, PutUserCredentials.getCommand());
+    verify(commands).put(MessageType.REMOVE_USER_AUTH, RemoveUserAuth.getCommand());
+    verify(commands).put(MessageType.EXECUTE_REGION_FUNCTION_SINGLE_HOP,
+        ExecuteRegionFunctionSingleHop.getCommand());
+
+    verify(commands).put(MessageType.QUERY_WITH_PARAMETERS, Query651.getCommand());
+
+    verify(commands).put(MessageType.GET_CLIENT_PR_METADATA,
+        GetClientPRMetadataCommand66.getCommand());
+
+    verify(commands).put(MessageType.ADD_PDX_TYPE, AddPdxType.getCommand());
+    verify(commands).put(MessageType.GET_PDX_ID_FOR_TYPE, GetPDXIdForType.getCommand());
+    verify(commands).put(MessageType.GET_PDX_TYPE_BY_ID, GetPDXTypeById.getCommand());
+    verify(commands).put(MessageType.SIZE, Size.getCommand());
+    verify(commands).put(MessageType.COMMIT, CommitCommand.getCommand());
+    verify(commands).put(MessageType.ROLLBACK, RollbackCommand.getCommand());
+    verify(commands).put(MessageType.TX_FAILOVER, TXFailoverCommand.getCommand());
+    verify(commands).put(MessageType.TX_SYNCHRONIZATION, TXSynchronizationCommand.getCommand());
+    verify(commands).put(MessageType.GET_CLIENT_PARTITION_ATTRIBUTES,
+        GetClientPartitionAttributesCommand66.getCommand());
+    verify(commands).put(MessageType.REGISTER_INTEREST_LIST, RegisterInterestList66.getCommand());
+    verify(commands).put(MessageType.GET_FUNCTION_ATTRIBUTES, GetFunctionAttribute.getCommand());
+    verify(commands).put(MessageType.EXECUTE_REGION_FUNCTION, ExecuteRegionFunction66.getCommand());
+    verify(commands).put(MessageType.GATEWAY_RECEIVER_COMMAND, GatewayReceiverCommand.getCommand());
+    verify(commands).put(MessageType.CONTAINS_KEY, ContainsKey66.getCommand());
+
+    verify(commands).put(MessageType.ADD_PDX_ENUM, AddPdxEnum.getCommand());
+    verify(commands).put(MessageType.GET_PDX_ID_FOR_ENUM, GetPDXIdForEnum.getCommand());
+    verify(commands).put(MessageType.GET_PDX_ENUM_BY_ID, GetPDXEnumById.getCommand());
+
+    verify(commands).put(MessageType.REQUEST, Get70.getCommand());
+    verify(commands).put(MessageType.GET_ENTRY, GetEntry70.getCommand());
+    verify(commands).put(MessageType.GET_ALL_70, GetAll70.getCommand());
+    verify(commands).put(MessageType.PUT, Put70.getCommand());
+    verify(commands).put(MessageType.DESTROY, Destroy70.getCommand());
+    verify(commands).put(MessageType.INVALIDATE, Invalidate70.getCommand());
+    verify(commands).put(MessageType.GET_PDX_TYPES, GetPdxTypes70.getCommand());
+    verify(commands).put(MessageType.GET_PDX_ENUMS, GetPdxEnums70.getCommand());
+    verify(commands).put(MessageType.EXECUTE_FUNCTION, ExecuteFunction70.getCommand());
+
+    verify(commands).put(MessageType.PUTALL, PutAll80.getCommand());
+    verify(commands).put(MessageType.GET_ALL_WITH_CALLBACK, GetAllWithCallback.getCommand());
+    verify(commands).put(MessageType.PUT_ALL_WITH_CALLBACK, PutAllWithCallback.getCommand());
+    verify(commands).put(MessageType.REMOVE_ALL, RemoveAll.getCommand());
+
+    verifyNoMoreInteractions(commands);
+  }
+
+  @Test
+  public void commandMapUnmodifiable() {
+    final CommandInitializer commandInitializer = new CommandInitializer();
+    final Map<Integer, Command> commands = commandInitializer.get(KnownVersion.CURRENT);
+    assertThatThrownBy(() -> commands.put(1, Put70.getCommand()))
+        .isInstanceOf(UnsupportedOperationException.class);
+  }
+
+  @Test
+  public void newlyRegisteredCommandsVisibleInCommandMap() {
+    final Command command = mock(Command.class);
+    Map<KnownVersion, Command> newCommandMap = new HashMap<>();
+    newCommandMap.put(KnownVersion.CURRENT, command);
+
+    final CommandInitializer commandInitializer = new CommandInitializer();
+    final Map<Integer, Command> commands = commandInitializer.get(KnownVersion.CURRENT);
+    assertThat(commands).doesNotContainKeys(-2);
+    commandInitializer.register(-2, newCommandMap);
+    assertThat(commands).containsEntry(-2, command);
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java
index 638114d..792dbf0 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java
@@ -117,7 +117,7 @@
     long uniqueIdFromMessage = 23456L;
     MessageIdExtractor messageIdExtractor = mock(MessageIdExtractor.class);
     when(handshake.getEncryptor()).thenReturn(mock(Encryptor.class));
-    when(handshake.getVersion()).thenReturn(KnownVersion.GFE_82);
+    when(handshake.getVersion()).thenReturn(KnownVersion.GFE_81);
     when(messageIdExtractor.getUniqueIdFromMessage(any(Message.class), any(Encryptor.class),
         anyLong())).thenReturn(uniqueIdFromMessage);
     when(requestMessage.isSecureMode()).thenReturn(true);
@@ -131,7 +131,7 @@
 
   @Test
   public void post65NonSecureShouldThrow() {
-    when(handshake.getVersion()).thenReturn(KnownVersion.GFE_82);
+    when(handshake.getVersion()).thenReturn(KnownVersion.GFE_81);
 
     Throwable thrown = catchThrowable(() -> serverConnection.getUniqueId());
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ContainsKeyTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ContainsKeyTest.java
deleted file mode 100644
index f324b7f..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ContainsKeyTest.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.eq;
-import static org.mockito.Mockito.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class ContainsKeyTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private Message replyMessage;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-
-  @InjectMocks
-  private ContainsKey containsKey;
-
-  @Before
-  public void setUp() throws Exception {
-    MockitoAnnotations.initMocks(this);
-    Region region = mock(LocalRegion.class);
-    when(region.containsKey(isA(String.class))).thenReturn(true);
-
-    InternalCache cache = mock(InternalCache.class);
-    when(cache.getRegion(isA(String.class))).thenReturn(region);
-
-    CacheServerStats cacheServerStats = mock(CacheServerStats.class);
-
-    when(this.serverConnection.getCache()).thenReturn(cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(cacheServerStats);
-    when(this.serverConnection.getResponseMessage()).thenReturn(this.replyMessage);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-
-    Part regionNamePart = mock(Part.class);
-    when(regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    Part keyPart = mock(Part.class);
-    when(keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getPart(eq(0))).thenReturn(regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(keyPart);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    containsKey.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    containsKey.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME, KEY);
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.READ, REGION_NAME, KEY);
-
-    containsKey.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME, KEY);
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-
-    containsKey.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).containsKeyAuthorize(eq(REGION_NAME), eq(KEY));
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest)
-        .containsKeyAuthorize(eq(REGION_NAME), eq(KEY));
-
-    containsKey.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).containsKeyAuthorize(eq(REGION_NAME), eq(KEY));
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy65Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy65Test.java
deleted file mode 100644
index c1617ef..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Destroy65Test.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThatCode;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.isA;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.EntryNotFoundException;
-import org.apache.geode.cache.operations.DestroyOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.OpType;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class Destroy65Test {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] EVENT = new byte[8];
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private LocalRegion region;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private CacheServerStats cacheServerStats;
-  @Mock
-  private Message responseMessage;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part operationPart;
-  @Mock
-  private Part eventPart;
-  @Mock
-  private Part callbackArgPart;
-  @Mock
-  private DestroyOperationContext destroyOperationContext;
-  @InjectMocks
-  private Destroy65 destroy65;
-
-  @Before
-  public void setUp() throws Exception {
-    this.destroy65 = new Destroy65();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG)))
-        .thenReturn(this.destroyOperationContext);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(this.region);
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.callbackArgPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(this.eventPart.getSerializedForm()).thenReturn(EVENT);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(6);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(3))).thenReturn(operationPart);
-    when(this.message.getPart(eq(4))).thenReturn(this.eventPart);
-    when(this.message.getPart(eq(5))).thenReturn(this.callbackArgPart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(this.cacheServerStats);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getReplyMessage()).thenReturn(this.responseMessage);
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.destroy65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.responseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.destroy65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.responseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.WRITE, REGION_NAME, KEY);
-
-    this.destroy65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.destroy65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-    verify(this.responseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest)
-        .destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-
-    this.destroy65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-  @Test
-  public void destroyThrowsAndHandlesEntryNotFoundExceptionOnServer() {
-    doThrow(new EntryNotFoundException("")).when(region).basicBridgeDestroy(any(), any(), any(),
-        anyBoolean(), any());
-
-    assertThatCode(() -> destroy65.cmdExecute(message, serverConnection, securityService, 0))
-        .doesNotThrowAnyException();
-  }
-
-  @Test
-  public void cmdExecuteWhenOperationPartIsNotBytes_whenOperationPartGetObjectIsNull_thenCallsRegionBasicBridgeDestroy()
-      throws Exception {
-    when(operationPart.getObject()).thenReturn(null);
-
-    destroy65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(region).basicBridgeDestroy(any(), any(), any(), anyBoolean(), any());
-  }
-
-  @Test
-  public void cmdExecuteWhenOperationPartIsNotBytes_whenOperationPartIsNotDestroyOrRemove_thenCallsRegionBasicBridgeRemove()
-      throws Exception {
-    when(operationPart.getObject()).thenReturn(org.apache.geode.cache.Operation.CREATE);
-
-    destroy65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(region).basicBridgeRemove(any(), any(), any(), any(), anyBoolean(), any());
-  }
-
-  @Test
-  public void cmdExecuteWhenOperationPartIsBytes_whenOperationSerializedFormIsNull_thenCallsRegionBasicBridgeDestroy()
-      throws Exception {
-    when(operationPart.isBytes()).thenReturn(true);
-    when(operationPart.getSerializedForm()).thenReturn(null);
-
-    destroy65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(region).basicBridgeDestroy(any(), any(), any(), anyBoolean(), any());
-  }
-
-  @Test
-  public void cmdExecuteWhenOperationPartIsBytes_whenOperationSerializedFormIsLengthZero_thenCallsRegionBasicBridgeDestroy()
-      throws Exception {
-    when(operationPart.isBytes()).thenReturn(true);
-    when(operationPart.getSerializedForm()).thenReturn(new byte[0]);
-
-    destroy65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(region).basicBridgeDestroy(any(), any(), any(), anyBoolean(), any());
-  }
-
-  @Test
-  public void cmdExecuteWhenIsBytesIsTrue_whenOperationSerializedFormIsValidAndIsNotDestroyOrRemove_thenCallsRegionBasicBridgeRemove()
-      throws Exception {
-    when(operationPart.isBytes()).thenReturn(true);
-    byte[] serializedForm = {OpType.CREATE};
-    when(operationPart.getSerializedForm()).thenReturn(serializedForm);
-
-    destroy65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(region).basicBridgeRemove(any(), any(), any(), any(), anyBoolean(), any());
-  }
-
-  @Test
-  public void cmdExecuteWhenIsBytesIsFalse_whenOperationPartIsByteAndOpTypeDestroy_thenCallsRegionBasicBridgeRemove()
-      throws Exception {
-    when(message.getPart(2)).thenReturn(mock(Part.class));
-    when(operationPart.getObject()).thenReturn(OpType.DESTROY);
-
-    destroy65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(region).basicBridgeRemove(any(), any(), any(), any(), anyBoolean(), any());
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/DestroyTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/DestroyTest.java
deleted file mode 100644
index 5a9eff0..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/DestroyTest.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThatCode;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.isA;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.EntryNotFoundException;
-import org.apache.geode.cache.operations.DestroyOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class DestroyTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] EVENT = new byte[8];
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private LocalRegion region;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private CacheServerStats cacheServerStats;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private Message replyMessage;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part eventPart;
-  @Mock
-  private Part callbackArgPart;
-  @Mock
-  private DestroyOperationContext destroyOperationContext;
-  @InjectMocks
-  private Destroy destroy;
-
-  @Before
-  public void setUp() throws Exception {
-    this.destroy = new Destroy();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG)))
-        .thenReturn(this.destroyOperationContext);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(this.region);
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.callbackArgPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(this.destroyOperationContext.getCallbackArg()).thenReturn(CALLBACK_ARG);
-
-    when(this.eventPart.getSerializedForm()).thenReturn(EVENT);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(4);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(2))).thenReturn(this.eventPart);
-    when(this.message.getPart(eq(3))).thenReturn(this.callbackArgPart);
-
-    when(this.region.containsKey(eq(REGION_NAME))).thenReturn(true);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(this.cacheServerStats);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-    when(this.serverConnection.getReplyMessage()).thenReturn(this.replyMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.destroy.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.destroy.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.WRITE, REGION_NAME, KEY);
-
-    this.destroy.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.destroy.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest)
-        .destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-
-    this.destroy.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).destroyAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-  @Test
-  public void destroyThrowsAndHandlesEntryNotFoundExceptionOnServer() {
-    doThrow(new EntryNotFoundException("")).when(region).basicBridgeDestroy(any(), any(), any(),
-        anyBoolean(), any());
-
-    assertThatCode(() -> destroy.cmdExecute(message, serverConnection, securityService, 0))
-        .doesNotThrowAnyException();
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction65Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction65Test.java
deleted file mode 100644
index c29d4b7..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction65Test.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.LowMemoryException;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
-import org.apache.geode.internal.cache.control.InternalResourceManager;
-import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.tier.sockets.ServerSideHandshakeImpl;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction65.FunctionContextImplFactory;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction65.ServerToClientFunctionResultSender65Factory;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.security.ResourcePermissions;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-import org.apache.geode.util.internal.GeodeGlossary;
-
-@Category(ClientServerTest.class)
-public class ExecuteFunction65Test {
-
-  private static final String FUNCTION = "function";
-  private static final String FUNCTION_ID = "function_id";
-  private static final boolean OPTIMIZE_FOR_WRITE = false;
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] RESULT = new byte[] {Integer.valueOf(0).byteValue()};
-
-  private AuthorizeRequest authorizeRequest;
-  private ChunkedMessage chunkedResponseMessage;
-  private ChunkedMessage functionResponseMessage;
-  private Message message;
-  private SecurityService securityService;
-  private ServerConnection serverConnection;
-  private ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory;
-
-  private ExecuteFunction65 executeFunction65;
-
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  @Before
-  public void setUp() throws Exception {
-    System.setProperty(GeodeGlossary.GEMFIRE_PREFIX + "statsDisabled", "true");
-
-    authorizeRequest = mock(AuthorizeRequest.class);
-    chunkedResponseMessage = mock(ChunkedMessage.class);
-    functionResponseMessage = mock(ChunkedMessage.class);
-    message = mock(Message.class);
-    securityService = mock(SecurityService.class);
-    serverConnection = mock(ServerConnection.class);
-    serverToClientFunctionResultSender65Factory =
-        mock(ServerToClientFunctionResultSender65Factory.class);
-
-    AcceptorImpl acceptor = mock(AcceptorImpl.class);
-    InternalCache cache = mock(InternalCache.class);
-    ExecuteFunctionOperationContext executeFunctionOperationContext =
-        mock(ExecuteFunctionOperationContext.class);
-    Function functionObject = mock(Function.class);
-    FunctionContextImplFactory functionContextImplFactory = mock(FunctionContextImplFactory.class);
-    HeapMemoryMonitor heapMemoryMonitor = mock(HeapMemoryMonitor.class);
-    InternalFunctionExecutionService internalFunctionExecutionService =
-        mock(InternalFunctionExecutionService.class);
-    InternalResourceManager internalResourceManager = mock(InternalResourceManager.class);
-    ServerToClientFunctionResultSender65 serverToClientFunctionResultSender65 =
-        mock(ServerToClientFunctionResultSender65.class);
-
-    Part argsPart = mock(Part.class);
-    Part callbackArgPart = mock(Part.class);
-    Part functionPart = mock(Part.class);
-    Part functionStatePart = mock(Part.class);
-    Part partPart = mock(Part.class);
-
-    when(authorizeRequest.executeFunctionAuthorize(eq(FUNCTION_ID), eq(null), eq(null), eq(null),
-        eq(OPTIMIZE_FOR_WRITE))).thenReturn(executeFunctionOperationContext);
-
-    when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-    when(cache.getDistributedSystem()).thenReturn(mock(InternalDistributedSystem.class));
-    when(cache.getInternalResourceManager()).thenReturn(internalResourceManager);
-    when(cache.getMyId()).thenReturn(mock(InternalDistributedMember.class));
-
-    when(callbackArgPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(functionObject.getId()).thenReturn(FUNCTION_ID);
-    doCallRealMethod().when(functionObject).getRequiredPermissions(any());
-    doCallRealMethod().when(functionObject).getRequiredPermissions(any(), any());
-
-    when(functionPart.getStringOrObject()).thenReturn(FUNCTION);
-    when(functionStatePart.getSerializedForm()).thenReturn(RESULT);
-    when(heapMemoryMonitor.createLowMemoryIfNeeded(any(), any(DistributedMember.class)))
-        .thenReturn(mock(LowMemoryException.class));
-    when(internalFunctionExecutionService.getFunction(eq(FUNCTION))).thenReturn(functionObject);
-    when(internalResourceManager.getHeapMonitor()).thenReturn(heapMemoryMonitor);
-
-    when(message.getNumberOfParts()).thenReturn(4);
-    when(message.getPart(eq(0))).thenReturn(functionStatePart);
-    when(message.getPart(eq(1))).thenReturn(functionPart);
-    when(message.getPart(eq(2))).thenReturn(argsPart);
-    when(message.getPart(eq(3))).thenReturn(partPart);
-
-    when(internalResourceManager.getHeapMonitor()).thenReturn(heapMemoryMonitor);
-
-    when(serverConnection.getAcceptor()).thenReturn(acceptor);
-    when(serverConnection.getAuthzRequest()).thenReturn(authorizeRequest);
-    when(serverConnection.getCache()).thenReturn(cache);
-    when(serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(serverConnection.getChunkedResponseMessage()).thenReturn(chunkedResponseMessage);
-    when(serverConnection.getFunctionResponseMessage()).thenReturn(functionResponseMessage);
-    when(serverConnection.getHandshake()).thenReturn(mock(ServerSideHandshakeImpl.class));
-
-    when(serverToClientFunctionResultSender65Factory.create(any(), anyInt(), any(), any(), any()))
-        .thenReturn(
-            serverToClientFunctionResultSender65);
-
-    executeFunction65 = new ExecuteFunction65(internalFunctionExecutionService,
-        serverToClientFunctionResultSender65Factory, functionContextImplFactory);
-  }
-
-  @Test
-  public void nonSecureShouldSucceed() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(false);
-
-    executeFunction65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withIntegratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(true);
-
-    executeFunction65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withIntegratedSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(securityService).authorize(Resource.DATA,
-        Operation.WRITE);
-
-    executeFunction65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withOldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(false);
-
-    executeFunction65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(authorizeRequest).executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(),
-        eq(false));
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withOldSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(authorizeRequest)
-        .executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(), eq(false));
-
-    executeFunction65.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verifyNoMoreInteractions(serverToClientFunctionResultSender65Factory);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction66Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction66Test.java
deleted file mode 100644
index 092a32e..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction66Test.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import java.util.concurrent.Executor;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.OperationExecutors;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
-import org.apache.geode.internal.cache.control.InternalResourceManager;
-import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.OldClientSupportService;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction66.FunctionContextImplFactory;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction66.ServerToClientFunctionResultSender65Factory;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.security.ResourcePermissions;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-import org.apache.geode.util.internal.GeodeGlossary;
-
-@Category(ClientServerTest.class)
-public class ExecuteFunction66Test {
-
-  private static final String FUNCTION = "function";
-  private static final String FUNCTION_ID = "function_id";
-  private static final boolean OPTIMIZE_FOR_WRITE = false;
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] RESULT = new byte[] {Integer.valueOf(0).byteValue()};
-
-  private AuthorizeRequest authzRequest;
-  private ChunkedMessage functionResponseMessage;
-  private Message message;
-  private OldClientSupportService oldClientSupportService;
-  private SecurityService securityService;
-  private ServerConnection serverConnection;
-
-  // the following fields are all accessed in sub-class
-  InternalFunctionExecutionService internalFunctionExecutionService;
-  ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory;
-  FunctionContextImplFactory functionContextImplFactory;
-
-  ExecuteFunction66 executeFunction;
-
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  @Before
-  public void setUp() throws Exception {
-    System.setProperty(GeodeGlossary.GEMFIRE_PREFIX + "statsDisabled", "true");
-
-    authzRequest = mock(AuthorizeRequest.class);
-    functionResponseMessage = mock(ChunkedMessage.class);
-    internalFunctionExecutionService = mock(InternalFunctionExecutionService.class);
-    message = mock(Message.class);
-    oldClientSupportService = mock(OldClientSupportService.class);
-    securityService = mock(SecurityService.class);
-    serverConnection = mock(ServerConnection.class);
-
-    serverToClientFunctionResultSender65Factory =
-        mock(ServerToClientFunctionResultSender65Factory.class);
-    functionContextImplFactory = mock(FunctionContextImplFactory.class);
-
-    AcceptorImpl acceptor = mock(AcceptorImpl.class);
-    InternalCache cache = mock(InternalCache.class);
-    ClusterDistributionManager clusterDistributionManager = mock(ClusterDistributionManager.class);
-    Message errorResponseMessage = mock(Message.class);
-    ExecuteFunctionOperationContext executeFunctionOperationContext =
-        mock(ExecuteFunctionOperationContext.class);
-    Function functionObject = mock(Function.class);
-    Executor functionExecutor = mock(Executor.class);
-    InternalResourceManager internalResourceManager = mock(InternalResourceManager.class);
-    Message replyMessage = mock(Message.class);
-    ServerToClientFunctionResultSender65 serverToClientFunctionResultSender65 =
-        mock(ServerToClientFunctionResultSender65.class);
-
-    Part argsPart = mock(Part.class);
-    Part callbackArgPart = mock(Part.class);
-    Part functionPart = mock(Part.class);
-    Part functionStatePart = mock(Part.class);
-    Part partPart = mock(Part.class);
-
-    when(authzRequest.executeFunctionAuthorize(eq(FUNCTION_ID), eq(null), eq(null), eq(null),
-        eq(OPTIMIZE_FOR_WRITE))).thenReturn(executeFunctionOperationContext);
-
-    when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-    when(cache.getDistributionManager()).thenReturn(clusterDistributionManager);
-    when(cache.getDistributedSystem()).thenReturn(mock(InternalDistributedSystem.class));
-    when(cache.getInternalResourceManager()).thenReturn(internalResourceManager);
-    when(cache.getResourceManager()).thenReturn(internalResourceManager);
-    when(cache.getService(eq(OldClientSupportService.class))).thenReturn(oldClientSupportService);
-
-    when(callbackArgPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    OperationExecutors executors = mock(OperationExecutors.class);
-    when(clusterDistributionManager.getExecutors()).thenReturn(executors);
-    when(executors.getFunctionExecutor()).thenReturn(functionExecutor);
-
-    when(functionObject.getId()).thenReturn(FUNCTION_ID);
-    doCallRealMethod().when(functionObject).getRequiredPermissions(any());
-    doCallRealMethod().when(functionObject).getRequiredPermissions(any(), any());
-
-    when(functionPart.getStringOrObject()).thenReturn(FUNCTION);
-    when(functionStatePart.getSerializedForm()).thenReturn(RESULT);
-    when(internalFunctionExecutionService.getFunction(eq(FUNCTION))).thenReturn(functionObject);
-    when(internalResourceManager.getHeapMonitor()).thenReturn(mock(HeapMemoryMonitor.class));
-
-    when(message.getNumberOfParts()).thenReturn(4);
-    when(message.getPart(eq(0))).thenReturn(functionStatePart);
-    when(message.getPart(eq(1))).thenReturn(functionPart);
-    when(message.getPart(eq(2))).thenReturn(argsPart);
-    when(message.getPart(eq(3))).thenReturn(partPart);
-
-    when(serverConnection.getAcceptor()).thenReturn(acceptor);
-    when(serverConnection.getAuthzRequest()).thenReturn(authzRequest);
-    when(serverConnection.getCache()).thenReturn(cache);
-    when(serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(serverConnection.getErrorResponseMessage()).thenReturn(errorResponseMessage);
-    when(serverConnection.getFunctionResponseMessage()).thenReturn(functionResponseMessage);
-    when(serverConnection.getHandshake()).thenReturn(mock(ServerSideHandshake.class));
-    when(serverConnection.getReplyMessage()).thenReturn(replyMessage);
-
-    when(serverToClientFunctionResultSender65Factory.create(any(), anyInt(), any(), any(), any()))
-        .thenReturn(
-            serverToClientFunctionResultSender65);
-
-    executeFunction = new ExecuteFunction66(internalFunctionExecutionService,
-        serverToClientFunctionResultSender65Factory, functionContextImplFactory);
-
-    postSetUp();
-  }
-
-  void postSetUp() {
-    // override in sub-class
-  }
-
-  @Test
-  public void nonSecureShouldSucceed() throws Exception {
-    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
-    when(securityService.isClientSecurityRequired()).thenReturn(false);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withIntegratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(true);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withIntegratedSecurityShouldThrowIfNotAuthorized() {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(securityService)
-        .authorize(ResourcePermissions.DATA_WRITE);
-
-    assertThatThrownBy(() -> executeFunction.cmdExecute(message, serverConnection,
-        securityService, 0)).isExactlyInstanceOf(NullPointerException.class);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    // verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-    verifyNoMoreInteractions(serverToClientFunctionResultSender65Factory);
-  }
-
-  @Test
-  public void withOldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(false);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(authzRequest).executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(),
-        eq(false));
-    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
-        anyInt(), any(), any(), any());
-  }
-
-  @Test
-  public void withOldSecurityShouldThrowIfNotAuthorized() {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(authzRequest)
-        .executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(), eq(false));
-
-    assertThatThrownBy(() -> executeFunction.cmdExecute(message, serverConnection,
-        securityService, 0)).isExactlyInstanceOf(NullPointerException.class);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verifyNoMoreInteractions(serverToClientFunctionResultSender65Factory);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70Test.java
index 4db3887..4b0b439 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70Test.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunction70Test.java
@@ -1,31 +1,239 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
  *
  * http://www.apache.org/licenses/LICENSE-2.0
  *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
  */
 package org.apache.geode.internal.cache.tier.sockets.command;
 
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.concurrent.Executor;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.OperationExecutors;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
+import org.apache.geode.internal.cache.control.InternalResourceManager;
+import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
+import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender65;
+import org.apache.geode.internal.cache.tier.CachedRegionHelper;
+import org.apache.geode.internal.cache.tier.ServerSideHandshake;
+import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
+import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
+import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.internal.cache.tier.sockets.OldClientSupportService;
+import org.apache.geode.internal.cache.tier.sockets.Part;
+import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
+import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction70.FunctionContextImplFactory;
+import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction70.ServerToClientFunctionResultSender65Factory;
+import org.apache.geode.internal.security.AuthorizeRequest;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.management.internal.security.ResourcePermissions;
+import org.apache.geode.security.NotAuthorizedException;
 import org.apache.geode.test.junit.categories.ClientServerTest;
+import org.apache.geode.util.internal.GeodeGlossary;
 
 @Category(ClientServerTest.class)
-public class ExecuteFunction70Test extends ExecuteFunction66Test {
+public class ExecuteFunction70Test {
 
-  @Override
-  public void postSetUp() {
+  private static final String FUNCTION = "function";
+  private static final String FUNCTION_ID = "function_id";
+  private static final boolean OPTIMIZE_FOR_WRITE = false;
+  private static final Object CALLBACK_ARG = "arg";
+  private static final byte[] RESULT = new byte[] {0};
+
+  private AuthorizeRequest authzRequest;
+  private ChunkedMessage functionResponseMessage;
+  private Message message;
+  private OldClientSupportService oldClientSupportService;
+  private SecurityService securityService;
+  private ServerConnection serverConnection;
+
+  // the following fields are all accessed in sub-class
+  InternalFunctionExecutionService internalFunctionExecutionService;
+  ServerToClientFunctionResultSender65Factory serverToClientFunctionResultSender65Factory;
+  FunctionContextImplFactory functionContextImplFactory;
+
+  ExecuteFunction70 executeFunction;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(GeodeGlossary.GEMFIRE_PREFIX + "statsDisabled", "true");
+
+    authzRequest = mock(AuthorizeRequest.class);
+    functionResponseMessage = mock(ChunkedMessage.class);
+    internalFunctionExecutionService = mock(InternalFunctionExecutionService.class);
+    message = mock(Message.class);
+    oldClientSupportService = mock(OldClientSupportService.class);
+    securityService = mock(SecurityService.class);
+    serverConnection = mock(ServerConnection.class);
+
+    serverToClientFunctionResultSender65Factory =
+        mock(ServerToClientFunctionResultSender65Factory.class);
+    functionContextImplFactory = mock(FunctionContextImplFactory.class);
+
+    AcceptorImpl acceptor = mock(AcceptorImpl.class);
+    InternalCache cache = mock(InternalCache.class);
+    ClusterDistributionManager clusterDistributionManager = mock(ClusterDistributionManager.class);
+    Message errorResponseMessage = mock(Message.class);
+    ExecuteFunctionOperationContext executeFunctionOperationContext =
+        mock(ExecuteFunctionOperationContext.class);
+    Function functionObject = mock(Function.class);
+    Executor functionExecutor = mock(Executor.class);
+    InternalResourceManager internalResourceManager = mock(InternalResourceManager.class);
+    Message replyMessage = mock(Message.class);
+    ServerToClientFunctionResultSender65 serverToClientFunctionResultSender65 =
+        mock(ServerToClientFunctionResultSender65.class);
+
+    Part argsPart = mock(Part.class);
+    Part callbackArgPart = mock(Part.class);
+    Part functionPart = mock(Part.class);
+    Part functionStatePart = mock(Part.class);
+    Part partPart = mock(Part.class);
+
+    when(authzRequest.executeFunctionAuthorize(eq(FUNCTION_ID), eq(null), eq(null), eq(null),
+        eq(OPTIMIZE_FOR_WRITE))).thenReturn(executeFunctionOperationContext);
+
+    when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
+    when(cache.getDistributionManager()).thenReturn(clusterDistributionManager);
+    when(cache.getDistributedSystem()).thenReturn(mock(InternalDistributedSystem.class));
+    when(cache.getInternalResourceManager()).thenReturn(internalResourceManager);
+    when(cache.getResourceManager()).thenReturn(internalResourceManager);
+    when(cache.getService(eq(OldClientSupportService.class))).thenReturn(oldClientSupportService);
+
+    when(callbackArgPart.getObject()).thenReturn(CALLBACK_ARG);
+
+    OperationExecutors executors = mock(OperationExecutors.class);
+    when(clusterDistributionManager.getExecutors()).thenReturn(executors);
+    when(executors.getFunctionExecutor()).thenReturn(functionExecutor);
+
+    when(functionObject.getId()).thenReturn(FUNCTION_ID);
+    doCallRealMethod().when(functionObject).getRequiredPermissions(any());
+    doCallRealMethod().when(functionObject).getRequiredPermissions(any(), any());
+
+    when(functionPart.getStringOrObject()).thenReturn(FUNCTION);
+    when(functionStatePart.getSerializedForm()).thenReturn(RESULT);
+    when(internalFunctionExecutionService.getFunction(eq(FUNCTION))).thenReturn(functionObject);
+    when(internalResourceManager.getHeapMonitor()).thenReturn(mock(HeapMemoryMonitor.class));
+
+    when(message.getNumberOfParts()).thenReturn(4);
+    when(message.getPart(eq(0))).thenReturn(functionStatePart);
+    when(message.getPart(eq(1))).thenReturn(functionPart);
+    when(message.getPart(eq(2))).thenReturn(argsPart);
+    when(message.getPart(eq(3))).thenReturn(partPart);
+
+    when(serverConnection.getAcceptor()).thenReturn(acceptor);
+    when(serverConnection.getAuthzRequest()).thenReturn(authzRequest);
+    when(serverConnection.getCache()).thenReturn(cache);
+    when(serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
+    when(serverConnection.getErrorResponseMessage()).thenReturn(errorResponseMessage);
+    when(serverConnection.getFunctionResponseMessage()).thenReturn(functionResponseMessage);
+    when(serverConnection.getHandshake()).thenReturn(mock(ServerSideHandshake.class));
+    when(serverConnection.getReplyMessage()).thenReturn(replyMessage);
+
+    when(serverToClientFunctionResultSender65Factory.create(any(), anyInt(), any(), any(), any()))
+        .thenReturn(
+            serverToClientFunctionResultSender65);
+
     executeFunction = new ExecuteFunction70(internalFunctionExecutionService,
         serverToClientFunctionResultSender65Factory, functionContextImplFactory);
   }
+
+  @Test
+  public void nonSecureShouldSucceed() throws Exception {
+    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
+    when(securityService.isClientSecurityRequired()).thenReturn(false);
+
+    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
+        anyInt(), any(), any(), any());
+  }
+
+  @Test
+  public void withIntegratedSecurityShouldSucceedIfAuthorized() throws Exception {
+    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(true);
+
+    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
+    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
+        anyInt(), any(), any(), any());
+  }
+
+  @Test
+  public void withIntegratedSecurityShouldThrowIfNotAuthorized() {
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(true);
+    doThrow(new NotAuthorizedException("")).when(securityService)
+        .authorize(ResourcePermissions.DATA_WRITE);
+
+    assertThatThrownBy(() -> executeFunction.cmdExecute(message, serverConnection,
+        securityService, 0)).isExactlyInstanceOf(NullPointerException.class);
+
+    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
+    // verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
+    verifyNoMoreInteractions(serverToClientFunctionResultSender65Factory);
+  }
+
+  @Test
+  public void withOldSecurityShouldSucceedIfAuthorized() throws Exception {
+    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(false);
+
+    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(authzRequest).executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(),
+        eq(false));
+    verify(serverToClientFunctionResultSender65Factory).create(eq(functionResponseMessage),
+        anyInt(), any(), any(), any());
+  }
+
+  @Test
+  public void withOldSecurityShouldThrowIfNotAuthorized() {
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(false);
+    doThrow(new NotAuthorizedException("")).when(authzRequest)
+        .executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(), eq(false));
+
+    assertThatThrownBy(() -> executeFunction.cmdExecute(message, serverConnection,
+        securityService, 0)).isExactlyInstanceOf(NullPointerException.class);
+
+    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
+    verifyNoMoreInteractions(serverToClientFunctionResultSender65Factory);
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunctionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunctionTest.java
deleted file mode 100644
index abf9831..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteFunctionTest.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.operations.ExecuteFunctionOperationContext;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
-import org.apache.geode.internal.cache.control.InternalResourceManager;
-import org.apache.geode.internal.cache.execute.FunctionContextImpl;
-import org.apache.geode.internal.cache.execute.InternalFunctionExecutionService;
-import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.ServerSideHandshake;
-import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.OldClientSupportService;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction.FunctionContextImplFactory;
-import org.apache.geode.internal.cache.tier.sockets.command.ExecuteFunction.ServerToClientFunctionResultSenderFactory;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.security.ResourcePermissions;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-import org.apache.geode.util.internal.GeodeGlossary;
-
-@Category(ClientServerTest.class)
-public class ExecuteFunctionTest {
-  private static final String FUNCTION = "function";
-  private static final String FUNCTION_ID = "function_id";
-  private static final boolean OPTIMIZE_FOR_WRITE = false;
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] RESULT = new byte[] {Integer.valueOf(1).byteValue()};
-
-  private AuthorizeRequest authorizeRequest;
-  private ChunkedMessage chunkedResponseMessage;
-  private ChunkedMessage functionResponseMessage;
-  private Message message;
-  private SecurityService securityService;
-  private ServerConnection serverConnection;
-  private ServerToClientFunctionResultSenderFactory serverToClientFunctionResultSenderFactory;
-
-  private ExecuteFunction executeFunction;
-
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  @Before
-  public void setUp() throws Exception {
-    System.setProperty(GeodeGlossary.GEMFIRE_PREFIX + "statsDisabled", "true");
-
-    authorizeRequest = mock(AuthorizeRequest.class);
-    chunkedResponseMessage = mock(ChunkedMessage.class);
-    functionResponseMessage = mock(ChunkedMessage.class);
-    message = mock(Message.class);
-    securityService = mock(SecurityService.class);
-    serverConnection = mock(ServerConnection.class);
-    serverToClientFunctionResultSenderFactory =
-        mock(ServerToClientFunctionResultSenderFactory.class);
-
-    InternalCache cache = mock(InternalCache.class, RETURNS_DEEP_STUBS);
-    AcceptorImpl acceptor = mock(AcceptorImpl.class);
-    FunctionContextImplFactory functionContextImplFactory = mock(FunctionContextImplFactory.class);
-    ExecuteFunctionOperationContext executeFunctionOperationContext =
-        mock(ExecuteFunctionOperationContext.class);
-    Function functionObject = mock(Function.class);
-    FunctionContextImpl functionContextImpl = mock(FunctionContextImpl.class);
-    InternalFunctionExecutionService internalFunctionExecutionService =
-        mock(InternalFunctionExecutionService.class);
-    InternalResourceManager internalResourceManager =
-        mock(InternalResourceManager.class, RETURNS_DEEP_STUBS);
-    OldClientSupportService oldClientSupportService = mock(OldClientSupportService.class);
-    ServerSideHandshake handshake = mock(ServerSideHandshake.class);
-    ServerToClientFunctionResultSender serverToClientFunctionResultSender =
-        mock(ServerToClientFunctionResultSender.class);
-
-    Part argsPart = mock(Part.class);
-    Part callbackArgPart = mock(Part.class);
-    Part functionPart = mock(Part.class);
-    Part hasResultPart = mock(Part.class);
-    Part partPart = mock(Part.class);
-
-    when(authorizeRequest.executeFunctionAuthorize(eq(FUNCTION_ID), eq(null), eq(null), eq(null),
-        eq(OPTIMIZE_FOR_WRITE))).thenReturn(executeFunctionOperationContext);
-
-    when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-    when(cache.getDistributedSystem()).thenReturn(mock(InternalDistributedSystem.class));
-    when(cache.getResourceManager()).thenReturn(internalResourceManager);
-    when(cache.getService(eq(OldClientSupportService.class))).thenReturn(oldClientSupportService);
-
-    when(callbackArgPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(functionContextImplFactory.create(any(), any(), any(), any())).thenReturn(
-        functionContextImpl);
-
-    when(functionObject.getId()).thenReturn(FUNCTION_ID);
-    doCallRealMethod().when(functionObject).getRequiredPermissions(any());
-    doCallRealMethod().when(functionObject).getRequiredPermissions(any(), any());
-
-    when(functionPart.getStringOrObject()).thenReturn(FUNCTION);
-    when(hasResultPart.getSerializedForm()).thenReturn(RESULT);
-    when(internalFunctionExecutionService.getFunction(eq(FUNCTION))).thenReturn(functionObject);
-    when(internalResourceManager.getHeapMonitor()).thenReturn(mock(HeapMemoryMonitor.class));
-
-    when(message.getNumberOfParts()).thenReturn(4);
-    when(message.getPart(eq(0))).thenReturn(hasResultPart);
-    when(message.getPart(eq(1))).thenReturn(functionPart);
-    when(message.getPart(eq(2))).thenReturn(argsPart);
-    when(message.getPart(eq(3))).thenReturn(partPart);
-
-    when(oldClientSupportService.getThrowable(any(), any())).thenReturn(mock(Throwable.class));
-
-    when(serverConnection.getAcceptor()).thenReturn(acceptor);
-    when(serverConnection.getAuthzRequest()).thenReturn(authorizeRequest);
-    when(serverConnection.getCache()).thenReturn(cache);
-    when(serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(serverConnection.getChunkedResponseMessage()).thenReturn(chunkedResponseMessage);
-    when(serverConnection.getFunctionResponseMessage()).thenReturn(functionResponseMessage);
-    when(serverConnection.getHandshake()).thenReturn(handshake);
-
-    when(serverToClientFunctionResultSenderFactory.create(any(), anyInt(), any(), any(), any()))
-        .thenReturn(
-            serverToClientFunctionResultSender);
-
-    executeFunction = new ExecuteFunction(internalFunctionExecutionService,
-        serverToClientFunctionResultSenderFactory,
-        functionContextImplFactory);
-  }
-
-  @Test
-  public void nonSecureShouldSucceed() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(false);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(serverToClientFunctionResultSenderFactory).create(eq(functionResponseMessage), anyInt(),
-        any(), any(), any());
-  }
-
-  @Test
-  public void withIntegratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(true);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verify(serverToClientFunctionResultSenderFactory).create(eq(functionResponseMessage), anyInt(),
-        any(), any(), any());
-  }
-
-  @Test
-  public void withIntegratedSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(securityService)
-        .authorize(ResourcePermissions.DATA_WRITE);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(securityService).authorize(ResourcePermissions.DATA_WRITE);
-    verify(chunkedResponseMessage).sendChunk(serverConnection);
-    verifyNoMoreInteractions(serverToClientFunctionResultSenderFactory);
-  }
-
-  @Test
-  public void withOldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(false);
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(authorizeRequest).executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(),
-        eq(false));
-    verify(serverToClientFunctionResultSenderFactory).create(eq(functionResponseMessage), anyInt(),
-        any(), any(), any());
-  }
-
-  @Test
-  public void withOldSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(securityService.isClientSecurityRequired()).thenReturn(true);
-    when(securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(authorizeRequest)
-        .executeFunctionAuthorize(eq(FUNCTION_ID), any(), any(), any(), eq(false));
-
-    executeFunction.cmdExecute(message, serverConnection, securityService, 0);
-
-    verify(chunkedResponseMessage).sendChunk(serverConnection);
-    verifyNoMoreInteractions(serverToClientFunctionResultSenderFactory);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll651Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll651Test.java
deleted file mode 100644
index 5adedb6..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetAll651Test.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.GetOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.ObjectPartList;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class GetAll651Test {
-
-  private static final String REGION_NAME = "region1";
-  private static final Object[] KEYS = new Object[] {"key1", "key2", "key3"};
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private ChunkedMessage chunkedResponseMessage;
-
-  @InjectMocks
-  private GetAll651 getAll651;
-
-  @Before
-  public void setUp() throws Exception {
-    this.getAll651 = new GetAll651();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.getAuthorize(any(), any(), any()))
-        .thenReturn(mock(GetOperationContext.class));
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(mock(LocalRegion.class));
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.keyPart.getObject()).thenReturn(KEYS);
-
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(mock(CacheServerStats.class));
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getChunkedResponseMessage()).thenReturn(this.chunkedResponseMessage);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.getAll651.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.getAll651.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object key : argument.getValue().getKeys()) {
-      assertThat(key).isIn(KEYS);
-    }
-    for (Object key : KEYS) {
-      verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME,
-          key.toString());
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    for (Object key : KEYS) {
-      doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-          Operation.READ, REGION_NAME, key.toString());
-    }
-
-    this.getAll651.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    for (Object key : KEYS) {
-      verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME,
-          key.toString());
-    }
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object key : argument.getValue().getObjects()) {
-      assertThat(key).isExactlyInstanceOf(NotAuthorizedException.class);
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.getAll651.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object key : argument.getValue().getKeys()) {
-      assertThat(key).isIn(KEYS);
-    }
-
-    for (Object key : KEYS) {
-      verify(this.authzRequest).getAuthorize(eq(REGION_NAME), eq(key.toString()), eq(null));
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    for (Object key : KEYS) {
-      doThrow(new NotAuthorizedException("")).when(this.authzRequest).getAuthorize(eq(REGION_NAME),
-          eq(key.toString()), eq(null));
-    }
-    this.getAll651.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object o : argument.getValue().getObjects()) {
-      assertThat(o).isExactlyInstanceOf(NotAuthorizedException.class);
-    }
-    for (Object key : KEYS) {
-      verify(this.authzRequest).getAuthorize(eq(REGION_NAME), eq(key.toString()), eq(null));
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetAllTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetAllTest.java
deleted file mode 100644
index 3ff58f2..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetAllTest.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.ArgumentMatchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.GetOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.ObjectPartList;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class GetAllTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final Object[] KEYS = new Object[] {"key1", "key2", "key3"};
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private ChunkedMessage chunkedResponseMessage;
-
-  @InjectMocks
-  private GetAll getAll;
-
-  @Before
-  public void setUp() throws Exception {
-    this.getAll = new GetAll();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.getAuthorize(any(), any(), any()))
-        .thenReturn(mock(GetOperationContext.class));
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(mock(LocalRegion.class));
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.keyPart.getObject()).thenReturn(KEYS);
-
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(mock(CacheServerStats.class));
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getChunkedResponseMessage()).thenReturn(this.chunkedResponseMessage);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.getAll.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.getAll.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object key : argument.getValue().getKeys()) {
-      assertThat(key).isIn(KEYS);
-    }
-    for (Object key : KEYS) {
-      verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME,
-          key.toString());
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    for (Object key : KEYS) {
-      doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-          Operation.READ, REGION_NAME, key.toString());
-    }
-
-    this.getAll.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    for (Object key : KEYS) {
-      verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME,
-          key.toString());
-    }
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object key : argument.getValue().getObjects()) {
-      assertThat(key).isExactlyInstanceOf(NotAuthorizedException.class);
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.getAll.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object key : argument.getValue().getKeys()) {
-      assertThat(key).isIn(KEYS);
-    }
-
-    for (Object key : KEYS) {
-      verify(this.authzRequest).getAuthorize(eq(REGION_NAME), eq(key.toString()), eq(null));
-    }
-
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    for (Object key : KEYS) {
-      doThrow(new NotAuthorizedException("")).when(this.authzRequest).getAuthorize(eq(REGION_NAME),
-          eq(key.toString()), eq(null));
-    }
-    this.getAll.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<ObjectPartList> argument = ArgumentCaptor.forClass(ObjectPartList.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture(), eq(false));
-
-    assertThat(argument.getValue().getObjects()).hasSize(KEYS.length);
-    for (Object o : argument.getValue().getObjects()) {
-      assertThat(o).isExactlyInstanceOf(NotAuthorizedException.class);
-    }
-
-    for (Object key : KEYS) {
-      verify(this.authzRequest).getAuthorize(eq(REGION_NAME), eq(key.toString()), eq(null));
-    }
-    verify(this.chunkedResponseMessage).sendChunk(eq(this.serverConnection));
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPartitionAttributesCommandTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPartitionAttributesCommandTest.java
deleted file mode 100644
index 429b832..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/GetClientPartitionAttributesCommandTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class GetClientPartitionAttributesCommandTest {
-
-  private static final String REGION_NAME = "region1";
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Message responseMessage;
-
-  @InjectMocks
-  private GetClientPartitionAttributesCommand getClientPartitionAttributesCommand;
-
-  @Before
-  public void setUp() throws Exception {
-    this.getClientPartitionAttributesCommand = new GetClientPartitionAttributesCommand();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(mock(PartitionedRegion.class));
-
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getResponseMessage()).thenReturn(this.responseMessage);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.getClientPartitionAttributesCommand.cmdExecute(this.message, this.serverConnection,
-        this.securityService, 0);
-
-    verify(this.responseMessage).send();
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/InvalidateTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70Test.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/InvalidateTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70Test.java
index 3999586..61a5116 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/InvalidateTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Invalidate70Test.java
@@ -49,7 +49,7 @@
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
 @Category({ClientServerTest.class})
-public class InvalidateTest {
+public class Invalidate70Test {
 
   private static final String REGION_NAME = "region1";
   private static final String KEY_STRING = "key1";
@@ -80,11 +80,12 @@
   private Message responseMessage;
 
   @InjectMocks
-  private Invalidate invalidate;
+  private Invalidate70 invalidate;
 
   @Before
   public void setUp() throws Exception {
-    this.invalidate = new Invalidate();
+    this.invalidate = (Invalidate70) Invalidate70.getCommand();
+
     MockitoAnnotations.initMocks(this);
 
     when(this.authzRequest.invalidateAuthorize(any(), any(), any()))
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put61Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put61Test.java
deleted file mode 100644
index f73bd51..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put61Test.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.PutOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class Put61Test {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] EVENT = new byte[8];
-  private static final byte[] VALUE = new byte[8];
-  private static final byte[] OK_BYTES = new byte[] {0};
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private LocalRegion localRegion;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part valuePart;
-  @Mock
-  private Part deltaPart;
-  @Mock
-  private Part eventPart;
-  @Mock
-  private Part callbackArgsPart;
-  @Mock
-  private PutOperationContext putOperationContext;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private Message replyMessage;
-  @InjectMocks
-  private Put61 put61;
-
-  @Before
-  public void setUp() throws Exception {
-    this.put61 = new Put61();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.putAuthorize(eq(REGION_NAME), eq(KEY), eq(null), eq(true),
-        eq(CALLBACK_ARG))).thenReturn(this.putOperationContext);
-
-    when(this.putOperationContext.getCallbackArg()).thenReturn(CALLBACK_ARG);
-    when(this.putOperationContext.getValue()).thenReturn(VALUE);
-    when(this.putOperationContext.isObject()).thenReturn(true);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(this.localRegion);
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.callbackArgsPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(this.deltaPart.getObject()).thenReturn(true);
-
-    when(this.eventPart.getSerializedForm()).thenReturn(EVENT);
-
-    when(this.valuePart.getSerializedForm()).thenReturn(VALUE);
-    when(this.valuePart.isObject()).thenReturn(true);
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(6);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(2))).thenReturn(this.deltaPart);
-    when(this.message.getPart(eq(3))).thenReturn(this.valuePart);
-    when(this.message.getPart(eq(4))).thenReturn(this.eventPart);
-    when(this.message.getPart(eq(5))).thenReturn(this.callbackArgsPart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(mock(CacheServerStats.class));
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getReplyMessage()).thenReturn(this.replyMessage);
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
-
-    when(this.localRegion.basicBridgePut(eq(KEY), eq(VALUE), eq(VALUE), eq(true), eq(CALLBACK_ARG),
-        any(), eq(true), any())).thenReturn(true);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.put61.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.put61.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.WRITE, REGION_NAME, KEY);
-
-    this.put61.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.put61.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<byte[]> argument = ArgumentCaptor.forClass(byte[].class);
-    verify(this.replyMessage).addBytesPart(argument.capture());
-
-    assertThat(argument.getValue()).isEqualTo(OK_BYTES);
-
-    verify(this.authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(null), eq(true),
-        eq(CALLBACK_ARG));
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest).putAuthorize(eq(REGION_NAME),
-        eq(KEY), eq(null), eq(true), eq(CALLBACK_ARG));
-
-    this.put61.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(null), eq(true),
-        eq(CALLBACK_ARG));
-
-    ArgumentCaptor<NotAuthorizedException> argument =
-        ArgumentCaptor.forClass(NotAuthorizedException.class);
-    verify(this.errorResponseMessage).addObjPart(argument.capture());
-    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put65Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put65Test.java
deleted file mode 100644
index a1f59f6..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put65Test.java
+++ /dev/null
@@ -1,309 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.DataPolicy;
-import org.apache.geode.cache.RegionAttributes;
-import org.apache.geode.cache.operations.PutOperationContext;
-import org.apache.geode.internal.cache.EventIDHolder;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.TXManagerImpl;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class Put65Test {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] EVENT = new byte[8];
-  private static final byte[] VALUE = new byte[8];
-  private static final byte[] OK_BYTES = new byte[] {0};
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private LocalRegion localRegion;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part valuePart;
-  @Mock
-  private Part oldValuePart;
-  @Mock
-  private Part deltaPart;
-  @Mock
-  private Part operationPart;
-  @Mock
-  private Part flagsPart;
-  @Mock
-  private Part eventPart;
-  @Mock
-  private Part callbackArgsPart;
-  @Mock
-  private PutOperationContext putOperationContext;
-  @Mock
-  private Object object;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private Message replyMessage;
-  @Mock
-  private RegionAttributes attributes;
-  @Mock
-  private EventIDHolder clientEvent;
-  @Mock
-  private DataPolicy dataPolicy;
-
-  @InjectMocks
-  private Put65 put65;
-
-  @Before
-  public void setUp() throws Exception {
-    this.put65 = new Put65();
-    MockitoAnnotations.initMocks(this);
-
-    when(
-        this.authzRequest.putAuthorize(eq(REGION_NAME), eq(KEY), any(), eq(true), eq(CALLBACK_ARG)))
-            .thenReturn(this.putOperationContext);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(this.localRegion);
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-    when(this.cache.getCacheTransactionManager()).thenReturn(mock(TXManagerImpl.class));
-
-    when(this.callbackArgsPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(this.deltaPart.getObject()).thenReturn(Boolean.FALSE);
-
-    when(this.eventPart.getSerializedForm()).thenReturn(EVENT);
-
-    when(this.flagsPart.getInt()).thenReturn(1);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.localRegion.basicBridgePut(eq(KEY), eq(VALUE), eq(null), eq(true), eq(CALLBACK_ARG),
-        any(), eq(true), any())).thenReturn(true);
-
-    when(this.message.getNumberOfParts()).thenReturn(8);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(operationPart);
-    when(this.message.getPart(eq(2))).thenReturn(this.flagsPart);
-    when(this.message.getPart(eq(3))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(4))).thenReturn(this.deltaPart);
-    when(this.message.getPart(eq(5))).thenReturn(this.valuePart);
-    when(this.message.getPart(eq(6))).thenReturn(this.eventPart);
-    when(this.message.getPart(eq(7))).thenReturn(this.callbackArgsPart);
-
-    when(this.operationPart.getObject()).thenReturn(null);
-
-    when(this.oldValuePart.getObject()).thenReturn(mock(Object.class));
-
-    when(this.putOperationContext.getCallbackArg()).thenReturn(CALLBACK_ARG);
-    when(this.putOperationContext.getValue()).thenReturn(VALUE);
-    when(this.putOperationContext.isObject()).thenReturn(true);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(mock(CacheServerStats.class));
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getReplyMessage()).thenReturn(this.replyMessage);
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
-
-    when(this.valuePart.getSerializedForm()).thenReturn(VALUE);
-    when(this.valuePart.isObject()).thenReturn(true);
-
-    when(localRegion.getAttributes()).thenReturn(attributes);
-    when(attributes.getDataPolicy()).thenReturn(dataPolicy);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void noRegionNameShouldFail() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-    when(this.regionNamePart.getCachedString()).thenReturn(null);
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
-    verify(this.errorResponseMessage).addStringPart(argument.capture());
-    assertThat(argument.getValue()).contains("The input region name for the put request is null");
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void noKeyShouldFail() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-    when(this.keyPart.getStringOrObject()).thenReturn(null);
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
-    verify(this.errorResponseMessage).addStringPart(argument.capture());
-    assertThat(argument.getValue()).contains("The input key for the put request is null");
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.WRITE, REGION_NAME, KEY);
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<byte[]> argument = ArgumentCaptor.forClass(byte[].class);
-    verify(this.replyMessage).addBytesPart(argument.capture());
-
-    assertThat(argument.getValue()).isEqualTo(OK_BYTES);
-
-    verify(this.authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
-        eq(CALLBACK_ARG));
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest).putAuthorize(eq(REGION_NAME),
-        eq(KEY), eq(VALUE), eq(true), eq(CALLBACK_ARG));
-
-    this.put65.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
-        eq(CALLBACK_ARG));
-
-    ArgumentCaptor<NotAuthorizedException> argument =
-        ArgumentCaptor.forClass(NotAuthorizedException.class);
-    verify(this.errorResponseMessage).addObjPart(argument.capture());
-
-    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void shouldSetPossibleDuplicateReturnsTrueIfConcurrencyChecksNotEnabled() {
-
-    when(attributes.getConcurrencyChecksEnabled()).thenReturn(false);
-
-    assertThat(put65.shouldSetPossibleDuplicate(localRegion, clientEvent)).isTrue();
-  }
-
-  @Test
-  public void shouldSetPossibleDuplicateReturnsTrueIfRecoveredVersionTagForRetriedOperation() {
-    Put65 spy = Mockito.spy(put65);
-    when(attributes.getConcurrencyChecksEnabled()).thenReturn(true);
-    doReturn(true).when(spy).recoverVersionTagForRetriedOperation(clientEvent);
-
-    assertThat(spy.shouldSetPossibleDuplicate(localRegion, clientEvent)).isTrue();
-  }
-
-  @Test
-  public void shouldSetPossibleDuplicateReturnsFalseIfNotRecoveredVersionTagAndNoPersistence() {
-    Put65 spy = Mockito.spy(put65);
-    when(attributes.getConcurrencyChecksEnabled()).thenReturn(true);
-    when(dataPolicy.withPersistence()).thenReturn(false);
-    doReturn(false).when(spy).recoverVersionTagForRetriedOperation(clientEvent);
-
-    assertThat(spy.shouldSetPossibleDuplicate(localRegion, clientEvent)).isFalse();
-  }
-
-  @Test
-  public void shouldSetPossibleDuplicateReturnsTrueIfNotRecoveredVersionTagAndWithPersistence() {
-    Put65 spy = Mockito.spy(put65);
-    when(attributes.getConcurrencyChecksEnabled()).thenReturn(true);
-    when(dataPolicy.withPersistence()).thenReturn(true);
-    doReturn(false).when(spy).recoverVersionTagForRetriedOperation(clientEvent);
-
-    assertThat(spy.shouldSetPossibleDuplicate(localRegion, clientEvent)).isTrue();
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put70Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put70Test.java
new file mode 100644
index 0000000..7ee3e26
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/Put70Test.java
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache.tier.sockets.command;
+
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isA;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.operations.PutOperationContext;
+import org.apache.geode.internal.cache.EventIDHolder;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.cache.tier.CachedRegionHelper;
+import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
+import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.internal.cache.tier.sockets.Part;
+import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
+import org.apache.geode.internal.security.AuthorizeRequest;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.test.junit.categories.ClientServerTest;
+
+@Category({ClientServerTest.class})
+public class Put70Test {
+
+  private static final String REGION_NAME = "region1";
+  private static final String KEY = "key1";
+  private static final Object CALLBACK_ARG = "arg";
+  private static final byte[] EVENT = new byte[8];
+  private static final byte[] VALUE = new byte[8];
+  private static final byte[] OK_BYTES = new byte[] {0};
+
+  @Mock
+  private SecurityService securityService;
+  @Mock
+  private Message message;
+  @Mock
+  private ServerConnection serverConnection;
+  @Mock
+  private AuthorizeRequest authzRequest;
+  @Mock
+  private InternalCache cache;
+  @Mock
+  private LocalRegion localRegion;
+  @Mock
+  private Part regionNamePart;
+  @Mock
+  private Part keyPart;
+  @Mock
+  private Part valuePart;
+  @Mock
+  private Part oldValuePart;
+  @Mock
+  private Part deltaPart;
+  @Mock
+  private Part operationPart;
+  @Mock
+  private Part flagsPart;
+  @Mock
+  private Part eventPart;
+  @Mock
+  private Part callbackArgsPart;
+  @Mock
+  private PutOperationContext putOperationContext;
+  @Mock
+  private Message errorResponseMessage;
+  @Mock
+  private Message replyMessage;
+  @Mock
+  private RegionAttributes<?, ?> attributes;
+  @Mock
+  private EventIDHolder clientEvent;
+  @Mock
+  private DataPolicy dataPolicy;
+
+  @InjectMocks
+  private Put70 put70;
+
+  private AutoCloseable mockitoMocks;
+
+  @Before
+  public void setUp() throws Exception {
+    put70 = (Put70) Put70.getCommand();
+    mockitoMocks = MockitoAnnotations.openMocks(this);
+
+    when(
+        authzRequest.putAuthorize(eq(REGION_NAME), eq(KEY), any(), eq(true), eq(CALLBACK_ARG)))
+            .thenReturn(putOperationContext);
+
+    when(cache.getRegion(isA(String.class))).thenReturn(uncheckedCast(localRegion));
+    when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
+    when(cache.getCacheTransactionManager()).thenReturn(mock(TXManagerImpl.class));
+
+    when(callbackArgsPart.getObject()).thenReturn(CALLBACK_ARG);
+
+    when(deltaPart.getObject()).thenReturn(Boolean.FALSE);
+
+    when(eventPart.getSerializedForm()).thenReturn(EVENT);
+
+    when(flagsPart.getInt()).thenReturn(1);
+
+    when(keyPart.getStringOrObject()).thenReturn(KEY);
+
+    when(localRegion.basicBridgePut(eq(KEY), eq(VALUE), eq(null), eq(true), eq(CALLBACK_ARG),
+        any(), eq(true), any())).thenReturn(true);
+
+    when(message.getNumberOfParts()).thenReturn(8);
+    when(message.getPart(eq(0))).thenReturn(regionNamePart);
+    when(message.getPart(eq(1))).thenReturn(operationPart);
+    when(message.getPart(eq(2))).thenReturn(flagsPart);
+    when(message.getPart(eq(3))).thenReturn(keyPart);
+    when(message.getPart(eq(4))).thenReturn(deltaPart);
+    when(message.getPart(eq(5))).thenReturn(valuePart);
+    when(message.getPart(eq(6))).thenReturn(eventPart);
+    when(message.getPart(eq(7))).thenReturn(callbackArgsPart);
+
+    when(operationPart.getObject()).thenReturn(null);
+
+    when(oldValuePart.getObject()).thenReturn(mock(Object.class));
+
+    when(putOperationContext.getCallbackArg()).thenReturn(CALLBACK_ARG);
+    when(putOperationContext.getValue()).thenReturn(VALUE);
+    when(putOperationContext.isObject()).thenReturn(true);
+
+    when(regionNamePart.getCachedString()).thenReturn(REGION_NAME);
+
+    when(serverConnection.getCache()).thenReturn(cache);
+    when(serverConnection.getCacheServerStats()).thenReturn(mock(CacheServerStats.class));
+    when(serverConnection.getAuthzRequest()).thenReturn(authzRequest);
+    when(serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
+    when(serverConnection.getReplyMessage()).thenReturn(replyMessage);
+    when(serverConnection.getErrorResponseMessage()).thenReturn(errorResponseMessage);
+    when(serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
+
+    when(valuePart.getSerializedForm()).thenReturn(VALUE);
+    when(valuePart.isObject()).thenReturn(true);
+
+    when(localRegion.getAttributes()).thenReturn(attributes);
+    when(attributes.getDataPolicy()).thenReturn(dataPolicy);
+  }
+
+  @After
+  public void after() throws Exception {
+    mockitoMocks.close();
+  }
+
+  @Test
+  public void noSecurityShouldSucceed() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(false);
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(replyMessage).send(serverConnection);
+  }
+
+  @Test
+  public void noRegionNameShouldFail() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(false);
+    when(regionNamePart.getCachedString()).thenReturn(null);
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
+    verify(errorResponseMessage).addStringPart(argument.capture());
+    assertThat(argument.getValue()).contains("The input region name for the put request is null");
+    verify(errorResponseMessage).send(serverConnection);
+  }
+
+  @Test
+  public void noKeyShouldFail() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(false);
+    when(keyPart.getStringOrObject()).thenReturn(null);
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    ArgumentCaptor<String> argument = ArgumentCaptor.forClass(String.class);
+    verify(errorResponseMessage).addStringPart(argument.capture());
+    assertThat(argument.getValue()).contains("The input key for the put request is null");
+    verify(errorResponseMessage).send(serverConnection);
+  }
+
+  @Test
+  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(true);
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
+    verify(replyMessage).send(serverConnection);
+  }
+
+  @Test
+  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(true);
+    doThrow(new NotAuthorizedException("")).when(securityService).authorize(Resource.DATA,
+        Operation.WRITE, REGION_NAME, KEY);
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
+    verify(errorResponseMessage).send(serverConnection);
+  }
+
+  @Test
+  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(false);
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    ArgumentCaptor<byte[]> argument = ArgumentCaptor.forClass(byte[].class);
+    verify(replyMessage).addBytesPart(argument.capture());
+
+    assertThat(argument.getValue()).isEqualTo(OK_BYTES);
+
+    verify(authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
+        eq(CALLBACK_ARG));
+    verify(replyMessage).send(serverConnection);
+  }
+
+  @Test
+  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
+    when(securityService.isClientSecurityRequired()).thenReturn(true);
+    when(securityService.isIntegratedSecurity()).thenReturn(false);
+    doThrow(new NotAuthorizedException("")).when(authzRequest).putAuthorize(eq(REGION_NAME),
+        eq(KEY), eq(VALUE), eq(true), eq(CALLBACK_ARG));
+
+    put70.cmdExecute(message, serverConnection, securityService, 0);
+
+    verify(authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
+        eq(CALLBACK_ARG));
+
+    ArgumentCaptor<NotAuthorizedException> argument =
+        ArgumentCaptor.forClass(NotAuthorizedException.class);
+    verify(errorResponseMessage).addObjPart(argument.capture());
+
+    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
+    verify(errorResponseMessage).send(serverConnection);
+  }
+
+  @Test
+  public void shouldSetPossibleDuplicateReturnsTrueIfConcurrencyChecksNotEnabled() {
+
+    when(attributes.getConcurrencyChecksEnabled()).thenReturn(false);
+
+    assertThat(put70.shouldSetPossibleDuplicate(localRegion, clientEvent)).isTrue();
+  }
+
+  @Test
+  public void shouldSetPossibleDuplicateReturnsTrueIfRecoveredVersionTagForRetriedOperation() {
+    Put70 spy = Mockito.spy(put70);
+    when(attributes.getConcurrencyChecksEnabled()).thenReturn(true);
+    doReturn(true).when(spy).recoverVersionTagForRetriedOperation(clientEvent);
+
+    assertThat(spy.shouldSetPossibleDuplicate(localRegion, clientEvent)).isTrue();
+  }
+
+  @Test
+  public void shouldSetPossibleDuplicateReturnsFalseIfNotRecoveredVersionTagAndNoPersistence() {
+    Put70 spy = Mockito.spy(put70);
+    when(attributes.getConcurrencyChecksEnabled()).thenReturn(true);
+    when(dataPolicy.withPersistence()).thenReturn(false);
+    doReturn(false).when(spy).recoverVersionTagForRetriedOperation(clientEvent);
+
+    assertThat(spy.shouldSetPossibleDuplicate(localRegion, clientEvent)).isFalse();
+  }
+
+  @Test
+  public void shouldSetPossibleDuplicateReturnsTrueIfNotRecoveredVersionTagAndWithPersistence() {
+    Put70 spy = Mockito.spy(put70);
+    when(attributes.getConcurrencyChecksEnabled()).thenReturn(true);
+    when(dataPolicy.withPersistence()).thenReturn(true);
+    doReturn(false).when(spy).recoverVersionTagForRetriedOperation(clientEvent);
+
+    assertThat(spy.shouldSetPossibleDuplicate(localRegion, clientEvent)).isTrue();
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/PutTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/PutTest.java
deleted file mode 100644
index 24cff7d..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/PutTest.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.anyBoolean;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.isNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.PutOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class PutTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] EVENT = new byte[8];
-  private static final byte[] VALUE = new byte[8];
-  private static final byte[] OK_BYTES = new byte[] {0};
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private LocalRegion localRegion;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part valuePart;
-  @Mock
-  private Part eventPart;
-  @Mock
-  private Part callbackArgsPart;
-  @Mock
-  private PutOperationContext putOperationContext;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private Message replyMessage;
-
-  @InjectMocks
-  private Put put;
-
-  @Before
-  public void setUp() throws Exception {
-    this.put = new Put();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
-        eq(CALLBACK_ARG))).thenReturn(this.putOperationContext);
-
-    when(this.putOperationContext.getCallbackArg()).thenReturn(CALLBACK_ARG);
-    when(this.putOperationContext.getSerializedValue()).thenReturn(VALUE);
-    when(this.putOperationContext.isObject()).thenReturn(true);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(this.localRegion);
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.callbackArgsPart.getObject()).thenReturn(CALLBACK_ARG);
-
-    when(this.eventPart.getSerializedForm()).thenReturn(EVENT);
-
-    when(this.valuePart.getSerializedForm()).thenReturn(VALUE);
-    when(this.valuePart.isObject()).thenReturn(true);
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(5);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(2))).thenReturn(this.valuePart);
-    when(this.message.getPart(eq(3))).thenReturn(this.eventPart);
-    when(this.message.getPart(eq(4))).thenReturn(this.callbackArgsPart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(mock(CacheServerStats.class));
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getReplyMessage()).thenReturn(this.replyMessage);
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
-
-    when(this.localRegion.basicBridgePut(eq(KEY), eq(VALUE), isNull(), eq(true), eq(CALLBACK_ARG),
-        any(), anyBoolean(), any())).thenReturn(true);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.put.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.put.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.WRITE, REGION_NAME, KEY);
-
-    this.put.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.WRITE, REGION_NAME, KEY);
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.put.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    ArgumentCaptor<byte[]> argument = ArgumentCaptor.forClass(byte[].class);
-    verify(this.replyMessage).addBytesPart(argument.capture());
-
-    assertThat(argument.getValue()).isEqualTo(OK_BYTES);
-
-    verify(this.authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
-        eq(CALLBACK_ARG));
-    verify(this.replyMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest).putAuthorize(eq(REGION_NAME),
-        eq(KEY), eq(VALUE), eq(true), eq(CALLBACK_ARG));
-
-    this.put.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).putAuthorize(eq(REGION_NAME), eq(KEY), eq(VALUE), eq(true),
-        eq(CALLBACK_ARG));
-
-    ArgumentCaptor<NotAuthorizedException> argument =
-        ArgumentCaptor.forClass(NotAuthorizedException.class);
-    verify(this.errorResponseMessage).addObjPart(argument.capture());
-
-    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
-    verify(this.errorResponseMessage).send(this.serverConnection);
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList61Test.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList61Test.java
deleted file mode 100644
index 7944990..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestList61Test.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.RegisterInterestOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class RegisterInterestList61Test {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final byte[] DURABLE = new byte[8];
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part interestTypePart;
-  @Mock
-  private Part durablePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part numberOfKeysPart;
-  @Mock
-  private Part notifyPart;
-  @Mock
-  private RegisterInterestOperationContext registerInterestOperationContext;
-  @Mock
-  private ChunkedMessage chunkedResponseMessage;
-
-  @InjectMocks
-  private RegisterInterestList61 registerInterestList61;
-
-  @Before
-  public void setUp() throws Exception {
-    this.registerInterestList61 = new RegisterInterestList61();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.registerInterestListAuthorize(eq(REGION_NAME), any(), any()))
-        .thenReturn(this.registerInterestOperationContext);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(mock(LocalRegion.class));
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.durablePart.getObject()).thenReturn(DURABLE);
-
-    when(this.interestTypePart.getInt()).thenReturn(0);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(6);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.interestTypePart);
-    when(this.message.getPart(eq(2))).thenReturn(this.durablePart);
-    when(this.message.getPart(eq(3))).thenReturn(this.numberOfKeysPart);
-    when(this.message.getPart(eq(4))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(5))).thenReturn(this.notifyPart);
-
-    when(this.notifyPart.getObject()).thenReturn(DURABLE);
-
-    when(this.numberOfKeysPart.getInt()).thenReturn(1);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.registerInterestOperationContext.getKey()).thenReturn(KEY);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getChunkedResponseMessage()).thenReturn(this.chunkedResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.GFE_80);
-    when(this.serverConnection.getAcceptor()).thenReturn(mock(AcceptorImpl.class));
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.registerInterestList61.cmdExecute(this.message, this.serverConnection,
-        this.securityService, 0);
-
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.registerInterestList61.cmdExecute(this.message, this.serverConnection,
-        this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.READ, REGION_NAME);
-
-    this.registerInterestList61.cmdExecute(this.message, this.serverConnection,
-        this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.registerInterestList61.cmdExecute(this.message, this.serverConnection,
-        this.securityService, 0);
-
-    verify(this.authzRequest).registerInterestListAuthorize(eq(REGION_NAME), any(), any());
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest)
-        .registerInterestListAuthorize(eq(REGION_NAME), any(), any());
-
-    this.registerInterestList61.cmdExecute(this.message, this.serverConnection,
-        this.securityService, 0);
-
-    verify(this.authzRequest).registerInterestListAuthorize(eq(REGION_NAME), any(), any());
-
-    ArgumentCaptor<NotAuthorizedException> argument =
-        ArgumentCaptor.forClass(NotAuthorizedException.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture());
-
-    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestListTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestListTest.java
deleted file mode 100644
index f1a4615..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestListTest.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.RegisterInterestOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class RegisterInterestListTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final byte[] DURABLE = new byte[8];
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part interestTypePart;
-  @Mock
-  private Part durablePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part numberOfKeysPart;
-  @Mock
-  private Part notifyPart;
-  @Mock
-  private RegisterInterestOperationContext registerInterestOperationContext;
-  @Mock
-  private ChunkedMessage chunkedResponseMessage;
-
-  @InjectMocks
-  private RegisterInterestList registerInterestList;
-
-  @Before
-  public void setUp() throws Exception {
-    this.registerInterestList = new RegisterInterestList();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.registerInterestListAuthorize(eq(REGION_NAME), any(), any()))
-        .thenReturn(this.registerInterestOperationContext);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(mock(LocalRegion.class));
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.durablePart.getObject()).thenReturn(DURABLE);
-
-    when(this.interestTypePart.getInt()).thenReturn(0);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(6);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.interestTypePart);
-    when(this.message.getPart(eq(2))).thenReturn(this.durablePart);
-    when(this.message.getPart(eq(3))).thenReturn(this.numberOfKeysPart);
-    when(this.message.getPart(eq(4))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(5))).thenReturn(this.notifyPart);
-
-    when(this.notifyPart.getObject()).thenReturn(DURABLE);
-
-    when(this.numberOfKeysPart.getInt()).thenReturn(1);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.registerInterestOperationContext.getKey()).thenReturn(KEY);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getChunkedResponseMessage()).thenReturn(this.chunkedResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.GFE_80);
-    when(this.serverConnection.getAcceptor()).thenReturn(mock(AcceptorImpl.class));
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.registerInterestList.cmdExecute(this.message, this.serverConnection, this.securityService,
-        0);
-
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.registerInterestList.cmdExecute(this.message, this.serverConnection, this.securityService,
-        0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.READ, REGION_NAME);
-
-    this.registerInterestList.cmdExecute(this.message, this.serverConnection, this.securityService,
-        0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.registerInterestList.cmdExecute(this.message, this.serverConnection, this.securityService,
-        0);
-
-    verify(this.authzRequest).registerInterestListAuthorize(eq(REGION_NAME), any(), any());
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest)
-        .registerInterestListAuthorize(eq(REGION_NAME), any(), any());
-
-    this.registerInterestList.cmdExecute(this.message, this.serverConnection, this.securityService,
-        0);
-
-    verify(this.authzRequest).registerInterestListAuthorize(eq(REGION_NAME), any(), any());
-
-    ArgumentCaptor<NotAuthorizedException> argument =
-        ArgumentCaptor.forClass(NotAuthorizedException.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture());
-
-    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestTest.java
deleted file mode 100644
index de2a219..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RegisterInterestTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.ArgumentCaptor;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.RegisterInterestOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class RegisterInterestTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final byte[] DURABLE = new byte[8];
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part interestTypePart;
-  @Mock
-  private Part durablePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part notifyPart;
-  @Mock
-  private ChunkedMessage chunkedResponseMessage;
-
-  @InjectMocks
-  private RegisterInterest registerInterest;
-
-  @Before
-  public void setUp() throws Exception {
-    this.registerInterest = new RegisterInterest();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.registerInterestAuthorize(eq(REGION_NAME), eq(KEY), anyInt(), any()))
-        .thenReturn(mock(RegisterInterestOperationContext.class));
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(mock(LocalRegion.class));
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.durablePart.getObject()).thenReturn(DURABLE);
-
-    when(this.interestTypePart.getInt()).thenReturn(0);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(6);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.interestTypePart);
-    when(this.message.getPart(eq(2))).thenReturn(mock(Part.class));
-    when(this.message.getPart(eq(3))).thenReturn(this.durablePart);
-    when(this.message.getPart(eq(4))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(5))).thenReturn(this.notifyPart);
-
-    when(this.notifyPart.getObject()).thenReturn(DURABLE);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getChunkedResponseMessage()).thenReturn(this.chunkedResponseMessage);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.registerInterest.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.registerInterest.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME, KEY);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldThrowIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.READ, REGION_NAME, KEY);
-
-    this.registerInterest.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME, KEY);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.registerInterest.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).registerInterestAuthorize(eq(REGION_NAME), eq(KEY), anyInt(), any());
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest)
-        .registerInterestAuthorize(eq(REGION_NAME), eq(KEY), anyInt(), any());
-
-    this.registerInterest.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).registerInterestAuthorize(eq(REGION_NAME), eq(KEY), anyInt(), any());
-
-    ArgumentCaptor<NotAuthorizedException> argument =
-        ArgumentCaptor.forClass(NotAuthorizedException.class);
-    verify(this.chunkedResponseMessage).addObjPart(argument.capture());
-
-    assertThat(argument.getValue()).isExactlyInstanceOf(NotAuthorizedException.class);
-    verify(this.chunkedResponseMessage).sendChunk(this.serverConnection);
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RequestTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RequestTest.java
deleted file mode 100644
index 60950e2..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/RequestTest.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache.tier.sockets.command;
-
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.ArgumentMatchers.isA;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.operations.GetOperationContext;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.test.junit.categories.ClientServerTest;
-
-@Category({ClientServerTest.class})
-public class RequestTest {
-
-  private static final String REGION_NAME = "region1";
-  private static final String KEY = "key1";
-  private static final Object CALLBACK_ARG = "arg";
-  private static final byte[] EVENT = new byte[8];
-
-  @Mock
-  private SecurityService securityService;
-  @Mock
-  private Message message;
-  @Mock
-  private ServerConnection serverConnection;
-  @Mock
-  private AuthorizeRequest authzRequest;
-  @Mock
-  private LocalRegion region;
-  @Mock
-  private InternalCache cache;
-  @Mock
-  private CacheServerStats cacheServerStats;
-  @Mock
-  private Message responseMessage;
-  @Mock
-  private Message errorResponseMessage;
-  @Mock
-  private Part regionNamePart;
-  @Mock
-  private Part keyPart;
-  @Mock
-  private Part valuePart;
-  @Mock
-  private GetOperationContext getOperationContext;
-  @InjectMocks
-  private Request request;
-
-  @Before
-  public void setUp() throws Exception {
-    this.request = new Request();
-    MockitoAnnotations.initMocks(this);
-
-    when(this.authzRequest.getAuthorize(any(), any(), any())).thenReturn(this.getOperationContext);
-
-    when(this.cache.getRegion(isA(String.class))).thenReturn(this.region);
-    when(this.cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
-
-    when(this.getOperationContext.getCallbackArg()).thenReturn(CALLBACK_ARG);
-
-    when(this.keyPart.getStringOrObject()).thenReturn(KEY);
-
-    when(this.message.getNumberOfParts()).thenReturn(3);
-    when(this.message.getPart(eq(0))).thenReturn(this.regionNamePart);
-    when(this.message.getPart(eq(1))).thenReturn(this.keyPart);
-    when(this.message.getPart(eq(2))).thenReturn(this.valuePart);
-
-    when(this.regionNamePart.getCachedString()).thenReturn(REGION_NAME);
-
-    when(this.serverConnection.getCache()).thenReturn(this.cache);
-    when(this.serverConnection.getCacheServerStats()).thenReturn(this.cacheServerStats);
-    when(this.serverConnection.getAuthzRequest()).thenReturn(this.authzRequest);
-    when(this.serverConnection.getResponseMessage()).thenReturn(this.responseMessage);
-    when(this.serverConnection.getCachedRegionHelper()).thenReturn(mock(CachedRegionHelper.class));
-    when(this.serverConnection.getErrorResponseMessage()).thenReturn(this.errorResponseMessage);
-    when(this.serverConnection.getClientVersion()).thenReturn(KnownVersion.CURRENT);
-
-    when(this.valuePart.getObject()).thenReturn(CALLBACK_ARG);
-  }
-
-  @Test
-  public void noSecurityShouldSucceed() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(false);
-
-    this.request.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-    verify(this.responseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-
-    this.request.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME, KEY);
-    verify(this.responseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void integratedSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(true);
-    doThrow(new NotAuthorizedException("")).when(this.securityService).authorize(Resource.DATA,
-        Operation.READ, REGION_NAME, KEY);
-
-    this.request.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.securityService).authorize(Resource.DATA, Operation.READ, REGION_NAME, KEY);
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-  @Test
-  public void oldSecurityShouldSucceedIfAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-
-    this.request.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).getAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-    verify(this.responseMessage).send(this.serverConnection);
-  }
-
-  @Test
-  public void oldSecurityShouldFailIfNotAuthorized() throws Exception {
-    when(this.securityService.isClientSecurityRequired()).thenReturn(true);
-    when(this.securityService.isIntegratedSecurity()).thenReturn(false);
-    doThrow(new NotAuthorizedException("")).when(this.authzRequest).getAuthorize(eq(REGION_NAME),
-        eq(KEY), eq(CALLBACK_ARG));
-
-    this.request.cmdExecute(this.message, this.serverConnection, this.securityService, 0);
-
-    verify(this.authzRequest).getAuthorize(eq(REGION_NAME), eq(KEY), eq(CALLBACK_ARG));
-    verify(this.errorResponseMessage).send(eq(this.serverConnection));
-  }
-
-}
diff --git a/geode-cq/src/integrationTest/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImplTest.java b/geode-cq/src/integrationTest/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImplTest.java
new file mode 100644
index 0000000..1e92fea
--- /dev/null
+++ b/geode-cq/src/integrationTest/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImplTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.cq.internal;
+
+import static java.util.Arrays.asList;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Test;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.sockets.CommandInitializer;
+import org.apache.geode.internal.cache.tier.sockets.CommandRegistry;
+import org.apache.geode.internal.serialization.KnownVersion;
+
+public class CqServiceFactoryImplTest {
+  @Test
+  public void registersCommandsOnCreate() {
+    final InternalCache internalCache = mock(InternalCache.class);
+    final CancelCriterion cancelCriterion = mock(CancelCriterion.class);
+    final DistributedSystem distributedSystem = mock(DistributedSystem.class);
+    doNothing().when(cancelCriterion).checkCancelInProgress(null);
+    when(internalCache.getCancelCriterion()).thenReturn(cancelCriterion);
+    when(internalCache.getDistributedSystem()).thenReturn(distributedSystem);
+
+    CommandRegistry commandRegistry = new CommandInitializer();
+
+    final Integer[] messageTypes = {
+        MessageType.EXECUTECQ_MSG_TYPE, MessageType.EXECUTECQ_WITH_IR_MSG_TYPE,
+        MessageType.GETCQSTATS_MSG_TYPE, MessageType.MONITORCQ_MSG_TYPE,
+        MessageType.STOPCQ_MSG_TYPE, MessageType.CLOSECQ_MSG_TYPE,
+        MessageType.GETDURABLECQS_MSG_TYPE};
+
+    final Set<Integer> initialKeys = commandRegistry.get(KnownVersion.OLDEST).keySet();
+    assertThat(initialKeys).doesNotContain(messageTypes);
+
+    final CqServiceFactoryImpl cqServiceFactory = new CqServiceFactoryImpl();
+    cqServiceFactory.initialize();
+    cqServiceFactory.create(internalCache, commandRegistry);
+
+    final Set<Integer> expectedKeys = new HashSet<>(initialKeys);
+    expectedKeys.addAll(asList(messageTypes));
+    assertThat(commandRegistry.get(KnownVersion.OLDEST)).containsOnlyKeys(expectedKeys);
+  }
+}
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImpl.java
index 45a8928..60cee60 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImpl.java
@@ -12,16 +12,15 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+
 package org.apache.geode.cache.query.cq.internal;
 
+import static java.util.Collections.singletonMap;
+
 import java.io.DataInput;
 import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.geode.cache.query.cq.internal.command.CloseCQ;
-import org.apache.geode.cache.query.cq.internal.command.ExecuteCQ;
 import org.apache.geode.cache.query.cq.internal.command.ExecuteCQ61;
 import org.apache.geode.cache.query.cq.internal.command.GetCQStats;
 import org.apache.geode.cache.query.cq.internal.command.GetDurableCQs;
@@ -31,35 +30,32 @@
 import org.apache.geode.cache.query.internal.cq.ServerCQ;
 import org.apache.geode.cache.query.internal.cq.spi.CqServiceFactory;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.CommandInitializer;
+import org.apache.geode.internal.cache.tier.sockets.CommandRegistry;
 import org.apache.geode.internal.serialization.KnownVersion;
 
 public class CqServiceFactoryImpl implements CqServiceFactory {
 
   @Override
-  public void initialize() {
-    Map<KnownVersion, Command> versions = new HashMap<>();
-    versions.put(KnownVersion.GFE_57, ExecuteCQ.getCommand());
-    versions.put(KnownVersion.GFE_61, ExecuteCQ61.getCommand());
-    CommandInitializer.registerCommand(MessageType.EXECUTECQ_MSG_TYPE, versions);
-    CommandInitializer.registerCommand(MessageType.EXECUTECQ_WITH_IR_MSG_TYPE, versions);
-
-    CommandInitializer.registerCommand(MessageType.GETCQSTATS_MSG_TYPE,
-        Collections.singletonMap(KnownVersion.GFE_57, GetCQStats.getCommand()));
-    CommandInitializer.registerCommand(MessageType.MONITORCQ_MSG_TYPE,
-        Collections.singletonMap(KnownVersion.GFE_57, MonitorCQ.getCommand()));
-    CommandInitializer.registerCommand(MessageType.STOPCQ_MSG_TYPE,
-        Collections.singletonMap(KnownVersion.GFE_57, StopCQ.getCommand()));
-    CommandInitializer.registerCommand(MessageType.CLOSECQ_MSG_TYPE,
-        Collections.singletonMap(KnownVersion.GFE_57, CloseCQ.getCommand()));
-    CommandInitializer.registerCommand(MessageType.GETDURABLECQS_MSG_TYPE,
-        Collections.singletonMap(KnownVersion.GFE_70, GetDurableCQs.getCommand()));
-  }
+  public void initialize() {}
 
   @Override
-  public CqService create(InternalCache cache) {
+  public CqService create(InternalCache cache, CommandRegistry commandRegistry) {
+    commandRegistry.register(MessageType.EXECUTECQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, ExecuteCQ61.getCommand()));
+    commandRegistry.register(MessageType.EXECUTECQ_WITH_IR_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, ExecuteCQ61.getCommand()));
+    commandRegistry.register(MessageType.GETCQSTATS_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, GetCQStats.getCommand()));
+    commandRegistry.register(MessageType.MONITORCQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, MonitorCQ.getCommand()));
+    commandRegistry.register(MessageType.STOPCQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, StopCQ.getCommand()));
+    commandRegistry.register(MessageType.CLOSECQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, CloseCQ.getCommand()));
+    commandRegistry.register(MessageType.GETDURABLECQS_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, GetDurableCQs.getCommand()));
+
     return new CqServiceImpl(cache);
   }
 
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/command/ExecuteCQ.java b/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/command/ExecuteCQ.java
deleted file mode 100644
index 0f10c07..0000000
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/cq/internal/command/ExecuteCQ.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.cache.query.cq.internal.command;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.cache.operations.ExecuteCQOperationContext;
-import org.apache.geode.cache.query.CqException;
-import org.apache.geode.cache.query.Query;
-import org.apache.geode.cache.query.internal.DefaultQuery;
-import org.apache.geode.cache.query.internal.DefaultQueryService;
-import org.apache.geode.cache.query.internal.cq.CqService;
-import org.apache.geode.cache.query.internal.cq.ServerCQ;
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.tier.Acceptor;
-import org.apache.geode.internal.cache.tier.CachedRegionHelper;
-import org.apache.geode.internal.cache.tier.Command;
-import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
-import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
-import org.apache.geode.internal.cache.tier.sockets.Message;
-import org.apache.geode.internal.cache.tier.sockets.Part;
-import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
-import org.apache.geode.internal.security.AuthorizeRequest;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.logging.internal.log4j.api.LogService;
-
-public class ExecuteCQ extends BaseCQCommand {
-  protected static final Logger logger = LogService.getLogger();
-
-  private static final ExecuteCQ singleton = new ExecuteCQ();
-
-  public static Command getCommand() {
-    return singleton;
-  }
-
-  private ExecuteCQ() {
-    // nothing
-  }
-
-  @Override
-  public void cmdExecute(final Message clientMessage, final ServerConnection serverConnection,
-      final SecurityService securityService, long start) throws IOException, InterruptedException {
-    Acceptor acceptor = serverConnection.getAcceptor();
-    CachedRegionHelper crHelper = serverConnection.getCachedRegionHelper();
-    ClientProxyMembershipID id = serverConnection.getProxyID();
-    CacheServerStats stats = serverConnection.getCacheServerStats();
-
-    serverConnection.setAsTrue(REQUIRES_RESPONSE);
-    serverConnection.setAsTrue(REQUIRES_CHUNKED_RESPONSE);
-
-    // Retrieve the data from the message parts
-    String cqName = clientMessage.getPart(0).getString();
-    String cqQueryString = clientMessage.getPart(1).getString();
-    int cqState = clientMessage.getPart(2).getInt();
-
-    Part isDurablePart = clientMessage.getPart(3);
-    byte[] isDurableByte = isDurablePart.getSerializedForm();
-    boolean isDurable = !(isDurableByte == null || isDurableByte[0] == 0);
-    if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received {} request from {} CqName: {} queryString: {}",
-          serverConnection.getName(), MessageType.getString(clientMessage.getMessageType()),
-          serverConnection.getSocketString(), cqName, cqQueryString);
-    }
-
-    DefaultQueryService qService;
-    CqService cqServiceForExec;
-    Query query = null;
-    Set cqRegionNames = null;
-    ExecuteCQOperationContext executeCQContext = null;
-    ServerCQ cqQuery;
-
-    try {
-      qService = (DefaultQueryService) crHelper.getCache().getLocalQueryService();
-
-      // Authorization check
-      AuthorizeRequest authzRequest = serverConnection.getAuthzRequest();
-      if (authzRequest != null) {
-        query = qService.newQuery(cqQueryString);
-        cqRegionNames = ((DefaultQuery) query).getRegionsInQuery(null);
-        executeCQContext = authzRequest.executeCQAuthorize(cqName, cqQueryString, cqRegionNames);
-        String newCqQueryString = executeCQContext.getQuery();
-
-        if (!cqQueryString.equals(newCqQueryString)) {
-          query = qService.newQuery(newCqQueryString);
-          cqQueryString = newCqQueryString;
-          cqRegionNames = executeCQContext.getRegionNames();
-          if (cqRegionNames == null) {
-            cqRegionNames = ((DefaultQuery) query).getRegionsInQuery(null);
-          }
-        }
-      }
-
-      cqServiceForExec = qService.getCqService();
-      cqQuery = cqServiceForExec.executeCq(cqName, cqQueryString, cqState, id,
-          acceptor.getCacheClientNotifier(), isDurable, false, 0, null);
-    } catch (CqException cqe) {
-      sendCqResponse(MessageType.CQ_EXCEPTION_TYPE, "", clientMessage.getTransactionId(), cqe,
-          serverConnection);
-      return;
-    } catch (Exception e) {
-      writeChunkedException(clientMessage, e, serverConnection);
-      return;
-    }
-
-    boolean sendResults = false;
-
-    if (clientMessage.getMessageType() == MessageType.EXECUTECQ_WITH_IR_MSG_TYPE) {
-      sendResults = true;
-    }
-
-    // Execute the query and send the result-set to client.
-    boolean successQuery = false;
-    try {
-      if (query == null) {
-        query = qService.newQuery(cqQueryString);
-        cqRegionNames = ((DefaultQuery) query).getRegionsInQuery(null);
-      }
-      ((DefaultQuery) query).setIsCqQuery(true);
-      successQuery =
-          processQuery(clientMessage, query, cqQueryString, cqRegionNames,
-              start, cqQuery, executeCQContext, serverConnection, sendResults, securityService);
-
-      // Update the CQ statistics.
-      cqQuery.getVsdStats().setCqInitialResultsTime(DistributionStats.getStatTime() - start);
-      stats.incProcessExecuteCqWithIRTime(DistributionStats.getStatTime() - start);
-      // logger.fine("Time spent in execute with initial results :" +
-      // DistributionStats.getStatTime() + ", " + oldstart);
-    } finally { // To handle any exception.
-      // If failure to execute the query, close the CQ.
-      if (!successQuery) {
-        try {
-          cqServiceForExec.closeCq(cqName, id);
-        } catch (Exception ignore) {
-          // Ignore.
-        }
-      }
-    }
-
-    if (!sendResults && successQuery) {
-      // Send OK to client
-      sendCqResponse(MessageType.REPLY,
-          "cq created successfully.",
-          clientMessage.getTransactionId(), null, serverConnection);
-
-      long start2 = DistributionStats.getStatTime();
-      stats.incProcessCreateCqTime(start2 - start);
-    }
-    serverConnection.setAsTrue(RESPONDED);
-  }
-
-}
diff --git a/geode-cq/src/test/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImplTest.java b/geode-cq/src/test/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImplTest.java
new file mode 100644
index 0000000..69b7e6a
--- /dev/null
+++ b/geode-cq/src/test/java/org/apache/geode/cache/query/cq/internal/CqServiceFactoryImplTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.cq.internal;
+
+import static java.util.Collections.singletonMap;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import org.junit.Test;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.cache.query.cq.internal.command.CloseCQ;
+import org.apache.geode.cache.query.cq.internal.command.ExecuteCQ61;
+import org.apache.geode.cache.query.cq.internal.command.GetCQStats;
+import org.apache.geode.cache.query.cq.internal.command.GetDurableCQs;
+import org.apache.geode.cache.query.cq.internal.command.MonitorCQ;
+import org.apache.geode.cache.query.cq.internal.command.StopCQ;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.tier.MessageType;
+import org.apache.geode.internal.cache.tier.sockets.CommandRegistry;
+import org.apache.geode.internal.serialization.KnownVersion;
+
+public class CqServiceFactoryImplTest {
+
+  @Test
+  public void registersCommandsOnCreate() {
+    final InternalCache internalCache = mock(InternalCache.class);
+    final CancelCriterion cancelCriterion = mock(CancelCriterion.class);
+    final DistributedSystem distributedSystem = mock(DistributedSystem.class);
+    doNothing().when(cancelCriterion).checkCancelInProgress(null);
+    when(internalCache.getCancelCriterion()).thenReturn(cancelCriterion);
+    when(internalCache.getDistributedSystem()).thenReturn(distributedSystem);
+
+    final CommandRegistry commandRegistry = mock(CommandRegistry.class);
+
+    final CqServiceFactoryImpl cqServiceFactory = new CqServiceFactoryImpl();
+    cqServiceFactory.initialize();
+    cqServiceFactory.create(internalCache, commandRegistry);
+
+    verify(commandRegistry).register(MessageType.EXECUTECQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, ExecuteCQ61.getCommand()));
+    verify(commandRegistry).register(MessageType.EXECUTECQ_WITH_IR_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, ExecuteCQ61.getCommand()));
+    verify(commandRegistry).register(MessageType.GETCQSTATS_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, GetCQStats.getCommand()));
+    verify(commandRegistry).register(MessageType.MONITORCQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, MonitorCQ.getCommand()));
+    verify(commandRegistry).register(MessageType.STOPCQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, StopCQ.getCommand()));
+    verify(commandRegistry).register(MessageType.CLOSECQ_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, CloseCQ.getCommand()));
+    verify(commandRegistry).register(MessageType.GETDURABLECQS_MSG_TYPE,
+        singletonMap(KnownVersion.OLDEST, GetDurableCQs.getCommand()));
+
+    verifyNoMoreInteractions(commandRegistry);
+  }
+
+}
diff --git a/geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index 247a0e6..0fb00d0 100644
--- a/geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -175,7 +175,7 @@
     mockOldMember = services.getMemberFactory().create(
         MemberDataBuilder.newBuilderForLocalHost("localhost")
             .setMembershipPort(8700).build());
-    ((MemberIdentifierImpl) mockOldMember).setVersionForTest(KnownVersion.GFE_56);
+    ((MemberIdentifierImpl) mockOldMember).setVersionForTest(KnownVersion.OLDEST);
     locatorClient = mock(TcpClient.class);
 
     if (useTestGMSJoinLeave) {
diff --git a/geode-old-client-support/src/distributedTest/java/org/apache/geode/OldClientSupportDUnitTest.java b/geode-old-client-support/src/distributedTest/java/org/apache/geode/OldClientSupportDUnitTest.java
index 17eff19..e0b8f5b 100644
--- a/geode-old-client-support/src/distributedTest/java/org/apache/geode/OldClientSupportDUnitTest.java
+++ b/geode-old-client-support/src/distributedTest/java/org/apache/geode/OldClientSupportDUnitTest.java
@@ -14,13 +14,15 @@
  */
 package org.apache.geode;
 
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
@@ -31,7 +33,6 @@
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.distributed.internal.DistributedSystemService;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.internal.HeapDataOutputStream;
@@ -52,17 +53,17 @@
 public class OldClientSupportDUnitTest extends JUnit4CacheTestCase {
 
   private static final List<String> allGeodeThrowableClasses =
-      Arrays.asList(new String[] {"org.apache.geode.cache.execute.EmptyRegionFunctionException",});
+      singletonList("org.apache.geode.cache.execute.EmptyRegionFunctionException");
 
-  private static final List<String> newArrayClassNames = Arrays.asList(new String[] {
-      "[Lorg.apache.geode.class1", "[[Lorg.apache.geode.class1", "[[[Lorg.apache.geode.class1"});
+  private static final List<String> newArrayClassNames = asList("[Lorg.apache.geode.class1",
+      "[[Lorg.apache.geode.class1", "[[[Lorg.apache.geode.class1");
 
   private static final List<String> oldArrayClassNames =
-      Arrays.asList(new String[] {"[Lcom.gemstone.gemfire.class1", "[[Lcom.gemstone.gemfire.class1",
-          "[[[Lcom.gemstone.gemfire.class1"});
+      asList("[Lcom.gemstone.gemfire.class1", "[[Lcom.gemstone.gemfire.class1",
+          "[[[Lcom.gemstone.gemfire.class1");
 
-  private static final List<String> allNonconformingArrayClassNames = Arrays.asList(
-      new String[] {"[Lmypackage.org.apache.geode.class2", "[[Lmypackage.org.apache.geode.class2"});
+  private static final List<String> allNonconformingArrayClassNames = asList(
+      "[Lmypackage.org.apache.geode.class2", "[[Lmypackage.org.apache.geode.class2");
 
   private Cache myCache;
 
@@ -73,7 +74,7 @@
   }
 
   @Test
-  public void cacheInstallsOldClientSupportServiceProvider() throws Exception {
+  public void cacheInstallsOldClientSupportServiceProvider() {
     Assert.assertNotNull(((InternalCache) myCache).getService(OldClientSupportService.class));
   }
 
@@ -83,7 +84,7 @@
    * EmtpyRegionFunctionException.
    */
   @Test
-  public void testConversionOfThrowablesForOldClients() throws Exception {
+  public void testConversionOfThrowablesForOldClients() {
     List<Throwable> problems = new LinkedList<>();
 
     for (String geodeClassName : allGeodeThrowableClasses) {
@@ -92,8 +93,7 @@
       } catch (Exception e) {
         System.out.println("-- failed");
         Exception failure =
-            new Exception("Failed processing " + geodeClassName + ": " + e.toString());
-        failure.initCause(e);
+            new Exception("Failed processing " + geodeClassName + ": " + e.toString(), e);
         problems.add(failure);
       }
     }
@@ -104,10 +104,10 @@
   }
 
   @Test
-  public void testConversionOfArrayTypes() throws Exception {
+  public void testConversionOfArrayTypes() {
     OldClientSupportService oldClientSupport = OldClientSupportProvider.getService(myCache);
 
-    KnownVersion oldClientVersion = KnownVersion.GFE_82;
+    KnownVersion oldClientVersion = KnownVersion.GFE_81;
     VersionedDataOutputStream dout = new VersionedDataOutputStream(
         new HeapDataOutputStream(10, oldClientVersion), oldClientVersion);
 
@@ -132,14 +132,14 @@
   }
 
   private void convertThrowable(String geodeClassName) throws Exception {
-    KnownVersion oldClientVersion = KnownVersion.GFE_82;
+    KnownVersion oldClientVersion = KnownVersion.GFE_81;
     final String comGemstoneGemFire = "com.gemstone.gemfire";
     final int comGemstoneGemFireLength = comGemstoneGemFire.length();
 
     OldClientSupportService oldClientSupport = OldClientSupportProvider.getService(myCache);
 
     System.out.println("checking " + geodeClassName);
-    Class geodeClass = Class.forName(geodeClassName);
+    Class<?> geodeClass = Class.forName(geodeClassName);
     Object geodeObject = instantiate(geodeClass);
     if (geodeObject instanceof Throwable) {
       Throwable geodeThrowable = (Throwable) instantiate(geodeClass);
@@ -153,7 +153,7 @@
   }
 
   private Object instantiate(Class<?> aClass) throws Exception {
-    Constructor<?> c = null;
+    Constructor<?> c;
     try {
       c = aClass.getConstructor();
       return c.newInstance();
@@ -183,7 +183,7 @@
     properties.put(ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER,
         "org.apache.geode.ClientSerializableObjec");
     DistributionConfig config = new DistributionConfigImpl(properties);
-    InternalDataSerializer.initialize(config, new ArrayList<DistributedSystemService>());
+    InternalDataSerializer.initialize(config, new ArrayList<>());
 
     com.gemstone.gemfire.ClientSerializableObject gemfireObject =
         new com.gemstone.gemfire.ClientSerializableObject();
@@ -198,7 +198,7 @@
     byte[] serializedForm = byteStream.toByteArray();
 
     ByteArrayDataInput byteDataInput = new ByteArrayDataInput();
-    byteDataInput.initialize(serializedForm, KnownVersion.GFE_82);
+    byteDataInput.initialize(serializedForm, KnownVersion.GFE_81);
     ClientSerializableObject result = DataSerializer.readObject(byteDataInput);
     Assert.assertEquals("Expected an org.apache.geode exception but found " + result,
         result.getClass().getName().substring(0, "org.apache.geode".length()), "org.apache.geode");
@@ -225,7 +225,7 @@
     byte[] serializedForm = byteStream.toByteArray();
 
     ByteArrayDataInput byteDataInput = new ByteArrayDataInput();
-    byteDataInput.initialize(serializedForm, KnownVersion.GFE_82);
+    byteDataInput.initialize(serializedForm, KnownVersion.GFE_81);
     Object result = DataSerializer.readObject(byteDataInput);
     Assert.assertEquals("Expected an org.apache.geode object but found " + result,
         result.getClass().getName().substring(0, "org.apache.geode".length()), "org.apache.geode");
@@ -249,7 +249,7 @@
     byte[] serializedForm = byteStream.toByteArray();
 
     ByteArrayDataInput byteDataInput = new ByteArrayDataInput();
-    byteDataInput.initialize(serializedForm, KnownVersion.GFE_82);
+    byteDataInput.initialize(serializedForm, KnownVersion.GFE_81);
     Object result = DataSerializer.readObject(byteDataInput);
     Assert.assertEquals("Expected an org.apache.geode object but found " + result,
         result.getClass().getName().substring(0, "org.apache.geode".length()), "org.apache.geode");
diff --git a/geode-serialization/src/main/java/org/apache/geode/internal/serialization/KnownVersion.java b/geode-serialization/src/main/java/org/apache/geode/internal/serialization/KnownVersion.java
index d3edbd5..434c2d7 100644
--- a/geode-serialization/src/main/java/org/apache/geode/internal/serialization/KnownVersion.java
+++ b/geode-serialization/src/main/java/org/apache/geode/internal/serialization/KnownVersion.java
@@ -42,8 +42,8 @@
   private final transient String methodSuffix;
 
   // the major, minor and release bits of the release
-  private final byte majorVersion;
-  private final byte minorVersion;
+  private final byte major;
+  private final byte minor;
   private final byte release;
   private final byte patch;
 
@@ -62,93 +62,101 @@
   public static final KnownVersion TOKEN =
       new KnownVersion("", "TOKEN", (byte) -1, (byte) 0, (byte) 0, (byte) 0, TOKEN_ORDINAL);
 
-  private static final short GFE_56_ORDINAL = 0;
-
-  @Immutable
-  public static final KnownVersion GFE_56 =
-      new KnownVersion("GFE", "5.6", (byte) 5, (byte) 6, (byte) 0, (byte) 0, GFE_56_ORDINAL);
-
   private static final short GFE_57_ORDINAL = 1;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_57 =
       new KnownVersion("GFE", "5.7", (byte) 5, (byte) 7, (byte) 0, (byte) 0, GFE_57_ORDINAL);
 
   private static final short GFE_58_ORDINAL = 3;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_58 =
       new KnownVersion("GFE", "5.8", (byte) 5, (byte) 8, (byte) 0, (byte) 0, GFE_58_ORDINAL);
 
   private static final short GFE_603_ORDINAL = 4;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_603 =
       new KnownVersion("GFE", "6.0.3", (byte) 6, (byte) 0, (byte) 3, (byte) 0, GFE_603_ORDINAL);
 
   private static final short GFE_61_ORDINAL = 5;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_61 =
       new KnownVersion("GFE", "6.1", (byte) 6, (byte) 1, (byte) 0, (byte) 0, GFE_61_ORDINAL);
 
   private static final short GFE_65_ORDINAL = 6;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_65 =
       new KnownVersion("GFE", "6.5", (byte) 6, (byte) 5, (byte) 0, (byte) 0, GFE_65_ORDINAL);
 
   private static final short GFE_651_ORDINAL = 7;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_651 =
       new KnownVersion("GFE", "6.5.1", (byte) 6, (byte) 5, (byte) 1, (byte) 0, GFE_651_ORDINAL);
 
   private static final short GFE_6516_ORDINAL = 12;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_6516 =
       new KnownVersion("GFE", "6.5.1.6", (byte) 6, (byte) 5, (byte) 1, (byte) 6, GFE_6516_ORDINAL);
 
   private static final short GFE_66_ORDINAL = 16;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_66 =
       new KnownVersion("GFE", "6.6", (byte) 6, (byte) 6, (byte) 0, (byte) 0, GFE_66_ORDINAL);
 
   private static final short GFE_662_ORDINAL = 17;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_662 =
       new KnownVersion("GFE", "6.6.2", (byte) 6, (byte) 6, (byte) 2, (byte) 0, GFE_662_ORDINAL);
 
   private static final short GFE_6622_ORDINAL = 18;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_6622 =
       new KnownVersion("GFE", "6.6.2.2", (byte) 6, (byte) 6, (byte) 2, (byte) 2, GFE_6622_ORDINAL);
 
   private static final short GFE_70_ORDINAL = 19;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_70 =
       new KnownVersion("GFE", "7.0", (byte) 7, (byte) 0, (byte) 0, (byte) 0, GFE_70_ORDINAL);
 
   private static final short GFE_701_ORDINAL = 20;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_701 =
       new KnownVersion("GFE", "7.0.1", (byte) 7, (byte) 0, (byte) 1, (byte) 0, GFE_701_ORDINAL);
 
   private static final short GFE_7099_ORDINAL = 21;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_7099 =
       new KnownVersion("GFE", "7.0.99", (byte) 7, (byte) 0, (byte) 99, (byte) 0, GFE_7099_ORDINAL);
 
   private static final short GFE_71_ORDINAL = 22;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_71 =
       new KnownVersion("GFE", "7.1", (byte) 7, (byte) 1, (byte) 0, (byte) 0, GFE_71_ORDINAL);
 
@@ -157,6 +165,7 @@
   private static final short GFE_80_ORDINAL = 30;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_80 =
       new KnownVersion("GFE", "8.0", (byte) 8, (byte) 0, (byte) 0, (byte) 0, GFE_80_ORDINAL);
 
@@ -165,6 +174,7 @@
   private static final short GFE_8009_ORDINAL = 31;
 
   @Immutable
+  @Deprecated
   public static final KnownVersion GFE_8009 =
       new KnownVersion("GFE", "8.0.0.9", (byte) 8, (byte) 0, (byte) 0, (byte) 9, GFE_8009_ORDINAL);
 
@@ -176,13 +186,7 @@
 
   // 36-39 available for 8.1.x variants
 
-  private static final short GFE_82_ORDINAL = 40;
-
-  @Immutable
-  public static final KnownVersion GFE_82 =
-      new KnownVersion("GFE", "8.2", (byte) 8, (byte) 2, (byte) 0, (byte) 0, GFE_82_ORDINAL);
-
-  // 41-44 available for 8.2.x variants
+  // 40-44 was reserved for 8.2.x but was never used in any release.
 
   private static final short GFE_90_ORDINAL = 45; // this is also GEODE 1.0.0-incubating
 
@@ -316,6 +320,12 @@
   /* NOTE: when adding a new version bump the ordinal by 5. Ordinals can be short ints */
 
   /**
+   * The oldest non-deprecated version supported.
+   */
+  @Immutable
+  public static final KnownVersion OLDEST = GFE_81;
+
+  /**
    * This constant must be set to the most current version of the product. !!! NOTE: update
    * HIGHEST_VERSION when changing CURRENT !!!
    */
@@ -338,19 +348,19 @@
       new KnownVersion("TEST", "VERSION", (byte) 0, (byte) 0,
           (byte) 0, (byte) 0, validOrdinalForTesting);
 
-  /** Creates a new instance of <code>Version</code> */
-  private KnownVersion(String product, String name, byte major, byte minor, byte release,
-      byte patch,
-      short ordinal) {
+  private KnownVersion(String productName, String name, byte major, byte minor, byte release,
+      byte patch, short ordinal) {
     super(ordinal);
-    this.productName = product;
+    this.productName = productName;
     this.name = name;
-    this.majorVersion = major;
-    this.minorVersion = minor;
+    this.major = major;
+    this.minor = minor;
     this.release = release;
     this.patch = patch;
-    this.methodSuffix = this.productName + "_" + this.majorVersion + "_" + this.minorVersion + "_"
-        + this.release + "_" + this.patch;
+
+    methodSuffix = this.productName + "_" + this.major + "_" + this.minor + "_" + this.release + "_"
+        + this.patch;
+
     if (ordinal != TOKEN_ORDINAL) {
       VALUES[ordinal()] = this;
     }
@@ -367,61 +377,48 @@
   }
 
   public String getMethodSuffix() {
-    return this.methodSuffix;
+    return methodSuffix;
   }
 
   public String getName() {
-    return this.name;
+    return name;
   }
 
-  public short getMajorVersion() {
-    return this.majorVersion;
+  public short getMajor() {
+    return major;
   }
 
-  public short getMinorVersion() {
-    return this.minorVersion;
+  public short getMinor() {
+    return minor;
   }
 
   public short getRelease() {
-    return this.release;
+    return release;
   }
 
   public short getPatch() {
-    return this.patch;
+    return patch;
   }
 
-  /**
-   * Returns a string representation for this <code>Version</code>.
-   *
-   * @return the name of this operation.
-   */
   @Override
   public String toString() {
-    return this.productName + " " + this.name;
-  }
-
-  public byte[] toBytes() {
-    byte[] bytes = new byte[2];
-    bytes[0] = (byte) (ordinal() >> 8);
-    bytes[1] = (byte) ordinal();
-    return bytes;
+    return productName + " " + name;
   }
 
   public static Iterable<? extends KnownVersion> getAllVersions() {
-    return Arrays.asList(VALUES).stream().filter(x -> x != null && x != TEST_VERSION)
+    return Arrays.stream(VALUES).filter(x -> x != null && x != TEST_VERSION)
         .collect(Collectors.toList());
   }
 
   /**
-   * package-protected for use by Versioning factory
+   * @return KnownVersion for ordinal if known, otherwise null.
    */
-  static KnownVersion getKnownVersionOrDefault(final short ordinal,
-      final KnownVersion defaultKnownVersion) {
+  static KnownVersion getKnownVersion(final short ordinal) {
     if (ordinal == TOKEN_ORDINAL) {
       return TOKEN;
     }
-    if (ordinal < TOKEN_ORDINAL || ordinal >= VALUES.length || VALUES[ordinal] == null) {
-      return defaultKnownVersion;
+    if (ordinal < TOKEN_ORDINAL || ordinal >= VALUES.length) {
+      return null;
     }
     return VALUES[ordinal];
   }
diff --git a/geode-serialization/src/main/java/org/apache/geode/internal/serialization/Versioning.java b/geode-serialization/src/main/java/org/apache/geode/internal/serialization/Versioning.java
index 985b272..cab062a 100644
--- a/geode-serialization/src/main/java/org/apache/geode/internal/serialization/Versioning.java
+++ b/geode-serialization/src/main/java/org/apache/geode/internal/serialization/Versioning.java
@@ -36,7 +36,7 @@
    *         {@link UnknownVersion}.
    */
   public static Version getVersion(final short ordinal) {
-    final KnownVersion knownVersion = KnownVersion.getKnownVersionOrDefault(ordinal, null);
+    final KnownVersion knownVersion = KnownVersion.getKnownVersion(ordinal);
     if (knownVersion == null) {
       return new UnknownVersion(ordinal);
     } else {
diff --git a/geode-serialization/src/test/java/org/apache/geode/internal/serialization/KnownVersionJUnitTest.java b/geode-serialization/src/test/java/org/apache/geode/internal/serialization/KnownVersionJUnitTest.java
index 178ec28..52e98c2 100644
--- a/geode-serialization/src/test/java/org/apache/geode/internal/serialization/KnownVersionJUnitTest.java
+++ b/geode-serialization/src/test/java/org/apache/geode/internal/serialization/KnownVersionJUnitTest.java
@@ -21,18 +21,12 @@
 import org.junit.Test;
 
 public class KnownVersionJUnitTest {
+
   @Test
-  public void testKnownVersionClass() throws Exception {
-    compare(KnownVersion.GFE_662, KnownVersion.GFE_66);
-    compare(KnownVersion.GFE_6622, KnownVersion.GFE_662);
-    compare(KnownVersion.GFE_71, KnownVersion.GFE_70);
-    compare(KnownVersion.GFE_80, KnownVersion.GFE_70);
-    compare(KnownVersion.GFE_80, KnownVersion.GFE_71);
-    compare(KnownVersion.GFE_81, KnownVersion.GFE_70);
-    compare(KnownVersion.GFE_81, KnownVersion.GFE_71);
-    compare(KnownVersion.GFE_81, KnownVersion.GFE_80);
-    compare(KnownVersion.GFE_82, KnownVersion.GFE_81);
-    compare(KnownVersion.GEODE_1_1_0, KnownVersion.GFE_82);
+  public void testKnownVersionClass() {
+    compare(KnownVersion.GFE_90, KnownVersion.GFE_81);
+    compare(KnownVersion.GEODE_1_1_0, KnownVersion.GFE_90);
+    compare(KnownVersion.GEODE_1_1_1, KnownVersion.GEODE_1_1_0);
     compare(KnownVersion.GEODE_1_2_0, KnownVersion.GEODE_1_1_1);
     compare(KnownVersion.GEODE_1_3_0, KnownVersion.GEODE_1_2_0);
     compare(KnownVersion.GEODE_1_4_0, KnownVersion.GEODE_1_3_0);
@@ -44,19 +38,23 @@
     compare(KnownVersion.GEODE_1_10_0, KnownVersion.GEODE_1_9_0);
     compare(KnownVersion.GEODE_1_11_0, KnownVersion.GEODE_1_10_0);
     compare(KnownVersion.GEODE_1_12_0, KnownVersion.GEODE_1_11_0);
-    compare(KnownVersion.GEODE_1_13_0, KnownVersion.GEODE_1_12_0);
-    compare(KnownVersion.GEODE_1_14_0, KnownVersion.GEODE_1_13_0);
+    compare(KnownVersion.GEODE_1_12_1, KnownVersion.GEODE_1_12_0);
+    compare(KnownVersion.GEODE_1_13_0, KnownVersion.GEODE_1_12_1);
+    compare(KnownVersion.GEODE_1_13_1, KnownVersion.GEODE_1_13_0);
+    compare(KnownVersion.GEODE_1_14_0, KnownVersion.GEODE_1_13_1);
   }
 
   private void compare(KnownVersion later, KnownVersion earlier) {
-    assertTrue(later.compareTo(earlier) > 0);
-    assertTrue(later.equals(later));
-    assertTrue(later.compareTo(later) == 0);
-    assertTrue(earlier.compareTo(later) < 0);
+    assertThat(later).isEqualTo(later);
+    assertThat(later).isNotEqualTo(earlier);
 
-    assertTrue(later.compareTo(Versioning.getVersion(earlier.ordinal())) > 0);
-    assertTrue(later.compareTo(Versioning.getVersion(later.ordinal())) == 0);
-    assertTrue(earlier.compareTo(Versioning.getVersion(later.ordinal())) < 0);
+    assertThat(later).isGreaterThan(earlier);
+    assertThat(later).isEqualByComparingTo(later);
+    assertThat(earlier).isLessThan(later);
+
+    assertThat((Version) later).isGreaterThan(Versioning.getVersion(earlier.ordinal()));
+    assertThat((Version) later).isEqualByComparingTo(Versioning.getVersion(later.ordinal()));
+    assertThat((Version) earlier).isLessThan(Versioning.getVersion(later.ordinal()));
 
     compareNewerVsOlder(later, earlier);
   }
@@ -84,14 +82,6 @@
   }
 
   @Test
-  public void testIsPre65() {
-    assertTrue(KnownVersion.GFE_61.isOlderThan(KnownVersion.GFE_65));
-    assertFalse(KnownVersion.GFE_65.isOlderThan(KnownVersion.GFE_65));
-    assertFalse(KnownVersion.GFE_70.isOlderThan(KnownVersion.GFE_65));
-    assertFalse(KnownVersion.GEODE_1_1_0.isOlderThan(KnownVersion.GFE_65));
-  }
-
-  @Test
   public void testFromOrdinalForCurrentVersionSucceeds() {
     final KnownVersion version = Versioning.getKnownVersionOrDefault(
         Versioning.getVersion(KnownVersion.CURRENT_ORDINAL), null);
diff --git a/geode-tcp-server/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerProductVersionDUnitTest.java b/geode-tcp-server/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerProductVersionDUnitTest.java
index 9d404d1..01ed555 100644
--- a/geode-tcp-server/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerProductVersionDUnitTest.java
+++ b/geode-tcp-server/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerProductVersionDUnitTest.java
@@ -133,6 +133,13 @@
       this.locatorProductVersion = locatorProductVersion;
     }
 
+    @Override
+    public String toString() {
+      return "VersionConfiguration{" +
+          "clientProductVersion=" + clientProductVersion +
+          ", locatorProductVersion=" + locatorProductVersion +
+          '}';
+    }
   }
 
   private final VersionConfiguration versions;
diff --git a/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java b/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
index 1e15121..420aa9e 100644
--- a/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
+++ b/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
@@ -23,8 +23,7 @@
 import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.Socket;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.LongSupplier;
 import java.util.function.Supplier;
 
@@ -54,8 +53,7 @@
 
   private static final int DEFAULT_REQUEST_TIMEOUT = 60 * 2 * 1000;
 
-  private final Map<HostAndPort, Short> serverVersions =
-      new HashMap<>();
+  private final ConcurrentHashMap<HostAndPort, Short> serverVersions = new ConcurrentHashMap<>();
 
   private final TcpSocketCreator socketCreator;
   private final ObjectSerializer objectSerializer;
@@ -165,13 +163,6 @@
       debugVersionMessage = null;
     }
 
-    // establish the old GossipVersion for the server
-    int gossipVersion = TcpServer.getCurrentGossipVersion();
-
-    if (serverVersion.isNotNewerThan(KnownVersion.GFE_71)) {
-      gossipVersion = TcpServer.getOldGossipVersion();
-    }
-
     long newTimeout = giveupTime - System.currentTimeMillis();
     if (newTimeout <= 0) {
       return null;
@@ -191,10 +182,8 @@
         out = new VersionedDataOutputStream(out, serverVersion);
       }
 
-      out.writeInt(gossipVersion);
-      if (gossipVersion > TcpServer.getOldGossipVersion()) {
-        out.writeShort(serverVersionShort);
-      }
+      out.writeInt(TcpServer.GOSSIPVERSION);
+      out.writeShort(serverVersionShort);
 
       objectSerializer.writeObject(request, out);
       out.flush();
@@ -243,26 +232,18 @@
     }
   }
 
-  private Short getServerVersion(HostAndPort addr, int timeout)
+  private Short getServerVersion(final HostAndPort address, final int timeout)
       throws IOException, ClassNotFoundException {
 
-    int gossipVersion;
-    Short serverVersion;
-    Socket sock;
-
     // Get GemFire version of TcpServer first, before sending any other request.
-    synchronized (serverVersions) {
-      serverVersion = serverVersions.get(addr);
-    }
-
+    Short serverVersion = serverVersions.get(address);
     if (serverVersion != null) {
       return serverVersion;
     }
 
-    gossipVersion = TcpServer.getOldGossipVersion();
-
+    final Socket sock;
     try {
-      sock = socketCreator.forCluster().connect(addr, timeout, null, socketFactory);
+      sock = socketCreator.forCluster().connect(address, timeout, null, socketFactory);
       sock.setSoTimeout(timeout);
     } catch (SSLHandshakeException e) {
       if ((e.getCause() instanceof EOFException)
@@ -274,33 +255,33 @@
     } catch (SSLException e) {
       throw new IllegalStateException("Unable to form SSL connection", e);
     }
+
+    // TODO do we really need versioned in/out? Should it be some OLDEST_VERSION constant?
     try (OutputStream outputStream = new BufferedOutputStream(sock.getOutputStream());
-        DataOutputStream out =
-            new VersionedDataOutputStream(new DataOutputStream(outputStream), KnownVersion.GFE_57);
-        InputStream inputStream = sock.getInputStream();
-        DataInputStream in = new DataInputStream(inputStream);
-        VersionedDataInputStream versionedIn =
-            new VersionedDataInputStream(in, KnownVersion.GFE_57)) {
+        final DataOutputStream out =
+            new VersionedDataOutputStream(new DataOutputStream(outputStream), KnownVersion.OLDEST);
+        final InputStream inputStream = sock.getInputStream();
+        final DataInputStream in = new DataInputStream(inputStream);
+        final VersionedDataInputStream versionedIn =
+            new VersionedDataInputStream(in, KnownVersion.OLDEST)) {
 
-      out.writeInt(gossipVersion);
+      out.writeInt(TcpServer.GOSSIPVERSION);
+      out.writeShort(KnownVersion.OLDEST.ordinal());
 
-      VersionRequest verRequest = new VersionRequest();
+      final VersionRequest verRequest = new VersionRequest();
       objectSerializer.writeObject(verRequest, out);
       out.flush();
 
       try {
-        Object readObject = objectDeserializer.readObject(versionedIn);
+        final Object readObject = objectDeserializer.readObject(versionedIn);
         if (!(readObject instanceof VersionResponse)) {
           throw new IllegalThreadStateException(
-              "Server version response invalid: "
-                  + "This could be the result of trying to connect a non-SSL-enabled client to an SSL-enabled locator.");
+              "Server version response invalid: This could be the result of trying to connect a non-SSL-enabled client to an SSL-enabled locator.");
         }
 
-        VersionResponse response = (VersionResponse) readObject;
+        final VersionResponse response = (VersionResponse) readObject;
         serverVersion = response.getVersionOrdinal();
-        synchronized (serverVersions) {
-          serverVersions.put(addr, serverVersion);
-        }
+        serverVersions.put(address, serverVersion);
 
         return serverVersion;
 
@@ -321,9 +302,9 @@
         }
       }
     }
-    synchronized (serverVersions) {
-      serverVersions.put(addr, KnownVersion.GFE_57.ordinal());
-    }
-    return KnownVersion.GFE_57.ordinal();
+
+    // TODO seems more appropriate for this to not be cached or to be an exception.
+    serverVersions.putIfAbsent(address, KnownVersion.OLDEST.ordinal());
+    return KnownVersion.OLDEST.ordinal();
   }
 }
diff --git a/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java b/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
index a553133..f83a9a2 100755
--- a/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
+++ b/geode-tcp-server/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
@@ -26,8 +26,6 @@
 import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.function.LongSupplier;
@@ -37,7 +35,6 @@
 
 import org.apache.logging.log4j.Logger;
 
-import org.apache.geode.annotations.internal.MutableForTesting;
 import org.apache.geode.internal.serialization.KnownVersion;
 import org.apache.geode.internal.serialization.ObjectDeserializer;
 import org.apache.geode.internal.serialization.ObjectSerializer;
@@ -68,13 +65,16 @@
 
   /**
    * Version 1001 was the on-wire protocol version prior to the introduction of the use of
-   * Geode's Version class to designate the on-wire protocol.
+   * Geode's Version class to designate the on-wire protocol. Even newer clients use this
+   * gossip version for initial server version discovery. We can't remove this until after
+   * Geode support 1.14 is removed.
+   *
+   * @deprecated Use {@link #GOSSIPVERSION}. Remove after {@link KnownVersion#GEODE_1_14_0} is
+   *             removed.
    */
+  @Deprecated
   public static final int OLDGOSSIPVERSION = 1001;
 
-  @MutableForTesting("The map used here is mutable, because some tests modify it")
-  private static final Map<Integer, Short> GOSSIP_TO_GEMFIRE_VERSION_MAP =
-      createGossipToVersionMap();
   public static final int GOSSIP_BYTE = 0;
   private static final String P2P_BACKLOG_PROPERTY_NAME = "p2p.backlog";
 
@@ -105,18 +105,6 @@
 
   private final LongSupplier nanoTimeSupplier;
 
-
-  /*
-   * Old on-wire protocol map. This should be removed in a release that breaks all backward
-   * compatibility since it has been replaced with Geode's Version class.
-   */
-  private static Map<Integer, Short> createGossipToVersionMap() {
-    HashMap<Integer, Short> map = new HashMap<>();
-    map.put(GOSSIPVERSION, KnownVersion.GFE_71.ordinal());
-    map.put(OLDGOSSIPVERSION, KnownVersion.GFE_57.ordinal());
-    return map;
-  }
-
   /**
    * The constructor for TcpServer
    *
@@ -149,7 +137,6 @@
     this.handler = handler;
     this.protocolChecker = protocolChecker;
     this.executorServiceSupplier = executorServiceSupplier;
-    this.executor = executorServiceSupplier.get();
     this.threadName = threadName;
     this.nanoTimeSupplier = nanoTimeSupplier;
     if (socketCreator == null) {
@@ -159,6 +146,8 @@
     }
     this.objectSerializer = objectSerializer;
     this.objectDeserializer = objectDeserializer;
+
+    executor = executorServiceSupplier.get();
     readTimeout = Integer.getInteger(readTimeoutPropertyName, 60 * 1000);
     final int p2pBacklog = Integer.getInteger(P2P_BACKLOG_PROPERTY_NAME, 1000);
     backlogLimit = Integer.getInteger(backlogLimitPropertyName, p2pBacklog);
@@ -168,14 +157,14 @@
    * This method is used during a Geode auto-reconnect to restart the server-socket thread
    */
   public void restarting() throws IOException {
-    this.shuttingDown = false;
+    shuttingDown = false;
     startServerThread();
-    if (this.executor == null || this.executor.isShutdown()) {
-      this.executor = executorServiceSupplier.get();
+    if (executor == null || executor.isShutdown()) {
+      executor = executorServiceSupplier.get();
     }
     logger.info("TcpServer@" + System.identityHashCode(this)
-        + " restarting: completed.  Server thread=" + this.serverThread + '@'
-        + System.identityHashCode(this.serverThread) + ";alive=" + this.serverThread.isAlive());
+        + " restarting: completed.  Server thread=" + serverThread + '@'
+        + System.identityHashCode(serverThread) + ";alive=" + serverThread.isAlive());
   }
 
   /**
@@ -185,7 +174,7 @@
    * @see TcpClient#stop(HostAndPort)
    */
   public void start() throws IOException {
-    this.shuttingDown = false;
+    shuttingDown = false;
     startServerThread();
     handler.init(this);
   }
@@ -208,8 +197,8 @@
       }
       // GEODE-4176 - set the port from a wild-card bind so that handlers know the correct value
 
-      if (this.port <= 0) {
-        this.port = srv_sock.getLocalPort();
+      if (port <= 0) {
+        port = srv_sock.getLocalPort();
       }
       if (logger.isInfoEnabled()) {
         logger.info("Locator was created at " + new Date());
@@ -251,7 +240,7 @@
    * to shut down
    */
   public boolean isShuttingDown() {
-    return this.shuttingDown;
+    return shuttingDown;
   }
 
   /**
@@ -274,7 +263,6 @@
   }
 
   protected void run() {
-    Socket sock = null;
 
     while (!shuttingDown) {
       if (srv_sock.isClosed()) {
@@ -282,6 +270,7 @@
         break;
       }
       try {
+        final Socket sock;
         try {
           sock = srv_sock.accept();
         } catch (SSLException ex) {
@@ -298,7 +287,6 @@
         if (!shuttingDown) {
           logger.error("exception=", ex);
         }
-        continue;
       }
     }
 
@@ -320,7 +308,7 @@
       }
       handler.shutDown();
       synchronized (this) {
-        this.notifyAll();
+        notifyAll();
       }
     }
   }
@@ -363,17 +351,11 @@
       } catch (EOFException | SocketException ignore) {
         // client went away - ignore
       } catch (SocketTimeoutException ex) {
-        String sender = null;
-        if (socket != null) {
-          sender = socket.getInetAddress().getHostAddress();
-        }
+        final String sender = socket.getInetAddress().getHostAddress();
         // Do not want the full stack trace to fill up the logs
         logger.info("Exception in processing request from " + sender + ": " + ex.getMessage());
       } catch (ClassNotFoundException ex) {
-        String sender = null;
-        if (socket != null) {
-          sender = socket.getInetAddress().getHostAddress();
-        }
+        final String sender = socket.getInetAddress().getHostAddress();
         logger.info("Unable to process request from " + sender + " exception=" + ex.getMessage());
       } catch (Exception ex) {
         String sender = null;
@@ -391,10 +373,7 @@
         }
 
       } catch (Throwable ex) {
-        String sender = null;
-        if (socket != null) {
-          sender = socket.getInetAddress().getHostAddress();
-        }
+        final String sender = socket.getInetAddress().getHostAddress();
         try {
           logger.fatal("Exception in processing request from " + sender, ex);
         } catch (Throwable t) {
@@ -419,70 +398,73 @@
 
   private void processOneConnection(Socket socket, final long startTime, DataInputStream input)
       throws IOException, UnsupportedSerializationVersionException, ClassNotFoundException {
-    // At this point we've read the leading byte of the gossip version and found it to be 0,
-    // continue reading the next three bytes
+    final int gossipVersion = readGossipVersion(input);
+    if (!(gossipVersion == GOSSIPVERSION || gossipVersion == OLDGOSSIPVERSION)) {
+      rejectUnknownProtocolConnection(socket, gossipVersion);
+      return;
+    }
+
+    final short versionOrdinal;
+    if (gossipVersion == OLDGOSSIPVERSION) {
+      // Even newer clients use OLDGOSSIPVERSION for initial connection
+      versionOrdinal = KnownVersion.OLDEST.ordinal();
+    } else {
+      // Recent versions of TcpClient will send the version ordinal
+      versionOrdinal = input.readShort();
+    }
+
+    if (logger.isDebugEnabled() && versionOrdinal != KnownVersion.CURRENT_ORDINAL) {
+      logger.debug("Locator reading request from " + socket.getInetAddress() + " with version "
+          + Versioning.getVersion(versionOrdinal));
+    }
+    final KnownVersion version = Versioning.getKnownVersionOrDefault(
+        Versioning.getVersion(versionOrdinal), null);
+    if (version == null) {
+      throw new UnsupportedSerializationVersionException(
+          KnownVersion.unsupportedVersionMessage(versionOrdinal));
+    }
+    input = new VersionedDataInputStream(input, version);
+    final Object request = objectDeserializer.readObject(input);
+    if (logger.isDebugEnabled()) {
+      logger.debug("Locator received request " + request + " from " + socket.getInetAddress());
+    }
+    final Object response;
+    if (request instanceof ShutdownRequest) {
+      shuttingDown = true;
+      // Don't call shutdown from within the worker thread, see java bug #6576792.
+      // Closing the socket will cause our acceptor thread to shutdown the executor
+      srv_sock.close();
+      response = new ShutdownResponse();
+    } else if (request instanceof VersionRequest) {
+      response = handleVersionRequest();
+    } else {
+      response = handler.processRequest(request);
+    }
+
+    handler.endRequest(request, startTime);
+
+    final long startTime2 = nanoTimeSupplier.getAsLong();
+    if (response != null) {
+      DataOutputStream output = new DataOutputStream(socket.getOutputStream());
+      if (version != KnownVersion.CURRENT) {
+        output = new VersionedDataOutputStream(output, version);
+      }
+      objectSerializer.writeObject(response, output);
+      output.flush();
+    }
+
+    handler.endResponse(request, startTime2);
+  }
+
+  /**
+   * Reads the next 3 bytes of the gossip version where first byte was 0.
+   */
+  private int readGossipVersion(final DataInputStream input) throws IOException {
     int gossipVersion = 0;
     for (int i = 0; i < 3; i++) {
       gossipVersion = (gossipVersion << 8) + (0xff & input.readUnsignedByte());
     }
-
-    Object request;
-    short versionOrdinal;
-    if (gossipVersion <= getCurrentGossipVersion()
-        && GOSSIP_TO_GEMFIRE_VERSION_MAP.containsKey(gossipVersion)) {
-      // Create a versioned stream to remember sender's GemFire version
-      versionOrdinal = (short) GOSSIP_TO_GEMFIRE_VERSION_MAP.get(gossipVersion);
-
-      if (KnownVersion.GFE_71.compareTo(Versioning.getVersion(versionOrdinal)) <= 0) {
-        // Recent versions of TcpClient will send the version ordinal
-        versionOrdinal = input.readShort();
-      }
-
-      if (logger.isDebugEnabled() && versionOrdinal != KnownVersion.CURRENT_ORDINAL) {
-        logger.debug("Locator reading request from " + socket.getInetAddress() + " with version "
-            + Versioning.getVersion(versionOrdinal));
-      }
-      final KnownVersion version = Versioning.getKnownVersionOrDefault(
-          Versioning.getVersion(versionOrdinal), null);
-      if (version == null) {
-        throw new UnsupportedSerializationVersionException(
-            KnownVersion.unsupportedVersionMessage(versionOrdinal));
-      }
-      input = new VersionedDataInputStream(input, version);
-      request = objectDeserializer.readObject(input);
-      if (logger.isDebugEnabled()) {
-        logger.debug("Locator received request " + request + " from " + socket.getInetAddress());
-      }
-      final Object response;
-      if (request instanceof ShutdownRequest) {
-        shuttingDown = true;
-        // Don't call shutdown from within the worker thread, see java bug #6576792.
-        // Closing the socket will cause our acceptor thread to shutdown the executor
-        srv_sock.close();
-        response = new ShutdownResponse();
-      } else if (request instanceof VersionRequest) {
-        response = handleVersionRequest(request);
-      } else {
-        response = handler.processRequest(request);
-      }
-
-      handler.endRequest(request, startTime);
-
-      final long startTime2 = nanoTimeSupplier.getAsLong();
-      if (response != null) {
-        DataOutputStream output = new DataOutputStream(socket.getOutputStream());
-        if (version != KnownVersion.CURRENT) {
-          output = new VersionedDataOutputStream(output, version);
-        }
-        objectSerializer.writeObject(response, output);
-        output.flush();
-      }
-
-      handler.endResponse(request, startTime2);
-    } else {
-      // Close the socket. We can not accept requests from a newer version
-      rejectUnknownProtocolConnection(socket, gossipVersion);
-    }
+    return gossipVersion;
   }
 
   private void rejectUnknownProtocolConnection(Socket socket, int gossipVersion) {
@@ -497,20 +479,10 @@
     }
   }
 
-
-
-  protected Object handleVersionRequest(Object request) {
+  private Object handleVersionRequest() {
     VersionResponse response = new VersionResponse();
     response.setVersionOrdinal(KnownVersion.CURRENT_ORDINAL);
     return response;
   }
 
-  public static int getCurrentGossipVersion() {
-    return GOSSIPVERSION;
-  }
-
-  public static int getOldGossipVersion() {
-    return OLDGOSSIPVERSION;
-  }
-
 }