You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2019/08/06 18:45:00 UTC

[geode] branch develop updated: GEODE-6883 Move the membership code into a separate gradle sub-project

This is an automated email from the ASF dual-hosted git repository.

bschuchardt pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 159dd7b  GEODE-6883 Move the membership code into a separate gradle sub-project
159dd7b is described below

commit 159dd7b694a5c3cde160dd4d5b14fe3b77aa7fb4
Author: Bruce Schuchardt <bs...@pivotal.io>
AuthorDate: Tue Aug 6 11:41:38 2019 -0700

    GEODE-6883 Move the membership code into a separate gradle sub-project
    
    This commit is focused on removing references to
    InternalDistributedMember and DistributionMessage from "gms"
    packages.
    
        GMS classes only refer to GMSMember
        GMS classes use GMSMembershipView.  NetView is now an interface
        GMS classes do not refer to DistributionMessage.  JGroupsMessenger
          expects GMSMessage instances.  Geode messages to be sent over UDP
          are wrapped in a GMSMessageAdapter.
        "gms" messages extend AbstractGMSMessage which implements GMSMessage
        GMSMembershipManager has an inner class that implements the GMS Manager
          interface and is now in the "adapter" package
        GMSMembershipManager translates GMSMembershipView into a MembershipView
          for the rest of Geode to use (this is the old NetView class)
        GMS instantiation allows us to inject the Manager into the new Services
          instance.
        Other adapter classes have been added to translate between Geode
          and GMS.
    
        GMSUtil has new methods for marshalling/unmarshalling
        InternalDistributedMember instances for backward-compatibility.
        GMSMember now has the same on-wire form as
        InternalDistributedMember.  This allows the GMS classes to
        deserialize a message from a pre-1.10 member whose code writes
        InternalDistributedMembers when serializing something like a
        JoinRequest.
---
 .../geode/ClusterCommunicationsDUnitTest.java      |  35 +-
 .../apache/geode/cache30/ReconnectDUnitTest.java   |   8 +-
 .../distributed/DistributedMemberDUnitTest.java    |  20 +-
 .../distributed/DistributedSystemDUnitTest.java    |   5 +-
 .../apache/geode/distributed/LocatorDUnitTest.java |   6 +-
 .../ClusterDistributionManagerDUnitTest.java       |  10 +-
 .../TcpServerBackwardCompatDUnitTest.java          |   4 +-
 .../LoggingWithReconnectDistributedTest.java       |   5 +-
 .../MeterSubregistryReconnectDistributedTest.java  |   5 +-
 .../cache/client/ClientCacheFactoryJUnitTest.java  |   7 +-
 .../internal/membership/MembershipJUnitTest.java   |  81 +--
 .../GMSMembershipManagerJUnitTest.java             | 121 ++--
 .../gms/fd/GMSHealthMonitorJUnitTest.java          | 138 ++--
 .../gms/locator/GMSLocatorIntegrationTest.java     |  10 +-
 .../locator/GMSLocatorRecoveryIntegrationTest.java |  20 +-
 .../gms/membership/GMSJoinLeaveJUnitTest.java      | 260 +++----
 .../gms/messenger/GMSEncryptJUnitTest.java         |  16 +-
 .../gms/messenger/GMSQuorumCheckerJUnitTest.java   |  38 +-
 .../gms/messenger/JGroupsMessengerJUnitTest.java   | 392 +++++------
 .../apache/geode/codeAnalysis/excludedClasses.txt  |   2 +-
 .../codeAnalysis/sanctionedDataSerializables.txt   |  62 +-
 .../internal/ClusterDistributionManager.java       |  21 +-
 .../distributed/internal/DistributionManager.java  |   7 -
 .../distributed/internal/DistributionMessage.java  |   1 +
 .../internal/InternalDistributedSystem.java        |  12 +-
 .../distributed/internal/InternalLocator.java      |   7 +-
 .../internal/LonerDistributionManager.java         |   5 -
 .../distributed/internal/ReplyProcessor21.java     |   4 +-
 .../membership/DistributedMembershipListener.java  |   5 +-
 .../membership/InternalDistributedMember.java      | 194 ++---
 .../internal/membership/MemberFactory.java         |  12 +-
 .../internal/membership/MemberServices.java        |   5 +-
 .../internal/membership/MembershipManager.java     |  33 +-
 .../internal/membership/MembershipView.java        | 226 ++++++
 .../internal/membership/{gms => }/NetLocator.java  |   6 +-
 .../distributed/internal/membership/NetMember.java |  18 +-
 .../internal/membership/QuorumChecker.java         |   4 -
 .../membership/adapter/GMSLocatorAdapter.java      |  98 +++
 .../membership/adapter/GMSMemberAdapter.java       | 225 ++++++
 .../{gms => adapter}/GMSMemberFactory.java         |  37 +-
 .../{gms/mgr => adapter}/GMSMembershipManager.java | 783 ++++++++++++---------
 .../membership/adapter/GMSMessageAdapter.java      | 128 ++++
 .../adapter/GMSQuorumCheckerAdapter.java           |  53 ++
 .../{gms/mgr => adapter}/LocalViewMessage.java     |   8 +-
 .../membership/adapter/auth/GMSAuthenticator.java  |  20 +-
 .../internal/membership/gms/GMSMember.java         | 354 ++++++----
 .../{NetView.java => gms/GMSMembershipView.java}   | 267 +++----
 .../internal/membership/gms/GMSUtil.java           | 126 ++++
 .../internal/membership/gms/ServiceConfig.java     |   1 +
 .../internal/membership/gms/Services.java          |  35 +-
 .../internal/membership/gms/SuspectMember.java     |  10 +-
 .../membership/gms/fd/GMSHealthMonitor.java        | 148 ++--
 .../membership/gms/interfaces/Authenticator.java   |   6 +-
 .../membership/gms/interfaces/GMSMessage.java      |  57 ++
 .../membership/gms/interfaces/HealthMonitor.java   |  13 +-
 .../membership/gms/interfaces/JoinLeave.java       |  22 +-
 .../membership/gms/interfaces/Locator.java         |   4 +-
 .../membership/gms/interfaces/Manager.java         |  75 +-
 .../membership/gms/interfaces/Messenger.java       |  28 +-
 .../membership/gms/interfaces/Service.java         |  10 +-
 .../gms/locator/FindCoordinatorRequest.java        |  42 +-
 .../gms/locator/FindCoordinatorResponse.java       |  58 +-
 .../membership/gms/locator/GMSLocator.java         |  88 +--
 .../membership/gms/locator/GetViewResponse.java    |   8 +-
 .../membership/gms/membership/GMSJoinLeave.java    | 451 ++++++------
 .../gms/messages/AbstractGMSMessage.java           |  95 +++
 .../gms/messages/FinalCheckPassedMessage.java      |  25 +-
 .../membership/gms/messages/HasMemberID.java       |   4 +-
 .../membership/gms/messages/HeartbeatMessage.java  |   9 +-
 .../gms/messages/HeartbeatRequestMessage.java      |  23 +-
 .../gms/messages/InstallViewMessage.java           |  30 +-
 .../gms/messages/JoinRequestMessage.java           |  38 +-
 .../gms/messages/JoinResponseMessage.java          |  30 +-
 .../gms/messages/LeaveRequestMessage.java          |  28 +-
 .../gms/messages/NetworkPartitionMessage.java      |  27 +-
 .../gms/messages/RemoveMemberMessage.java          |  26 +-
 .../gms/messages/SuspectMembersMessage.java        |  18 +-
 .../membership/gms/messages/SuspectRequest.java    |   8 +-
 .../membership/gms/messages/ViewAckMessage.java    |  35 +-
 .../membership/gms/messenger/GMSEncrypt.java       |  35 +-
 .../membership/gms/messenger/GMSQuorumChecker.java |  95 ++-
 .../membership/gms/messenger/JGAddress.java        |  12 -
 .../membership/gms/messenger/JGroupsMessenger.java | 243 +++----
 .../gms/messenger/MembershipInformation.java       |   8 +-
 .../org/apache/geode/internal/DSFIDFactory.java    |   4 +-
 .../geode/internal/InternalDataSerializer.java     |   5 +
 .../tier/sockets/ClientProxyMembershipID.java      |   6 +-
 .../apache/geode/internal/tcp/ConnectionTable.java |   2 +-
 .../callbacks/ConfigurationChangeListener.java     |   1 -
 .../messages/ConfigurationResponse.java            |   2 +-
 .../MembershipDependenciesJUnitTest.java           |  93 +--
 .../internal/membership/NetViewJUnitTest.java      | 138 ++--
 .../auth/AbstractGMSAuthenticatorTestCase.java     |   5 +-
 .../membership/gms/GMSMemberJUnitTest.java         |  20 +-
 .../rollingupgrade/RollingUpgradeDUnitTest.java    |   3 +
 .../membership/gms/MembershipManagerHelper.java    |  15 +-
 .../gms/membership/GMSJoinLeaveTestHelper.java     |  11 +-
 .../geode/test/dunit/rules/DistributedRule.java    |   1 -
 98 files changed, 3493 insertions(+), 2562 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
index eca86ed..046d656 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode;
 
 import static org.apache.geode.distributed.ConfigurationProperties.CONSERVE_SOCKETS;
+import static org.apache.geode.distributed.ConfigurationProperties.DISABLE_TCP;
 import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
@@ -75,8 +76,10 @@ import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.distributed.internal.ReplyMessage;
 import org.apache.geode.distributed.internal.SerialAckedMessage;
 import org.apache.geode.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.DSFIDFactory;
 import org.apache.geode.internal.cache.DirectReplyMessage;
+import org.apache.geode.test.dunit.DistributedTestUtils;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
@@ -104,6 +107,7 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
 
   private final String regionName = "clusterTestRegion";
 
+  private final boolean disableTcp;
   private boolean conserveSockets;
   private boolean useSSL;
 
@@ -126,6 +130,7 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
   public ClusterCommunicationsDUnitTest(RunConfiguration runConfiguration) {
     useSSL = runConfiguration.useSSL;
     conserveSockets = runConfiguration.conserveSockets;
+    disableTcp = runConfiguration.disableTcp;
   }
 
   @Before
@@ -198,7 +203,7 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
     // create a cluster with the previous version of Geode
     VM locatorVM = Host.getHost(0).getVM(testVersion, 0);
     VM server1VM = Host.getHost(0).getVM(testVersion, 1);
-    int locatorPort = createLocator(locatorVM);
+    int locatorPort = createLocator(locatorVM, true);
     createCacheAndRegion(server1VM, locatorPort);
     performCreate(getVM(1));
 
@@ -270,12 +275,24 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
   }
 
   private int createLocator(VM memberVM) {
+    return createLocator(memberVM, false);
+  }
+
+  private int createLocator(VM memberVM, boolean usingOldVersion) {
     return memberVM.invoke("create locator", () -> {
       // if you need to debug SSL communications use this property:
       // System.setProperty("javax.net.debug", "all");
       System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+      Properties dsProperties = getDistributedSystemProperties();
       try {
-        return Locator.startLocatorAndDS(0, new File(""), getDistributedSystemProperties())
+        int port = 0;
+        // for stress-tests make sure that an older-version locator doesn't try
+        // to read state persisted by another run's newer-version locator
+        if (usingOldVersion) {
+          port = AvailablePortHelper.getRandomAvailableTCPPort();
+          DistributedTestUtils.deleteLocatorStateFile(port);
+        }
+        return Locator.startLocatorAndDS(port, new File(""), getDistributedSystemProperties())
             .getPort();
       } finally {
         System.clearProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
@@ -297,6 +314,7 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
     properties.setProperty(USE_CLUSTER_CONFIGURATION, "false");
     properties.setProperty(NAME, "vm" + VM.getCurrentVMNum());
     properties.setProperty(CONSERVE_SOCKETS, "" + conserveSockets);
+    properties.setProperty(DISABLE_TCP, "" + disableTcp);
     properties.setProperty(SOCKET_LEASE_TIME, "10000");
     properties.setProperty(SOCKET_BUFFER_SIZE, "" + SMALL_BUFFER_SIZE);
 
@@ -317,17 +335,20 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
   }
 
   enum RunConfiguration {
-    SHARED_CONNECTIONS(true, false),
-    SHARED_CONNECTIONS_WITH_SSL(true, true),
-    UNSHARED_CONNECTIONS(false, false),
-    UNSHARED_CONNECTIONS_WITH_SSL(false, true);
+    SHARED_CONNECTIONS(true, false, false),
+    SHARED_CONNECTIONS_WITH_SSL(true, true, false),
+    UNSHARED_CONNECTIONS(false, false, false),
+    UNSHARED_CONNECTIONS_WITH_SSL(false, true, false),
+    UDP_CONNECTIONS(true, false, true);
 
     boolean useSSL;
     boolean conserveSockets;
+    boolean disableTcp;
 
-    RunConfiguration(boolean conserveSockets, boolean useSSL) {
+    RunConfiguration(boolean conserveSockets, boolean useSSL, boolean disableTcp) {
       this.useSSL = useSSL;
       this.conserveSockets = conserveSockets;
+      this.disableTcp = disableTcp;
     }
   }
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
index b93e234..87e33f1 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
@@ -91,8 +91,9 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem.Reconnect
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.ServerLocator;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.examples.SimpleSecurityManager;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.AvailablePortHelper;
@@ -419,7 +420,10 @@ public class ReconnectDUnitTest extends JUnit4CacheTestCase {
               system = cache.getInternalDistributedSystem();
               assertTrue(
                   ((GMSMembershipManager) getMembershipManager(system))
-                      .getServices().getMessenger().isOldMembershipIdentifier(dm));
+                      .getServices().getMessenger()
+                      .isOldMembershipIdentifier(
+                          ((GMSMemberAdapter) ((InternalDistributedMember) dm).getNetMember())
+                              .getGmsMember()));
               return ds.getReconnectedSystem().getDistributedMember();
             } catch (InterruptedException e) {
               System.err.println("interrupted while waiting for reconnect");
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java
index 634ebe0b..1282433 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java
@@ -27,6 +27,8 @@ import static org.apache.geode.test.dunit.Assert.assertNull;
 import static org.apache.geode.test.dunit.Assert.assertTrue;
 import static org.apache.geode.test.dunit.Assert.fail;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Arrays;
@@ -46,9 +48,12 @@ import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityAckedMessage;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
 import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import org.apache.geode.internal.HeapDataOutputStream;
+import org.apache.geode.internal.Version;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
@@ -260,10 +265,17 @@ public class DistributedMemberDUnitTest extends JUnit4DistributedTestCase {
     assertTrue(system == basicGetSystem()); // senders will use basicGetSystem()
     InternalDistributedMember internalDistributedMember = system.getDistributedMember();
 
-    GMSMember gmsMember = new GMSMember((GMSMember) internalDistributedMember.getNetMember());
-    assertTrue(gmsMember.equals(internalDistributedMember.getNetMember()));
+    GMSMember gmsMember =
+        new GMSMember(((GMSMemberAdapter) internalDistributedMember.getNetMember()).getGmsMember());
+    assertEquals(gmsMember,
+        ((GMSMemberAdapter) internalDistributedMember.getNetMember()).getGmsMember());
     gmsMember.setName(null);
-    InternalDistributedMember partialID = new InternalDistributedMember(gmsMember);
+    HeapDataOutputStream outputStream = new HeapDataOutputStream(100, Version.CURRENT);
+    new InternalDistributedMember(new GMSMemberAdapter(gmsMember)).writeEssentialData(outputStream);
+    DataInputStream dataInputStream =
+        new DataInputStream(new ByteArrayInputStream(outputStream.toByteArray()));
+    InternalDistributedMember partialID =
+        InternalDistributedMember.readEssentialData(dataInputStream);
     return partialID;
   }
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java
index 4a0401c..319a63f 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java
@@ -68,7 +68,7 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
 import org.apache.geode.distributed.internal.SizeableRunnable;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
@@ -120,7 +120,8 @@ public class DistributedSystemDUnitTest extends JUnit4DistributedTestCase {
 
     // construct a member ID that will represent a departed member
     InternalDistributedMember member =
-        new InternalDistributedMember("localhost", 12345, "", "", NORMAL_DM_TYPE, null, null);
+        new InternalDistributedMember("localhost", 12345, "", "", NORMAL_DM_TYPE, null,
+            null);
 
     // schedule a message in order to create a queue for the fake member
     ClusterDistributionManager distributionManager =
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/LocatorDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/LocatorDUnitTest.java
index 4381ef7..2db4761 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/LocatorDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/LocatorDUnitTest.java
@@ -91,7 +91,7 @@ import org.apache.geode.distributed.internal.MembershipListener;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.distributed.internal.membership.MembershipTestHook;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.MembershipView;
 import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
 import org.apache.geode.distributed.internal.tcpserver.LocatorCancelException;
 import org.apache.geode.internal.AvailablePort;
@@ -312,7 +312,7 @@ public class LocatorDUnitTest implements Serializable {
 
     assertThat(MembershipManagerHelper.getCoordinator(system))
         .describedAs("should be the coordinator").isEqualTo(system.getDistributedMember());
-    NetView view = MembershipManagerHelper.getMembershipManager(system).getView();
+    MembershipView view = MembershipManagerHelper.getMembershipManager(system).getView();
     logger.info("view after becoming coordinator is " + view);
     assertThat(system.getDistributedMember())
         .describedAs("should not be the first member in the view (" + view + ")")
@@ -1133,7 +1133,7 @@ public class LocatorDUnitTest implements Serializable {
     return (InternalDistributedMember) MembershipManagerHelper.getCoordinator(system);
   }
 
-  private NetView getView() {
+  private MembershipView getView() {
     return system.getDistributionManager().getMembershipManager().getView();
   }
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
index 1caf68a..fe22c28 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
@@ -61,9 +61,9 @@ import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.test.dunit.DistributedTestCase;
 import org.apache.geode.test.dunit.Host;
@@ -324,7 +324,7 @@ public class ClusterDistributionManagerDUnitTest extends DistributedTestCase {
     InternalDistributedSystem system = getSystem(new Properties());
     ClusterDistributionManager dm = (ClusterDistributionManager) system.getDM();
     GMSMembershipManager membershipManager = (GMSMembershipManager) dm.getMembershipManager();
-    NetView view = membershipManager.getView();
+    GMSMembershipView view = membershipManager.getServices().getJoinLeave().getView();
 
     AtomicBoolean waitForViewInstallationDone = new AtomicBoolean();
     executorService.submit(() -> {
@@ -338,8 +338,8 @@ public class ClusterDistributionManagerDUnitTest extends DistributedTestCase {
 
     pause(2000);
 
-    NetView newView = new NetView(view, view.getViewId() + 1);
-    membershipManager.installView(newView);
+    GMSMembershipView newView = new GMSMembershipView(view, view.getViewId() + 1);
+    membershipManager.getGMSManager().installView(newView);
 
     await()
         .untilAsserted(() -> assertThat(waitForViewInstallationDone.get()).isTrue());
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
index bcee1d7..3d07897 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
@@ -29,7 +29,7 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
 import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorRequest;
 import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
 import org.apache.geode.internal.AvailablePortHelper;
@@ -138,7 +138,7 @@ public class TcpServerBackwardCompatDUnitTest extends JUnit4DistributedTestCase
 
       // Start a gossip client to connect to first locator "locator0".
       FindCoordinatorRequest req = new FindCoordinatorRequest(
-          new InternalDistributedMember(SocketCreator.getLocalHost(), 1234));
+          new GMSMember("localhost", 1234));
       FindCoordinatorResponse response;
 
       response = (FindCoordinatorResponse) new TcpClient()
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/logging/LoggingWithReconnectDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/logging/LoggingWithReconnectDistributedTest.java
index bf65e09..37a11f3 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/logging/LoggingWithReconnectDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/logging/LoggingWithReconnectDistributedTest.java
@@ -47,7 +47,7 @@ import org.apache.geode.ForcedDisconnectException;
 import org.apache.geode.distributed.LocatorLauncher;
 import org.apache.geode.distributed.ServerLauncher;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.test.assertj.LogFileAssert;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedRule;
@@ -137,7 +137,8 @@ public class LoggingWithReconnectDistributedTest implements Serializable {
 
     server2VM.invoke(() -> {
       GMSMembershipManager membershipManager = (GMSMembershipManager) getMembershipManager(system);
-      membershipManager.forceDisconnect("Forcing disconnect in " + testName.getMethodName());
+      membershipManager.getGMSManager()
+          .forceDisconnect("Forcing disconnect in " + testName.getMethodName());
 
       await().until(() -> system.isReconnecting());
       system.waitUntilReconnected(TIMEOUT, MILLISECONDS);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/metrics/MeterSubregistryReconnectDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/metrics/MeterSubregistryReconnectDistributedTest.java
index f688fe3..d08a7a4 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/metrics/MeterSubregistryReconnectDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/metrics/MeterSubregistryReconnectDistributedTest.java
@@ -46,7 +46,7 @@ import org.apache.geode.ForcedDisconnectException;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.distributed.LocatorLauncher;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedRule;
@@ -127,7 +127,8 @@ public class MeterSubregistryReconnectDistributedTest implements Serializable {
 
     server2VM.invoke(() -> {
       GMSMembershipManager membershipManager = (GMSMembershipManager) getMembershipManager(system);
-      membershipManager.forceDisconnect("Forcing disconnect in " + testName.getMethodName());
+      membershipManager.getGMSManager()
+          .forceDisconnect("Forcing disconnect in " + testName.getMethodName());
 
       await().until(() -> system.isReconnecting());
       system.waitUntilReconnected(TIMEOUT, MILLISECONDS);
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 9439552..f35b79f 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
@@ -56,6 +56,7 @@ import org.apache.geode.cache.server.CacheServer;
 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.adapter.GMSMemberAdapter;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
 import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.Version;
@@ -345,7 +346,7 @@ public class ClientCacheFactoryJUnitTest {
     clientCache = new ClientCacheFactory().create();
     InternalDistributedMember memberID =
         (InternalDistributedMember) clientCache.getDistributedSystem().getDistributedMember();
-    GMSMember gmsID = (GMSMember) memberID.getNetMember();
+    GMSMember gmsID = ((GMSMemberAdapter) memberID.getNetMember()).getGmsMember();
     memberID.setVersionObjectForTest(Version.GFE_82);
     assertThat(memberID.getVersionObject()).isEqualTo(Version.GFE_82);
 
@@ -361,7 +362,7 @@ public class ClientCacheFactoryJUnitTest {
     assertThat(newMemberID.getVersionObject()).isEqualTo(Version.GFE_82);
     assertThat(newID.getClientVersion()).isEqualTo(Version.GFE_82);
 
-    GMSMember newGmsID = (GMSMember) newMemberID.getNetMember();
+    GMSMember newGmsID = ((GMSMemberAdapter) newMemberID.getNetMember()).getGmsMember();
     assertThat(newGmsID.getUuidLSBs()).isEqualTo(0);
     assertThat(newGmsID.getUuidMSBs()).isEqualTo(0);
 
@@ -377,7 +378,7 @@ public class ClientCacheFactoryJUnitTest {
     assertThat(newMemberID.getVersionObject()).isEqualTo(Version.CURRENT);
     assertThat(newID.getClientVersion()).isEqualTo(Version.CURRENT);
 
-    newGmsID = (GMSMember) newMemberID.getNetMember();
+    newGmsID = ((GMSMemberAdapter) newMemberID.getNetMember()).getGmsMember();
     assertThat(newGmsID.getUuidLSBs()).isEqualTo(gmsID.getUuidLSBs());
     assertThat(newGmsID.getUuidMSBs()).isEqualTo(gmsID.getUuidMSBs());
   }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
index 5207765..cff5061 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
@@ -31,6 +31,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.net.InetAddress;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 
@@ -50,21 +51,14 @@ import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.SerialAckedMessage;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.distributed.internal.membership.adapter.auth.GMSAuthenticator;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
 import org.apache.geode.distributed.internal.membership.gms.membership.GMSJoinLeave;
-import org.apache.geode.distributed.internal.membership.gms.messages.HeartbeatMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.HeartbeatRequestMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.InstallViewMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.JoinRequestMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.JoinResponseMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.LeaveRequestMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.RemoveMemberMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.SuspectMembersMessage;
-import org.apache.geode.distributed.internal.membership.gms.messages.ViewAckMessage;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.net.SocketCreator;
@@ -184,14 +178,14 @@ public class MembershipJUnitTest {
         }
       }
 
-      NetView view = jl1.getView();
-      InternalDistributedMember notCreator;
+      GMSMembershipView view = jl1.getView();
+      GMSMember notCreator;
       if (view.getCreator().equals(jl1.getMemberID())) {
         notCreator = view.getMembers().get(1);
       } else {
         notCreator = view.getMembers().get(0);
       }
-      List<String> result = notCreator.getGroups();
+      List<String> result = Arrays.asList(notCreator.getGroups());
 
       System.out.println("sending SerialAckedMessage from m1 to m2");
       SerialAckedMessage msg = new SerialAckedMessage();
@@ -452,65 +446,4 @@ public class MembershipJUnitTest {
       // expected
     }
   }
-
-  @Test
-  public void testMessagesThrowExceptionIfProcessed() throws Exception {
-    ClusterDistributionManager dm = null;
-    try {
-      new HeartbeatMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new HeartbeatRequestMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new InstallViewMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new JoinRequestMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new JoinResponseMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new LeaveRequestMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new RemoveMemberMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new SuspectMembersMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-    try {
-      new ViewAckMessage().process(dm);
-      fail("expected an exception to be thrown");
-    } catch (Exception e) {
-      // okay
-    }
-  }
-
-
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java
similarity index 81%
rename from geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
rename to geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java
index 9fa2efe..8f0e98a 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.distributed.internal.membership.gms.mgr;
+package org.apache.geode.distributed.internal.membership.adapter;
 
 import static org.apache.geode.distributed.ConfigurationProperties.ACK_SEVERE_ALERT_THRESHOLD;
 import static org.apache.geode.distributed.ConfigurationProperties.ACK_WAIT_THRESHOLD;
@@ -47,6 +47,7 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.Timer;
+import java.util.stream.Collectors;
 
 import org.assertj.core.api.Assertions;
 import org.jgroups.util.UUID;
@@ -69,17 +70,19 @@ import org.apache.geode.distributed.internal.ReplyProcessor21;
 import org.apache.geode.distributed.internal.direct.DirectChannel;
 import org.apache.geode.distributed.internal.membership.DistributedMembershipListener;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.MembershipView;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager.StartupEvent;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
 import org.apache.geode.distributed.internal.membership.gms.SuspectMember;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Authenticator;
+import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Messenger;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager.StartupEvent;
 import org.apache.geode.internal.admin.remote.AlertListenerMessage;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.statistics.DummyStatisticsRegistry;
@@ -130,9 +133,13 @@ public class GMSMembershipManagerJUnitTest {
 
     authenticator = mock(Authenticator.class);
     myMemberId = new InternalDistributedMember("localhost", 8887);
+    GMSMember m = ((GMSMemberAdapter) myMemberId.getNetMember()).getGmsMember();
+    UUID uuid = new UUID(12345, 12345);
+    m.setUUID(uuid);
 
     messenger = mock(Messenger.class);
-    when(messenger.getMemberID()).thenReturn(myMemberId);
+    when(messenger.getMemberID())
+        .thenReturn(((GMSMemberAdapter) myMemberId.getNetMember()).getGmsMember());
 
     stopper = mock(Stopper.class);
     when(stopper.isCancelInProgress()).thenReturn(false);
@@ -157,8 +164,8 @@ public class GMSMembershipManagerJUnitTest {
     mockMembers = new InternalDistributedMember[5];
     for (int i = 0; i < mockMembers.length; i++) {
       mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
-      GMSMember m = (GMSMember) mockMembers[i].getNetMember();
-      UUID uuid = new UUID(r.nextLong(), r.nextLong());
+      m = ((GMSMemberAdapter) mockMembers[i].getNetMember()).getGmsMember();
+      uuid = new UUID(r.nextLong(), r.nextLong());
       m.setUUID(uuid);
     }
     members = new ArrayList<>(Arrays.asList(mockMembers));
@@ -166,15 +173,15 @@ public class GMSMembershipManagerJUnitTest {
     listener = mock(DistributedMembershipListener.class);
 
     manager = new GMSMembershipManager(listener);
-    manager.init(services);
-    when(services.getManager()).thenReturn(manager);
+    manager.getGMSManager().init(services);
+    when(services.getManager()).thenReturn(manager.getGMSManager());
   }
 
   @After
   public void tearDown() throws Exception {
     if (manager != null) {
-      manager.stop();
-      manager.stopped();
+      manager.getGMSManager().stop();
+      manager.getGMSManager().stopped();
     }
   }
 
@@ -182,26 +189,44 @@ public class GMSMembershipManagerJUnitTest {
   public void testSendMessage() throws Exception {
     HighPriorityAckedMessage m = new HighPriorityAckedMessage();
     m.setRecipient(mockMembers[0]);
-    manager.start();
-    manager.started();
-    manager.installView(new NetView(myMemberId, 1, members));
-    Set<InternalDistributedMember> failures = manager.send(m);
-    verify(messenger).send(m);
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
+    GMSMember myGMSMemberId = ((GMSMemberAdapter) myMemberId.getNetMember()).getGmsMember();
+    List<GMSMember> gmsMembers =
+        members.stream().map(x -> ((GMSMemberAdapter) x.getNetMember()).getGmsMember()).collect(
+            Collectors.toList());
+    manager.getGMSManager().installView(new GMSMembershipView(myGMSMemberId, 1, gmsMembers));
+    Set<InternalDistributedMember> failures =
+        manager.send(m.getRecipients(), m, this.services.getStatistics());
+    verify(messenger).send(isA(GMSMessageAdapter.class));
     if (failures != null) {
       assertEquals(0, failures.size());
     }
   }
 
+
+
+  private GMSMembershipView createView(InternalDistributedMember creator, int viewId,
+      List<InternalDistributedMember> members) {
+    List<GMSMember> gmsMembers = new ArrayList<>(members.size());
+    for (InternalDistributedMember member : members) {
+      gmsMembers.add(((GMSMemberAdapter) member.getNetMember()).getGmsMember());
+    }
+    return new GMSMembershipView(((GMSMemberAdapter) creator.getNetMember()).getGmsMember(), viewId,
+        gmsMembers);
+  }
+
   @Test
   public void testSendAdminMessageFailsDuringShutdown() throws Exception {
     AlertListenerMessage m = AlertListenerMessage.create(mockMembers[0], 1,
         new Date(System.currentTimeMillis()), "thread", "", 1L, "", "");
-    manager.start();
-    manager.started();
-    manager.installView(new NetView(myMemberId, 1, members));
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
+    manager.getGMSManager().installView(createView(myMemberId, 1, members));
     manager.setShutdown();
-    Set<InternalDistributedMember> failures = manager.send(m);
-    verify(messenger, never()).send(m);
+    Set<InternalDistributedMember> failures =
+        manager.send(new InternalDistributedMember[] {mockMembers[0]}, m, null);
+    verify(messenger, never()).send(isA(GMSMessage.class));
     assertEquals(1, failures.size());
     assertEquals(mockMembers[0], failures.iterator().next());
   }
@@ -211,25 +236,25 @@ public class GMSMembershipManagerJUnitTest {
     InternalDistributedMember[] emptyList = new InternalDistributedMember[0];
     HighPriorityAckedMessage m = new HighPriorityAckedMessage();
     m.setRecipient(mockMembers[0]);
-    manager.start();
-    manager.started();
-    manager.installView(new NetView(myMemberId, 1, members));
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
+    manager.getGMSManager().installView(createView(myMemberId, 1, members));
     Set<InternalDistributedMember> failures = manager.send(null, m, null);
-    verify(messenger, never()).send(m);
+    verify(messenger, never()).send(isA(GMSMessage.class));
     reset(messenger);
     failures = manager.send(emptyList, m, null);
-    verify(messenger, never()).send(m);
+    verify(messenger, never()).send(isA(GMSMessage.class));
   }
 
   @Test
   public void testStartupEvents() throws Exception {
-    manager.start();
-    manager.started();
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
     manager.isJoining = true;
 
     List<InternalDistributedMember> viewmembers =
         Arrays.asList(new InternalDistributedMember[] {mockMembers[0], myMemberId});
-    manager.installView(new NetView(myMemberId, 2, viewmembers));
+    manager.getGMSManager().installView(createView(myMemberId, 2, viewmembers));
 
     // add a surprise member that will be shunned due to it's having
     // an old view ID
@@ -246,7 +271,9 @@ public class GMSMembershipManagerJUnitTest {
 
     // suspect a member
     InternalDistributedMember suspectMember = mockMembers[1];
-    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember, "testing"));
+    manager.handleOrDeferSuspect(
+        new SuspectMember(((GMSMemberAdapter) mockMembers[0].getNetMember()).getGmsMember(),
+            ((GMSMemberAdapter) suspectMember.getNetMember()).getGmsMember(), "testing"));
     // suspect messages aren't queued - they're ignored before joining the system
     assertEquals(2, manager.getStartupEvents().size());
     verify(listener, never()).memberSuspect(suspectMember, mockMembers[0], "testing");
@@ -261,7 +288,7 @@ public class GMSMembershipManagerJUnitTest {
     // this view officially adds surpriseMember2
     viewmembers = Arrays
         .asList(new InternalDistributedMember[] {mockMembers[0], myMemberId, surpriseMember2});
-    manager.handleOrDeferViewEvent(new NetView(myMemberId, 3, viewmembers));
+    manager.handleOrDeferViewEvent(new MembershipView(myMemberId, 3, viewmembers));
     assertEquals(4, manager.getStartupEvents().size());
 
     // add a surprise member that will be shunned due to it's having
@@ -276,7 +303,7 @@ public class GMSMembershipManagerJUnitTest {
     mockMembers[4].setVmViewId(4);
     viewmembers = Arrays.asList(new InternalDistributedMember[] {mockMembers[0], myMemberId,
         surpriseMember2, mockMembers[4]});
-    manager.handleOrDeferViewEvent(new NetView(myMemberId, 4, viewmembers));
+    manager.handleOrDeferViewEvent(new MembershipView(myMemberId, 4, viewmembers));
     assertEquals(6, manager.getStartupEvents().size());
 
     // exercise the toString methods for code coverage
@@ -299,14 +326,16 @@ public class GMSMembershipManagerJUnitTest {
     // event processing has started. This should notify the distribution manager
     // with a LocalViewMessage to process the view
     reset(listener);
-    manager.handleOrDeferViewEvent(new NetView(myMemberId, 5, viewmembers));
+    manager.handleOrDeferViewEvent(new MembershipView(myMemberId, 5, viewmembers));
     assertEquals(0, manager.getStartupEvents().size());
     verify(listener).messageReceived(isA(LocalViewMessage.class));
 
     // process a suspect now - it will be passed to the listener
     reset(listener);
     suspectMember = mockMembers[1];
-    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember, "testing"));
+    manager.handleOrDeferSuspect(
+        new SuspectMember(((GMSMemberAdapter) mockMembers[0].getNetMember()).getGmsMember(),
+            ((GMSMemberAdapter) suspectMember.getNetMember()).getGmsMember(), "testing"));
     verify(listener).memberSuspect(suspectMember, mockMembers[0], "testing");
   }
 
@@ -396,13 +425,15 @@ public class GMSMembershipManagerJUnitTest {
    */
   @Test
   public void testAddressesWithoutUUIDs() throws Exception {
-    manager.start();
-    manager.started();
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
     manager.isJoining = true;
 
     List<InternalDistributedMember> viewmembers =
         Arrays.asList(new InternalDistributedMember[] {mockMembers[0], mockMembers[1], myMemberId});
-    manager.installView(new NetView(myMemberId, 2, viewmembers));
+    GMSMembershipView view = createView(myMemberId, 2, viewmembers);
+    manager.getGMSManager().installView(view);
+    when(services.getJoinLeave().getView()).thenReturn(view);
 
     InternalDistributedMember[] destinations = new InternalDistributedMember[viewmembers.size()];
     for (int i = 0; i < destinations.length; i++) {
@@ -413,7 +444,7 @@ public class GMSMembershipManagerJUnitTest {
     // each destination w/o a UUID should have been replaced with the corresponding
     // ID from the membership view
     for (int i = 0; i < destinations.length; i++) {
-      assertTrue(viewmembers.get(i) == destinations[i]);
+      assertTrue(((GMSMemberAdapter) destinations[i].getNetMember()).getGmsMember().hasUUID());
     }
   }
 
@@ -438,13 +469,13 @@ public class GMSMembershipManagerJUnitTest {
     when(dm.addMembershipListenerAndGetDistributionManagerIds(any(MembershipListener.class)))
         .thenReturn(new HashSet(members));
 
-    manager.start();
-    manager.started();
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
     manager.isJoining = true;
 
     List<InternalDistributedMember> viewmembers =
         Arrays.asList(new InternalDistributedMember[] {mockMembers[0], mockMembers[1], myMemberId});
-    manager.installView(new NetView(myMemberId, 2, viewmembers));
+    manager.getGMSManager().installView(createView(myMemberId, 2, viewmembers));
 
     List<InternalDistributedMember> mbrs = new ArrayList<>(1);
     mbrs.add(mockMembers[0]);
@@ -452,7 +483,7 @@ public class GMSMembershipManagerJUnitTest {
     rp.enableSevereAlertProcessing();
     boolean result = rp.waitForReplies(WAIT_FOR_REPLIES_MILLIS);
     assertFalse(result); // the wait should have timed out
-    verify(healthMonitor, atLeastOnce()).checkIfAvailable(isA(InternalDistributedMember.class),
+    verify(healthMonitor, atLeastOnce()).checkIfAvailable(isA(GMSMember.class),
         isA(String.class), isA(Boolean.class));
   }
 
@@ -464,13 +495,13 @@ public class GMSMembershipManagerJUnitTest {
     when(dc.send(any(GMSMembershipManager.class), any(mockMembers.getClass()),
         any(DistributionMessage.class), anyInt(), anyInt())).thenReturn(100);
 
-    manager.start();
-    manager.started();
+    manager.getGMSManager().start();
+    manager.getGMSManager().started();
 
     manager.setDirectChannel(dc);
 
-    NetView view = new NetView(myMemberId, 1, members);
-    manager.installView(view);
+    GMSMembershipView view = createView(myMemberId, 1, members);
+    manager.getGMSManager().installView(view);
     when(joinLeave.getView()).thenReturn(view);
 
     manager.startEventProcessing();
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
index 2fb067f..264ac62 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -75,9 +75,8 @@ import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionStats;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
@@ -92,7 +91,6 @@ import org.apache.geode.distributed.internal.membership.gms.messages.SuspectMemb
 import org.apache.geode.distributed.internal.membership.gms.messages.SuspectRequest;
 import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
@@ -102,7 +100,7 @@ public class GMSHealthMonitorJUnitTest {
   private Services services;
   private ServiceConfig mockConfig;
   private DistributionConfig mockDistConfig;
-  private List<InternalDistributedMember> mockMembers;
+  private List<GMSMember> mockMembers;
   private Messenger messenger;
   private JoinLeave joinLeave;
   private GMSHealthMonitor gmsHealthMonitor;
@@ -157,11 +155,11 @@ public class GMSHealthMonitorJUnitTest {
     if (mockMembers == null) {
       mockMembers = new ArrayList<>();
       for (int i = 0; i < 7; i++) {
-        InternalDistributedMember mbr = new InternalDistributedMember("localhost", 8888 + i);
+        GMSMember mbr = new GMSMember("localhost", 8888 + i);
 
         if (i == 0 || i == 1) {
           mbr.setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
-          mbr.getNetMember().setPreferredForCoordinator(true);
+          mbr.setPreferredForCoordinator(true);
         }
         mockMembers.add(mbr);
       }
@@ -183,13 +181,13 @@ public class GMSHealthMonitorJUnitTest {
   @Test
   public void testHMServiceStarted() throws IOException {
 
-    InternalDistributedMember mbr =
-        new InternalDistributedMember(SocketCreator.getLocalHost(), 12345);
+    GMSMember mbr =
+        new GMSMember("localhost", 12345);
     mbr.setVmViewId(1);
     when(messenger.getMemberID()).thenReturn(mbr);
     gmsHealthMonitor.started();
 
-    NetView v = new NetView(mbr, 1, mockMembers);
+    GMSMembershipView v = new GMSMembershipView(mbr, 1, mockMembers);
 
     gmsHealthMonitor.processMessage(new HeartbeatRequestMessage(mbr, 1));
     verify(messenger, atLeastOnce()).send(any(HeartbeatMessage.class));
@@ -221,11 +219,11 @@ public class GMSHealthMonitorJUnitTest {
     System.out.println("testHMNextNeighborAfterTimeout starting");
 
     installAView();
-    InternalDistributedMember initialNeighbor = mockMembers.get(myAddressIndex + 1);
+    GMSMember initialNeighbor = mockMembers.get(myAddressIndex + 1);
 
     await("wait for new neighbor")
         .until(() -> gmsHealthMonitor.getNextNeighbor() != initialNeighbor);
-    InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
+    GMSMember neighbor = gmsHealthMonitor.getNextNeighbor();
 
     // neighbor should change. In order to not be a flaky test we don't demand
     // that it be myAddressIndex+2 but just require that the neighbor being
@@ -245,7 +243,7 @@ public class GMSHealthMonitorJUnitTest {
   public void testHMNextNeighborBeforeTimeout() throws IOException {
     long startTime = System.currentTimeMillis();
     installAView();
-    final InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
+    final GMSMember neighbor = gmsHealthMonitor.getNextNeighbor();
     System.out.println("next neighbor is " + neighbor + "\nmy address is "
         + mockMembers.get(myAddressIndex) + "\nview is " + joinLeave.getView());
     assertEquals(mockMembers.get(myAddressIndex + 1), neighbor);
@@ -275,8 +273,8 @@ public class GMSHealthMonitorJUnitTest {
     System.out.println("testSuspectMembersCalledThroughMemberCheckThread ending");
   }
 
-  private NetView installAView() {
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers);
+  private GMSMembershipView installAView() {
+    GMSMembershipView v = new GMSMembershipView(mockMembers.get(0), 2, mockMembers);
 
     // 3rd is current member
     when(messenger.getMemberID()).thenReturn(mockMembers.get(myAddressIndex));
@@ -287,8 +285,8 @@ public class GMSHealthMonitorJUnitTest {
     return v;
   }
 
-  private void setFailureDetectionPorts(NetView v) {
-    java.util.Iterator<InternalDistributedMember> itr = mockMembers.iterator();
+  private void setFailureDetectionPorts(GMSMembershipView v) {
+    java.util.Iterator<GMSMember> itr = mockMembers.iterator();
 
     int port = 7899;
     while (itr.hasNext()) {
@@ -303,7 +301,7 @@ public class GMSHealthMonitorJUnitTest {
   public void testSuspectMembersNotCalledThroughPingThreadBeforeTimeout() {
     long startTime = System.currentTimeMillis();
     installAView();
-    InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
+    GMSMember neighbor = gmsHealthMonitor.getNextNeighbor();
 
     await().until(() -> gmsHealthMonitor.isSuspectMember(neighbor));
     long endTime = System.currentTimeMillis();
@@ -347,7 +345,7 @@ public class GMSHealthMonitorJUnitTest {
   @Test
   public void testRemoveMemberCalled() throws Exception {
     System.out.println("testRemoveMemberCalled starting");
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers);
+    GMSMembershipView v = new GMSMembershipView(mockMembers.get(0), 2, mockMembers);
 
     // 3rd is current member
     when(messenger.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
@@ -355,7 +353,7 @@ public class GMSHealthMonitorJUnitTest {
 
     gmsHealthMonitor.installView(v);
 
-    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
+    ArrayList<GMSMember> recipient = new ArrayList<GMSMember>();
     recipient.add(mockMembers.get(0));
     ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
     SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member
@@ -367,7 +365,7 @@ public class GMSHealthMonitorJUnitTest {
     gmsHealthMonitor.processMessage(sm);
 
     await("waiting for remove(member) to be invoked").untilAsserted(() -> {
-      verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class),
+      verify(joinLeave, atLeastOnce()).remove(any(GMSMember.class),
           any(String.class));
     });
     Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
@@ -379,7 +377,7 @@ public class GMSHealthMonitorJUnitTest {
   @Test
   public void testRemoveMemberNotCalledBeforeTimeout() {
     System.out.println("testRemoveMemberNotCalledBeforeTimeout starting");
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers);
+    GMSMembershipView v = new GMSMembershipView(mockMembers.get(0), 2, mockMembers);
 
     // 3rd is current member
     when(messenger.getMemberID()).thenReturn(mockMembers.get(0)); // coordinator and local member
@@ -388,7 +386,7 @@ public class GMSHealthMonitorJUnitTest {
 
     gmsHealthMonitor.installView(v);
 
-    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
+    ArrayList<GMSMember> recipient = new ArrayList<GMSMember>();
     recipient.add(mockMembers.get(0));
     ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
     SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member
@@ -402,7 +400,7 @@ public class GMSHealthMonitorJUnitTest {
 
     await("waiting for remove(member) to be invoked")
         .untilAsserted(
-            () -> verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class),
+            () -> verify(joinLeave, atLeastOnce()).remove(any(GMSMember.class),
                 any(String.class)));
     long postRemove = System.currentTimeMillis();
 
@@ -416,7 +414,7 @@ public class GMSHealthMonitorJUnitTest {
   @Test
   public void testRemoveMemberCalledAfterDoingFinalCheckOnCoordinator() throws Exception {
 
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers);
+    GMSMembershipView v = new GMSMembershipView(mockMembers.get(0), 2, mockMembers);
 
     // preferred coordinators are 0 and 1
     when(messenger.getMemberID()).thenReturn(mockMembers.get(1));// next preferred coordinator
@@ -424,7 +422,7 @@ public class GMSHealthMonitorJUnitTest {
 
     gmsHealthMonitor.installView(v);
 
-    ArrayList<InternalDistributedMember> recipient = new ArrayList<InternalDistributedMember>();
+    ArrayList<GMSMember> recipient = new ArrayList<GMSMember>();
     recipient.add(mockMembers.get(0));
     recipient.add(mockMembers.get(1));
     ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
@@ -437,7 +435,7 @@ public class GMSHealthMonitorJUnitTest {
     gmsHealthMonitor.processMessage(sm);
 
     await("waiting for remove(member) to be invoked").untilAsserted(
-        () -> verify(joinLeave, atLeastOnce()).remove(any(InternalDistributedMember.class),
+        () -> verify(joinLeave, atLeastOnce()).remove(any(GMSMember.class),
             any(String.class)));
 
     Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
@@ -464,9 +462,9 @@ public class GMSHealthMonitorJUnitTest {
 
   @Test
   public void testCheckIfAvailableWithSimulatedHeartBeat() {
-    NetView v = installAView();
+    GMSMembershipView v = installAView();
 
-    InternalDistributedMember memberToCheck = mockMembers.get(1);
+    GMSMember memberToCheck = mockMembers.get(1);
     HeartbeatMessage fakeHeartbeat = new HeartbeatMessage();
     fakeHeartbeat.setSender(memberToCheck);
     when(messenger.send(any(HeartbeatRequestMessage.class))).then(new Answer() {
@@ -487,11 +485,11 @@ public class GMSHealthMonitorJUnitTest {
     useGMSHealthMonitorTestClass = true;
 
     try {
-      NetView v = installAView();
+      GMSMembershipView v = installAView();
 
       setFailureDetectionPorts(v);
 
-      InternalDistributedMember memberToCheck = mockMembers.get(1);
+      GMSMember memberToCheck = mockMembers.get(1);
 
       boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
       assertTrue("CheckIfAvailable should have return true", retVal);
@@ -506,11 +504,11 @@ public class GMSHealthMonitorJUnitTest {
     useGMSHealthMonitorTestClass = true;
 
     try {
-      NetView v = installAView();
+      GMSMembershipView v = installAView();
 
       setFailureDetectionPorts(v);
 
-      InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+      GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
 
       gmsHealthMonitor.setNextNeighbor(v, memberToCheck);
       assertNotEquals(memberToCheck, gmsHealthMonitor.getNextNeighbor());
@@ -533,11 +531,11 @@ public class GMSHealthMonitorJUnitTest {
     simulateHeartbeatInGMSHealthMonitorTestClass = false;
 
     try {
-      NetView v = installAView();
+      GMSMembershipView v = installAView();
 
       setFailureDetectionPorts(v);
 
-      InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+      GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
 
       boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
 
@@ -560,11 +558,11 @@ public class GMSHealthMonitorJUnitTest {
     useGMSHealthMonitorTestClass = true;
 
     try {
-      NetView v = installAView();
+      GMSMembershipView v = installAView();
 
       setFailureDetectionPorts(v);
 
-      InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+      GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
 
       gmsHealthMonitor.setNextNeighbor(v, memberToCheck);
       assertNotEquals(memberToCheck, gmsHealthMonitor.getNextNeighbor());
@@ -598,11 +596,11 @@ public class GMSHealthMonitorJUnitTest {
 
   @Test
   public void testInitiatorRewatchesSuspectAfterSuccessfulFinalCheck() {
-    NetView v = installAView();
+    GMSMembershipView v = installAView();
 
     setFailureDetectionPorts(v);
 
-    InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+    GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
     gmsHealthMonitor.memberSuspected(mockMembers.get(0), memberToCheck, "Not responding");
     assertTrue(gmsHealthMonitor.isSuspectMember(memberToCheck));
     gmsHealthMonitor.processMessage(new FinalCheckPassedMessage(mockMembers.get(0), memberToCheck));
@@ -615,14 +613,14 @@ public class GMSHealthMonitorJUnitTest {
     useGMSHealthMonitorTestClass = true;
     simulateHeartbeatInGMSHealthMonitorTestClass = false;
 
-    NetView v = installAView();
+    GMSMembershipView v = installAView();
 
     setFailureDetectionPorts(v);
 
-    InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+    GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
     boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
     assertFalse(available);
-    verify(joinLeave).remove(isA(InternalDistributedMember.class), isA(String.class));
+    verify(joinLeave).remove(isA(GMSMember.class), isA(String.class));
     assertTrue(gmsHealthMonitor.isSuspectMember(memberToCheck));
   }
 
@@ -632,15 +630,15 @@ public class GMSHealthMonitorJUnitTest {
     simulateHeartbeatInGMSHealthMonitorTestClass = false;
     allowSelfCheckToSucceed = false;
 
-    NetView v = installAView();
+    GMSMembershipView v = installAView();
 
     setFailureDetectionPorts(v);
 
-    InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+    GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
     gmsHealthMonitor.stopServer();
     boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", false);
     assertTrue(available);
-    verify(joinLeave, never()).remove(isA(InternalDistributedMember.class), isA(String.class));
+    verify(joinLeave, never()).remove(isA(GMSMember.class), isA(String.class));
     assertTrue(((GMSHealthMonitorTest) gmsHealthMonitor).availabilityCheckedMembers
         .contains(memberToCheck));
     assertTrue(((GMSHealthMonitorTest) gmsHealthMonitor).availabilityCheckedMembers
@@ -655,14 +653,14 @@ public class GMSHealthMonitorJUnitTest {
     useGMSHealthMonitorTestClass = true;
     simulateHeartbeatInGMSHealthMonitorTestClass = false;
 
-    NetView v = installAView();
+    GMSMembershipView v = installAView();
 
     setFailureDetectionPorts(v);
 
-    InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+    GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
     boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", false);
     assertFalse(available);
-    verify(joinLeave, never()).remove(isA(InternalDistributedMember.class), isA(String.class));
+    verify(joinLeave, never()).remove(isA(GMSMember.class), isA(String.class));
     assertTrue(gmsHealthMonitor.isSuspectMember(memberToCheck));
   }
 
@@ -675,14 +673,14 @@ public class GMSHealthMonitorJUnitTest {
     useGMSHealthMonitorTestClass = true;
     simulateHeartbeatInGMSHealthMonitorTestClass = false;
 
-    NetView v = installAView();
+    GMSMembershipView v = installAView();
 
     setFailureDetectionPorts(v);
 
-    InternalDistributedMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+    GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
     boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
     assertFalse(available);
-    verify(joinLeave).remove(isA(InternalDistributedMember.class), isA(String.class));
+    verify(joinLeave).remove(isA(GMSMember.class), isA(String.class));
   }
 
 
@@ -762,10 +760,11 @@ public class GMSHealthMonitorJUnitTest {
     // for some reason
     int viewId = gmsMember.getVmViewId();
 
-    InternalDistributedMember testMember =
-        new InternalDistributedMember("localhost", 9000, Version.CURRENT, gmsMember);
-    // We set to our expected test viewId in the IDM as well as reseting the gms member
-    testMember.setVmViewId(viewId);
+    GMSMember testMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, gmsMember.getUuidMSBs(),
+        gmsMember.getUuidLSBs());
+    testMember.setUdpPort(9000);
+
+    // We set to our expected test viewId in the IDM as well as resetting the gms member
     gmsMember.setBirthViewId(viewId);
 
 
@@ -797,7 +796,7 @@ public class GMSHealthMonitorJUnitTest {
 
   @Test
   public void testBeSickAndPlayDead() throws Exception {
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers);
+    GMSMembershipView v = new GMSMembershipView(mockMembers.get(0), 2, mockMembers);
     gmsHealthMonitor.installView(v);
     gmsHealthMonitor.beSick();
 
@@ -850,8 +849,8 @@ public class GMSHealthMonitorJUnitTest {
     };
     serverThread.setDaemon(true);
     serverThread.start();
-    InternalDistributedMember otherMember =
-        createInternalDistributedMember(Version.CURRENT_ORDINAL, 0, 1, 1);
+    GMSMember otherMember =
+        createGMSMember(Version.CURRENT_ORDINAL, 0, 1, 1);
     long startTime = System.currentTimeMillis();
     gmsHealthMonitor.doTCPCheckMember(otherMember, mySocket.getLocalPort(), true);
     mySocket.close();
@@ -876,10 +875,10 @@ public class GMSHealthMonitorJUnitTest {
   }
 
   private void executeTestDoTCPCheck(int receivedStatus, boolean expectedResult) throws Exception {
-    InternalDistributedMember otherMember =
-        createInternalDistributedMember(Version.CURRENT_ORDINAL, 0, 1, 1);
-    InternalDistributedMember gmsMember =
-        createInternalDistributedMember(Version.CURRENT_ORDINAL, 0, 1, 1);
+    GMSMember otherMember =
+        createGMSMember(Version.CURRENT_ORDINAL, 0, 1, 1);
+    GMSMember gmsMember =
+        createGMSMember(Version.CURRENT_ORDINAL, 0, 1, 1);
 
     // Set up the incoming/received bytes. We just wrap output streams and write out the gms member
     // information
@@ -903,21 +902,10 @@ public class GMSHealthMonitorJUnitTest {
 
     // we can check to see if the gms member information was written out by the tcp check
     byte[] bytesWritten = outputStream.toByteArray();
-    Assert.assertArrayEquals(writeMemberToBytes((GMSMember) gmsMember.getNetMember()),
+    Assert.assertArrayEquals(writeMemberToBytes(gmsMember),
         bytesWritten);
   }
 
-  private InternalDistributedMember createInternalDistributedMember(short version, int viewId,
-      long msb, long lsb) throws UnknownHostException {
-    GMSMember gmsMember = createGMSMember(version, viewId, msb, lsb);
-    InternalDistributedMember idm =
-        new InternalDistributedMember("localhost", 9000, Version.CURRENT, gmsMember);
-    // We set to our expected test viewId in the IDM as well as reseting the gms member
-    idm.setVmViewId(viewId);
-    gmsMember.setBirthViewId(viewId);
-    return idm;
-  }
-
   private GMSMember createGMSMember(short version, int viewId, long msb, long lsb)
       throws UnknownHostException {
     GMSMember gmsMember = new GMSMember();
@@ -937,10 +925,10 @@ public class GMSHealthMonitorJUnitTest {
 
   public class GMSHealthMonitorTest extends GMSHealthMonitor {
     public boolean useBlockingSocket = false;
-    public Set<InternalDistributedMember> availabilityCheckedMembers = new HashSet<>();
+    public Set<GMSMember> availabilityCheckedMembers = new HashSet<>();
 
     @Override
-    boolean doTCPCheckMember(InternalDistributedMember suspectMember, int port,
+    boolean doTCPCheckMember(GMSMember suspectMember, int port,
         boolean retryIfConnectFails) {
       availabilityCheckedMembers.add(suspectMember);
       if (useGMSHealthMonitorTestClass) {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java
index 15bc708..cc7ebf7 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java
@@ -25,7 +25,7 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.distributed.internal.LocatorStats;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.tcpserver.TcpServer;
 
 public class GMSLocatorIntegrationTest {
@@ -34,13 +34,13 @@ public class GMSLocatorIntegrationTest {
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
 
   private TcpServer tcpServer;
-  private NetView view;
+  private GMSMembershipView view;
   private GMSLocator gmsLocator;
 
   @Before
   public void setUp() {
     tcpServer = mock(TcpServer.class);
-    view = new NetView();
+    view = new GMSMembershipView();
     gmsLocator =
         new GMSLocator(null, null, false, false, new LocatorStats(), "",
             temporaryFolder.getRoot().toPath());
@@ -53,14 +53,14 @@ public class GMSLocatorIntegrationTest {
 
   @Test
   public void initDefinesViewFileInSpecifiedDirectory() {
-    gmsLocator.init(tcpServer);
+    gmsLocator.init(String.valueOf(tcpServer.getPort()));
 
     assertThat(gmsLocator.getViewFile()).isNotNull();
   }
 
   @Test
   public void installViewCreatesViewFileInSpecifiedDirectory() {
-    gmsLocator.init(tcpServer);
+    gmsLocator.init(String.valueOf(tcpServer.getPort()));
 
     gmsLocator.installView(view);
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java
index 70a5d5b..618c568 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java
@@ -17,7 +17,7 @@ package org.apache.geode.distributed.internal.membership.gms.locator;
 import static org.apache.geode.distributed.ConfigurationProperties.BIND_ADDRESS;
 import static org.apache.geode.distributed.ConfigurationProperties.DISABLE_TCP;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.internal.ClusterDistributionManager.NORMAL_DM_TYPE;
+import static org.apache.geode.distributed.internal.membership.gms.GMSMember.NORMAL_DM_TYPE;
 import static org.apache.geode.distributed.internal.membership.gms.locator.GMSLocator.LOCATOR_FILE_STAMP;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
@@ -50,8 +50,9 @@ import org.apache.geode.distributed.internal.LocatorStats;
 import org.apache.geode.distributed.internal.membership.DistributedMembershipListener;
 import org.apache.geode.distributed.internal.membership.MemberFactory;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
 import org.apache.geode.distributed.internal.membership.adapter.auth.GMSAuthenticator;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
@@ -100,7 +101,7 @@ public class GMSLocatorRecoveryIntegrationTest {
 
   @Test
   public void testRecoverFromFileWithNormalFile() throws Exception {
-    NetView view = new NetView();
+    GMSMembershipView view = new GMSMembershipView();
     populateStateFile(stateFile, LOCATOR_FILE_STAMP, Version.CURRENT_ORDINAL, view);
 
     assertThat(gmsLocator.recoverFromFile(stateFile)).isTrue();
@@ -119,10 +120,8 @@ public class GMSLocatorRecoveryIntegrationTest {
     // add 1 to ordinal to make it wrong
     populateStateFile(stateFile, LOCATOR_FILE_STAMP, Version.CURRENT_ORDINAL + 1, 1);
 
-    Throwable thrown = catchThrowable(() -> gmsLocator.recoverFromFile(stateFile));
-
-    assertThat(thrown)
-        .isInstanceOf(InternalGemFireException.class);
+    boolean recovered = gmsLocator.recoverFromFile(stateFile);
+    assertThat(recovered).isFalse();
   }
 
   @Test
@@ -173,12 +172,15 @@ public class GMSLocatorRecoveryIntegrationTest {
     gmsLocator.setViewFile(new File(temporaryFolder.getRoot(), "locator2.dat"));
     gmsLocator.init(null);
 
-    assertThat(gmsLocator.getMembers()).contains(membershipManager.getLocalMember());
+    assertThat(gmsLocator.getMembers())
+        .contains(
+            ((GMSMemberAdapter) membershipManager.getLocalMember().getNetMember()).getGmsMember());
   }
 
   @Test
   public void testViewFileNotFound() throws Exception {
-    populateStateFile(stateFile, LOCATOR_FILE_STAMP, Version.CURRENT_ORDINAL, new NetView());
+    populateStateFile(stateFile, LOCATOR_FILE_STAMP, Version.CURRENT_ORDINAL,
+        new GMSMembershipView());
     assertThat(stateFile).exists();
 
     File dir = temporaryFolder.newFolder(testName.getMethodName());
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index cb605ea..96c995c 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -55,9 +55,8 @@ import org.mockito.verification.Timeout;
 import org.apache.geode.SystemConnectException;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
@@ -91,17 +90,17 @@ public class GMSJoinLeaveJUnitTest {
   private DistributionConfig mockDistConfig;
   private Authenticator authenticator;
   private HealthMonitor healthMonitor;
-  private InternalDistributedMember gmsJoinLeaveMemberId;
-  private InternalDistributedMember[] mockMembers;
-  private InternalDistributedMember mockOldMember;
+  private GMSMember gmsJoinLeaveMemberId;
+  private GMSMember[] mockMembers;
+  private GMSMember mockOldMember;
   private Properties credentials = new Properties();
   private Messenger messenger;
   private GMSJoinLeave gmsJoinLeave;
   private Manager manager;
   private Stopper stopper;
   private TestLocator testLocator;
-  private InternalDistributedMember removeMember = null;
-  private InternalDistributedMember leaveMember = null;
+  private GMSMember removeMember = null;
+  private GMSMember leaveMember = null;
 
   public void initMocks() {
     initMocks(false);
@@ -123,7 +122,7 @@ public class GMSJoinLeaveJUnitTest {
     when(mockDistConfig.getMemberTimeout()).thenReturn(2000);
 
     authenticator = mock(Authenticator.class);
-    gmsJoinLeaveMemberId = new InternalDistributedMember("localhost", 8887);
+    gmsJoinLeaveMemberId = new GMSMember("localhost", 8887);
 
     messenger = mock(Messenger.class);
     when(messenger.getMemberID()).thenReturn(gmsJoinLeaveMemberId);
@@ -150,11 +149,11 @@ public class GMSJoinLeaveJUnitTest {
     Timer t = new Timer(true);
     when(services.getTimer()).thenReturn(t);
 
-    mockMembers = new InternalDistributedMember[4];
+    mockMembers = new GMSMember[4];
     for (int i = 0; i < mockMembers.length; i++) {
-      mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
+      mockMembers[i] = new GMSMember("localhost", 8888 + i);
     }
-    mockOldMember = new InternalDistributedMember("localhost", 8700, Version.GFE_56);
+    mockOldMember = new GMSMember("localhost", 8700, Version.GFE_56);
 
     if (useTestGMSJoinLeave) {
       gmsJoinLeave = new GMSJoinLeaveTest();
@@ -180,7 +179,7 @@ public class GMSJoinLeaveJUnitTest {
     boolean isCoordinator;
 
     @Override
-    public void installView(NetView v) {}
+    public void installView(GMSMembershipView v) {}
 
     @Override
     public void setIsCoordinator(boolean isCoordinator) {
@@ -221,7 +220,7 @@ public class GMSJoinLeaveJUnitTest {
     initMocks();
 
     int viewId = 1;
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    List<GMSMember> mbrs = new LinkedList<>();
     mbrs.add(mockMembers[0]);
     mbrs.add(mockMembers[1]);
     mbrs.add(mockMembers[2]);
@@ -229,16 +228,16 @@ public class GMSJoinLeaveJUnitTest {
     when(services.getMessenger()).thenReturn(messenger);
 
     // prepare the view
-    NetView netView = new NetView(mockMembers[0], viewId, mbrs);
+    GMSMembershipView netView = new GMSMembershipView(mockMembers[0], viewId, mbrs);
     SearchState state = gmsJoinLeave.searchState;
     state.view = netView;
     state.viewId = netView.getViewId();
 
-    InternalDistributedMember coordinator = mockMembers[2];
+    GMSMember coordinator = mockMembers[2];
     coordinator.setVmViewId(viewId);
 
     // already tried joining using members 0 and 1
-    Set<InternalDistributedMember> set = new HashSet<>();
+    Set<GMSMember> set = new HashSet<>();
     mockMembers[0].setVmViewId(viewId - 1);
     set.add(mockMembers[0]);
     mockMembers[1].setVmViewId(viewId - 1);
@@ -247,7 +246,7 @@ public class GMSJoinLeaveJUnitTest {
     state.hasContactedAJoinedLocator = true;
 
     // simulate a response being received
-    InternalDistributedMember sender = mockMembers[2];
+    GMSMember sender = mockMembers[2];
     FindCoordinatorResponse resp = new FindCoordinatorResponse(coordinator, sender, null, 0);
     gmsJoinLeave.processMessage(resp);
     // tell GMSJoinLeave that a unit test is running so it won't clear the
@@ -275,8 +274,8 @@ public class GMSJoinLeaveJUnitTest {
   public void testViewWithoutMemberInitiatesForcedDisconnect() throws Exception {
     initMocks();
     GMSJoinLeaveTestHelper.becomeCoordinatorForTest(gmsJoinLeave);
-    List<InternalDistributedMember> members = Arrays.asList(mockMembers);
-    NetView v = new NetView(mockMembers[0], 2, members);
+    List<GMSMember> members = Arrays.asList(mockMembers);
+    GMSMembershipView v = new GMSMembershipView(mockMembers[0], 2, members);
     InstallViewMessage message = getInstallViewMessage(v, null, false);
     gmsJoinLeave.processMessage(message);
     verify(manager).forceDisconnect(isA(String.class));
@@ -339,15 +338,15 @@ public class GMSJoinLeaveJUnitTest {
    * prepares and install a view
    *
    */
-  private void prepareAndInstallView(InternalDistributedMember coordinator,
-      List<InternalDistributedMember> members) throws IOException {
+  private void prepareAndInstallView(GMSMember coordinator,
+      List<GMSMember> members) throws IOException {
     int viewId = 1;
 
     when(services.getMessenger()).thenReturn(messenger);
 
     // prepare the view
-    NetView netView = new NetView(coordinator, viewId, members);
-    for (InternalDistributedMember member : netView.getMembers()) {
+    GMSMembershipView netView = new GMSMembershipView(coordinator, viewId, members);
+    for (GMSMember member : netView.getMembers()) {
       netView.setPublicKey(member, member.toString());
     }
     InstallViewMessage installViewMessage = getInstallViewMessage(netView, credentials, true);
@@ -360,10 +359,10 @@ public class GMSJoinLeaveJUnitTest {
     Assert.assertEquals(netView, gmsJoinLeave.getView());
   }
 
-  private List<InternalDistributedMember> createMemberList(InternalDistributedMember... members) {
-    List<InternalDistributedMember> memberList =
-        new ArrayList<InternalDistributedMember>(members.length);
-    for (InternalDistributedMember member : members) {
+  private List<GMSMember> createMemberList(GMSMember... members) {
+    List<GMSMember> memberList =
+        new ArrayList<GMSMember>(members.length);
+    for (GMSMember member : members) {
       memberList.add(member);
     }
     return memberList;
@@ -401,7 +400,7 @@ public class GMSJoinLeaveJUnitTest {
     initMocks();
     final int viewInstallationTime = 15000;
 
-    when(healthMonitor.checkIfAvailable(isA(InternalDistributedMember.class), isA(String.class),
+    when(healthMonitor.checkIfAvailable(isA(GMSMember.class), isA(String.class),
         isA(Boolean.class))).thenReturn(true);
 
     gmsJoinLeave.delayViewCreationForTest(5000); // ensures multiple requests are queued for a view
@@ -411,9 +410,9 @@ public class GMSJoinLeaveJUnitTest {
     await()
         .until(() -> gmsJoinLeave.getView() != null);
 
-    NetView oldView = gmsJoinLeave.getView();
+    GMSMembershipView oldView = gmsJoinLeave.getView();
 
-    NetView newView = new NetView(oldView, oldView.getViewId() + 1);
+    GMSMembershipView newView = new GMSMembershipView(oldView, oldView.getViewId() + 1);
     newView.add(mockMembers[1]);
     newView.add(mockMembers[2]);
     gmsJoinLeave.installView(newView);
@@ -439,12 +438,12 @@ public class GMSJoinLeaveJUnitTest {
     initMocks();
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
 
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    List<GMSMember> mbrs = new LinkedList<>();
     mbrs.add(mockMembers[0]);
     mbrs.add(mockMembers[1]);
 
     // try to install an older view where viewId < currentView.viewId
-    NetView olderNetView = new NetView(mockMembers[0], 0, mbrs);
+    GMSMembershipView olderNetView = new GMSMembershipView(mockMembers[0], 0, mbrs);
     InstallViewMessage installViewMessage = getInstallViewMessage(olderNetView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
     Assert.assertNotEquals(gmsJoinLeave.getView(), olderNetView);
@@ -458,13 +457,13 @@ public class GMSJoinLeaveJUnitTest {
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
 
     int viewId = 2;
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    List<GMSMember> mbrs = new LinkedList<>();
     mbrs.add(mockMembers[1]);
     mbrs.add(mockMembers[2]);
     mbrs.add(mockMembers[3]);
 
     // install the view
-    NetView netView = new NetView(mockMembers[0], viewId, mbrs);
+    GMSMembershipView netView = new GMSMembershipView(mockMembers[0], viewId, mbrs);
     InstallViewMessage installViewMessage = getInstallViewMessage(netView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
 
@@ -479,7 +478,7 @@ public class GMSJoinLeaveJUnitTest {
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
     // test that a non-member can't remove another member
     RemoveMemberMessage msg = new RemoveMemberMessage(mockMembers[0], mockMembers[1], reason);
-    msg.setSender(new InternalDistributedMember("localhost", 9000));
+    msg.setSender(new GMSMember("localhost", 9000));
     gmsJoinLeave.processMessage(msg);
     assertTrue("RemoveMemberMessage should not have been added to view requests",
         gmsJoinLeave.getViewRequests().size() == 0);
@@ -504,7 +503,7 @@ public class GMSJoinLeaveJUnitTest {
 
     waitForViewAndNoRequestsInProgress(7);
 
-    NetView view = gmsJoinLeave.getView();
+    GMSMembershipView view = gmsJoinLeave.getView();
     assertTrue("expected member to be removed: " + mockMembers[0] + "; view: " + view,
         !view.contains(mockMembers[0]));
     assertTrue("expected member to be in shutdownMembers collection: " + mockMembers[0] + "; view: "
@@ -530,7 +529,7 @@ public class GMSJoinLeaveJUnitTest {
 
     waitForViewAndNoRequestsInProgress(7);
 
-    NetView view = gmsJoinLeave.getView();
+    GMSMembershipView view = gmsJoinLeave.getView();
     assertTrue("expected member to be removed: " + mockMembers[0] + "; view: " + view,
         !view.contains(mockMembers[0]));
     assertTrue(
@@ -541,7 +540,7 @@ public class GMSJoinLeaveJUnitTest {
   @Test
   public void testDuplicateJoinRequestDoesNotCauseNewView() throws Exception {
     initMocks();
-    when(healthMonitor.checkIfAvailable(isA(InternalDistributedMember.class), isA(String.class),
+    when(healthMonitor.checkIfAvailable(isA(GMSMember.class), isA(String.class),
         isA(Boolean.class))).thenReturn(true);
     gmsJoinLeave.unitTesting.add("noRandomViewChange");
     prepareAndInstallView(gmsJoinLeaveMemberId,
@@ -558,19 +557,19 @@ public class GMSJoinLeaveJUnitTest {
 
     waitForViewAndNoRequestsInProgress(7);
 
-    NetView view = gmsJoinLeave.getView();
+    GMSMembershipView view = gmsJoinLeave.getView();
     assertTrue("expected member to be added: " + mockMembers[2] + "; view: " + view,
         view.contains(mockMembers[2]));
-    List<InternalDistributedMember> members = view.getMembers();
+    List<GMSMember> members = view.getMembers();
     int occurrences = 0;
-    for (InternalDistributedMember mbr : members) {
+    for (GMSMember mbr : members) {
       if (mbr.equals(mockMembers[2])) {
         occurrences += 1;
       }
     }
     assertTrue("expected member to only be in the view once: " + mockMembers[2] + "; view: " + view,
         occurrences == 1);
-    verify(healthMonitor, times(5)).checkIfAvailable(isA(InternalDistributedMember.class),
+    verify(healthMonitor, times(5)).checkIfAvailable(isA(GMSMember.class),
         isA(String.class), isA(Boolean.class));
   }
 
@@ -599,7 +598,7 @@ public class GMSJoinLeaveJUnitTest {
         createMemberList(gmsJoinLeaveMemberId, mockMembers[0], mockMembers[1]));
     reset(messenger);
     RemoveMemberMessage msg = new RemoveMemberMessage(gmsJoinLeaveMemberId,
-        new InternalDistributedMember("localhost", 10000), "removing for test");
+        new GMSMember("localhost", 10000), "removing for test");
     msg.setSender(mockMembers[0]);
     gmsJoinLeave.processMessage(msg);
     verify(messenger).send(isA(RemoveMemberMessage.class));
@@ -610,10 +609,12 @@ public class GMSJoinLeaveJUnitTest {
     initMocks();
     // gmsJoinLeave mistakenly uses an old viewID when joining, making it a rogue member
     gmsJoinLeaveMemberId.setVmViewId(-1);
-    InternalDistributedMember previousMemberId =
-        new InternalDistributedMember(gmsJoinLeaveMemberId.getId(), gmsJoinLeaveMemberId.getPort());
+    GMSMember previousMemberId =
+        new GMSMember(gmsJoinLeaveMemberId.getInetAddress().getHostName(),
+            gmsJoinLeaveMemberId.getPort());
     previousMemberId.setVmViewId(0);
-    NetView view = new NetView(mockMembers[0], 1,
+    previousMemberId.setUUID(gmsJoinLeaveMemberId.getUUID());
+    GMSMembershipView view = new GMSMembershipView(mockMembers[0], 1,
         createMemberList(mockMembers[0], previousMemberId, mockMembers[1]));
     InstallViewMessage viewMessage = new InstallViewMessage(view, 0, false);
     viewMessage.setSender(mockMembers[0]);
@@ -680,9 +681,9 @@ public class GMSJoinLeaveJUnitTest {
     String reason = "testing";
     initMocks();
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
-    NetView view = gmsJoinLeave.getView();
+    GMSMembershipView view = gmsJoinLeave.getView();
     view.add(gmsJoinLeaveMemberId);
-    InternalDistributedMember creator = view.getCreator();
+    GMSMember creator = view.getCreator();
     LeaveRequestMessage msg = new LeaveRequestMessage(creator, creator, reason);
     msg.setSender(creator);
     gmsJoinLeave.processMessage(msg);
@@ -694,9 +695,9 @@ public class GMSJoinLeaveJUnitTest {
     String reason = "testing";
     initMocks();
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
-    NetView view = gmsJoinLeave.getView();
+    GMSMembershipView view = gmsJoinLeave.getView();
     view.add(gmsJoinLeaveMemberId);
-    InternalDistributedMember creator = view.getCreator();
+    GMSMember creator = view.getCreator();
     RemoveMemberMessage msg = new RemoveMemberMessage(creator, creator, reason);
     msg.setSender(creator);
     gmsJoinLeave.processMessage(msg);
@@ -710,7 +711,7 @@ public class GMSJoinLeaveJUnitTest {
   @Test
   public void testBecomeCoordinatorThroughShutdownWhenOlderMemberCrashed() throws Exception {
     initMocks();
-    InternalDistributedMember A = mockMembers[0],
+    GMSMember A = mockMembers[0],
         B = gmsJoinLeaveMemberId,
         C = mockMembers[1],
         D = mockMembers[2],
@@ -733,7 +734,7 @@ public class GMSJoinLeaveJUnitTest {
   @Test
   public void testBecomeCoordinatorAndAcceptMemberWithViewID() throws Exception {
     initMocks();
-    InternalDistributedMember A = mockMembers[0],
+    GMSMember A = mockMembers[0],
         B = gmsJoinLeaveMemberId,
         C = mockMembers[1],
         D = mockMembers[2],
@@ -747,7 +748,7 @@ public class GMSJoinLeaveJUnitTest {
     gmsJoinLeave.processMessage(msg);
     assertTrue("Expected becomeCoordinator to be invoked", gmsJoinLeave.isCoordinator());
     await().until(() -> {
-      NetView preparedView = gmsJoinLeave.getPreparedView();
+      GMSMembershipView preparedView = gmsJoinLeave.getPreparedView();
       return preparedView != null && preparedView.contains(E);
     });
   }
@@ -756,10 +757,10 @@ public class GMSJoinLeaveJUnitTest {
   public void testBecomeCoordinatorThroughViewChange() throws Exception {
     initMocks();
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
-    NetView oldView = gmsJoinLeave.getView();
+    GMSMembershipView oldView = gmsJoinLeave.getView();
     oldView.add(gmsJoinLeaveMemberId);
-    NetView view = new NetView(oldView, oldView.getViewId() + 1);
-    InternalDistributedMember creator = view.getCreator();
+    GMSMembershipView view = new GMSMembershipView(oldView, oldView.getViewId() + 1);
+    GMSMember creator = view.getCreator();
     view.remove(creator);
     InstallViewMessage msg = getInstallViewMessage(view, creator, false);
     msg.setSender(creator);
@@ -772,10 +773,10 @@ public class GMSJoinLeaveJUnitTest {
       throws Exception {
     initMocks();
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
-    NetView oldView = gmsJoinLeave.getView();
+    GMSMembershipView oldView = gmsJoinLeave.getView();
     oldView.add(gmsJoinLeaveMemberId);
-    NetView view = new NetView(oldView, oldView.getViewId() + 1);
-    InternalDistributedMember creator = view.getCreator();
+    GMSMembershipView view = new GMSMembershipView(oldView, oldView.getViewId() + 1);
+    GMSMember creator = view.getCreator();
     LeaveRequestMessage leaveRequestMessage =
         new LeaveRequestMessage(gmsJoinLeaveMemberId, mockMembers[0], "leaving for test");
     gmsJoinLeave.processMessage(leaveRequestMessage);
@@ -790,11 +791,11 @@ public class GMSJoinLeaveJUnitTest {
   public void testBecomeParticipantThroughViewChange() throws Exception {
     initMocks();
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
-    NetView oldView = gmsJoinLeave.getView();
+    GMSMembershipView oldView = gmsJoinLeave.getView();
     oldView.add(gmsJoinLeaveMemberId);
-    InternalDistributedMember creator = oldView.getCreator();
+    GMSMember creator = oldView.getCreator();
     GMSJoinLeaveTestHelper.becomeCoordinatorForTest(gmsJoinLeave);
-    NetView view = new NetView(2, gmsJoinLeave.getView().getViewId() + 1);
+    GMSMembershipView view = new GMSMembershipView(2, gmsJoinLeave.getView().getViewId() + 1);
     view.setCreator(creator);
     view.add(creator);
     view.add(gmsJoinLeaveMemberId);
@@ -804,7 +805,7 @@ public class GMSJoinLeaveJUnitTest {
     assertTrue("Expected it to stop being coordinator", !gmsJoinLeave.isCoordinator());
   }
 
-  private InstallViewMessage getInstallViewMessage(NetView view, Object credentials,
+  private InstallViewMessage getInstallViewMessage(GMSMembershipView view, Object credentials,
       boolean preparing) {
     InstallViewMessage installViewMessage = new InstallViewMessage(view, credentials, preparing);
     installViewMessage.setSender(gmsJoinLeaveMemberId);
@@ -819,26 +820,28 @@ public class GMSJoinLeaveJUnitTest {
     // set up a view with sufficient members, then create a new view
     // where enough weight is lost to cause a network partition
 
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    List<GMSMember> mbrs = new LinkedList<>();
     mbrs.add(mockMembers[0]);
     mbrs.add(mockMembers[1]);
     mbrs.add(mockMembers[2]);
     mbrs.add(gmsJoinLeaveMemberId);
 
-    ((GMSMember) mockMembers[1].getNetMember()).setMemberWeight((byte) 20);
+    mockMembers[1].setMemberWeight((byte) 20);
 
-    NetView newView = new NetView(mockMembers[0], gmsJoinLeave.getView().getViewId() + 1, mbrs);
+    GMSMembershipView newView =
+        new GMSMembershipView(mockMembers[0], gmsJoinLeave.getView().getViewId() + 1, mbrs);
     InstallViewMessage installViewMessage = getInstallViewMessage(newView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
 
-    Set<InternalDistributedMember> crashes = new HashSet<>();
+    Set<GMSMember> crashes = new HashSet<>();
     crashes.add(mockMembers[1]);
     crashes.add(mockMembers[2]);
     mbrs = new LinkedList<>(mbrs);
     mbrs.remove(mockMembers[1]);
     mbrs.remove(mockMembers[2]);
-    NetView partitionView =
-        new NetView(mockMembers[0], newView.getViewId() + 1, mbrs, Collections.emptySet(), crashes);
+    GMSMembershipView partitionView =
+        new GMSMembershipView(mockMembers[0], newView.getViewId() + 1, mbrs, Collections.emptySet(),
+            crashes);
     installViewMessage = getInstallViewMessage(partitionView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
 
@@ -865,18 +868,19 @@ public class GMSJoinLeaveJUnitTest {
     // set up a view with sufficient members, then create a new view
     // where enough weight is lost to cause a network partition
 
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
-    Set<InternalDistributedMember> shutdowns = new HashSet<>();
-    Set<InternalDistributedMember> crashes = new HashSet<>();
+    List<GMSMember> mbrs = new LinkedList<>();
+    Set<GMSMember> shutdowns = new HashSet<>();
+    Set<GMSMember> crashes = new HashSet<>();
     mbrs.add(mockMembers[0]);
     mbrs.add(mockMembers[1]);
     mbrs.add(mockMembers[2]);
     mbrs.add(gmsJoinLeaveMemberId);
 
-    ((GMSMember) mockMembers[1].getNetMember()).setMemberWeight((byte) 20);
+    mockMembers[1].setMemberWeight((byte) 20);
 
-    NetView newView = new NetView(mockMembers[0], gmsJoinLeave.getView().getViewId() + 1, mbrs,
-        shutdowns, crashes);
+    GMSMembershipView newView =
+        new GMSMembershipView(mockMembers[0], gmsJoinLeave.getView().getViewId() + 1, mbrs,
+            shutdowns, crashes);
     InstallViewMessage installViewMessage = getInstallViewMessage(newView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
 
@@ -886,8 +890,8 @@ public class GMSJoinLeaveJUnitTest {
     mbrs = new LinkedList<>(mbrs);
     mbrs.remove(mockMembers[1]);
     mbrs.remove(mockMembers[2]);
-    NetView partitionView =
-        new NetView(mockMembers[0], newView.getViewId() + 1, mbrs, shutdowns, crashes);
+    GMSMembershipView partitionView =
+        new GMSMembershipView(mockMembers[0], newView.getViewId() + 1, mbrs, shutdowns, crashes);
     installViewMessage = getInstallViewMessage(partitionView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
 
@@ -900,12 +904,12 @@ public class GMSJoinLeaveJUnitTest {
     initMocks(true);
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
 
-    NetView gmsView = gmsJoinLeave.getView();
-    NetView newView = new NetView(gmsView, gmsView.getViewId() + 6);
+    GMSMembershipView gmsView = gmsJoinLeave.getView();
+    GMSMembershipView newView = new GMSMembershipView(gmsView, gmsView.getViewId() + 6);
     InstallViewMessage msg = getInstallViewMessage(newView, null, true);
     gmsJoinLeave.processMessage(msg);
 
-    NetView alternateView = new NetView(gmsView, gmsView.getViewId() + 1);
+    GMSMembershipView alternateView = new GMSMembershipView(gmsView, gmsView.getViewId() + 1);
     msg = getInstallViewMessage(alternateView, null, true);
     gmsJoinLeave.processMessage(msg);
 
@@ -915,11 +919,11 @@ public class GMSJoinLeaveJUnitTest {
   @Test
   public void testNoViewAckCausesRemovalMessage() throws Exception {
     initMocks(true);
-    when(healthMonitor.checkIfAvailable(isA(InternalDistributedMember.class), isA(String.class),
+    when(healthMonitor.checkIfAvailable(isA(GMSMember.class), isA(String.class),
         isA(Boolean.class))).thenReturn(false);
     prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
-    NetView oldView = gmsJoinLeave.getView();
-    NetView newView = new NetView(oldView, oldView.getViewId() + 1);
+    GMSMembershipView oldView = gmsJoinLeave.getView();
+    GMSMembershipView newView = new GMSMembershipView(oldView, oldView.getViewId() + 1);
 
     // the new view will remove the old coordinator (normal shutdown) and add a new member
     // who will not ack the view. This should cause it to be removed from the system
@@ -937,7 +941,7 @@ public class GMSJoinLeaveJUnitTest {
     // wait for suspect processing
 
     verify(healthMonitor, timeout(10000).atLeast(1)).checkIfAvailable(
-        isA(InternalDistributedMember.class),
+        isA(GMSMember.class),
         isA(String.class), isA(Boolean.class));
     // verify(messenger, atLeast(1)).send(isA(RemoveMemberMessage.class));
   }
@@ -957,7 +961,7 @@ public class GMSJoinLeaveJUnitTest {
 
     // The coordinator shuts down
     gmsJoinLeave.memberShutdown(mockMembers[0], "Shutdown");
-    NetView nextView = gmsJoinLeave.getViewCreator().initialView;
+    GMSMembershipView nextView = gmsJoinLeave.getViewCreator().initialView;
 
     assertTrue(gmsJoinLeave.isCoordinator());
     assertTrue(nextView.getCoordinator().equals(gmsJoinLeaveMemberId));
@@ -984,7 +988,7 @@ public class GMSJoinLeaveJUnitTest {
     gmsJoinLeave.memberShutdown(mockMembers[1], "Shutdown");
     gmsJoinLeave.memberShutdown(mockMembers[2], "Shutdown");
     gmsJoinLeave.memberShutdown(mockMembers[0], "Shutdown");
-    NetView nextView = gmsJoinLeave.getViewCreator().initialView;
+    GMSMembershipView nextView = gmsJoinLeave.getViewCreator().initialView;
 
     assertTrue(gmsJoinLeave.isCoordinator());
     assertTrue(nextView.getCoordinator().equals(gmsJoinLeaveMemberId));
@@ -1002,22 +1006,22 @@ public class GMSJoinLeaveJUnitTest {
   public void testCoordinatorGetsConflictingViewFromLocator() throws Exception {
     // create the GMSJoinLeave instance we'll be testing
     initMocks(false);
-    InternalDistributedMember otherMember = mockMembers[0];
+    GMSMember otherMember = mockMembers[0];
     gmsJoinLeaveMemberId.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
-    List<InternalDistributedMember> members = createMemberList(gmsJoinLeaveMemberId, otherMember);
+    List<GMSMember> members = createMemberList(gmsJoinLeaveMemberId, otherMember);
     prepareAndInstallView(gmsJoinLeaveMemberId, members);
-    NetView installedView = gmsJoinLeave.getView();
+    GMSMembershipView installedView = gmsJoinLeave.getView();
 
     gmsJoinLeave.unitTesting.add("noRandomViewChange"); // keep view numbers predictable
 
     // create a view coming from the locator that conflicts with the installed view
-    InternalDistributedMember locatorMemberId = new InternalDistributedMember("localhost",
+    GMSMember locatorMemberId = new GMSMember("localhost",
         mockMembers[mockMembers.length - 1].getPort() + 1);
     locatorMemberId.setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
-    List<InternalDistributedMember> newMemberList = new ArrayList<>(members);
+    List<GMSMember> newMemberList = new ArrayList<>(members);
     newMemberList.add(locatorMemberId);
-    NetView locatorView =
-        new NetView(locatorMemberId, installedView.getViewId() + 10, newMemberList);
+    GMSMembershipView locatorView =
+        new GMSMembershipView(locatorMemberId, installedView.getViewId() + 10, newMemberList);
 
     // start the process to make our GMSJoinLeave become coordinator. It will send out a view
     // and want an ACK from
@@ -1064,14 +1068,14 @@ public class GMSJoinLeaveJUnitTest {
 
     // Install a view that still contains one of the left members (as if something like a new
     // member, triggered a new view before coordinator leaves)
-    NetView netView = new NetView(mockMembers[0], 3/* new view id */,
+    GMSMembershipView netView = new GMSMembershipView(mockMembers[0], 3/* new view id */,
         createMemberList(mockMembers[0], gmsJoinLeaveMemberId, mockMembers[1], mockMembers[3]));
     InstallViewMessage installViewMessage = getInstallViewMessage(netView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
 
     // Now coordinator leaves
     gmsJoinLeave.memberShutdown(mockMembers[0], "Shutdown");
-    NetView nextView = gmsJoinLeave.getViewCreator().initialView;
+    GMSMembershipView nextView = gmsJoinLeave.getViewCreator().initialView;
 
     assertTrue(gmsJoinLeave.isCoordinator());
     assertTrue(nextView.getCoordinator().equals(gmsJoinLeaveMemberId));
@@ -1083,7 +1087,7 @@ public class GMSJoinLeaveJUnitTest {
   @Test
   public void testViewBroadcaster() throws Exception {
     initMocks();
-    List<InternalDistributedMember> members = new ArrayList<>(Arrays.asList(mockMembers));
+    List<GMSMember> members = new ArrayList<>(Arrays.asList(mockMembers));
     gmsJoinLeaveMemberId.setVmViewId(1);
     members.add(gmsJoinLeaveMemberId);
     prepareAndInstallView(gmsJoinLeaveMemberId, members);
@@ -1093,10 +1097,10 @@ public class GMSJoinLeaveJUnitTest {
     verify(messenger).sendUnreliably(isA(InstallViewMessage.class));
   }
 
-  private void installView(int viewId, InternalDistributedMember coordinator,
-      List<InternalDistributedMember> members) throws IOException {
+  private void installView(int viewId, GMSMember coordinator,
+      List<GMSMember> members) throws IOException {
     // prepare the view
-    NetView netView = new NetView(coordinator, viewId, members);
+    GMSMembershipView netView = new GMSMembershipView(coordinator, viewId, members);
     InstallViewMessage installViewMessage = getInstallViewMessage(netView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
     // verify(messenger).send(isA(ViewAckMessage.class));
@@ -1143,7 +1147,7 @@ public class GMSJoinLeaveJUnitTest {
       initMocks(false);
       System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
       gmsJoinLeave.join();
-      Set<InternalDistributedMember> recips = new HashSet<>();
+      Set<GMSMember> recips = new HashSet<>();
       recips.add(mockMembers[0]);
       recips.add(mockMembers[1]);
       recips.add(mockMembers[2]);
@@ -1174,7 +1178,7 @@ public class GMSJoinLeaveJUnitTest {
       initMocks(false);
       System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
       gmsJoinLeave.join();
-      Set<InternalDistributedMember> recips = new HashSet<>();
+      Set<GMSMember> recips = new HashSet<>();
       recips.add(mockMembers[0]);
       recips.add(mockMembers[1]);
       recips.add(mockMembers[2]);
@@ -1183,9 +1187,9 @@ public class GMSJoinLeaveJUnitTest {
       prepareProcessor.initialize(1, recips);
       assertTrue("Prepare processor should be waiting ",
           gmsJoinLeave.testPrepareProcessorWaiting());
-      Set<InternalDistributedMember> pendingLeaves = new HashSet<>();
+      Set<GMSMember> pendingLeaves = new HashSet<>();
       pendingLeaves.add(mockMembers[0]);
-      Set<InternalDistributedMember> pendingRemovals = new HashSet<>();
+      Set<GMSMember> pendingRemovals = new HashSet<>();
       pendingRemovals.add(mockMembers[1]);
 
       prepareProcessor.processPendingRequests(pendingLeaves, pendingRemovals);
@@ -1208,7 +1212,7 @@ public class GMSJoinLeaveJUnitTest {
   // gmsJoinLeaveMemberId.getNetMember().setPreferredForCoordinator(false);
   // JoinRequestMessage reqMsg = new JoinRequestMessage(gmsJoinLeaveMemberId, mockMembers[0], null,
   // 56734);
-  // InternalDistributedMember ids = new InternalDistributedMember("localhost", 97898);
+  // GMSMember ids = new GMSMember("localhost", 97898);
   // ids.getNetMember().setPreferredForCoordinator(true);
   // gmsJoinLeave.processMessage(reqMsg);
   // ArgumentCaptor<JoinResponseMessage> ac = ArgumentCaptor.forClass(JoinResponseMessage.class);
@@ -1297,8 +1301,8 @@ public class GMSJoinLeaveJUnitTest {
     prepareAndInstallView(gmsJoinLeaveMemberId,
         createMemberList(gmsJoinLeaveMemberId, mockMembers[0]));
     // a new member is joining
-    NetView preparedView =
-        new NetView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId() + 5);
+    GMSMembershipView preparedView =
+        new GMSMembershipView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId() + 5);
     mockMembers[1].setVmViewId(preparedView.getViewId());
     preparedView.add(mockMembers[1]);
 
@@ -1326,7 +1330,7 @@ public class GMSJoinLeaveJUnitTest {
     gmsJoinLeave.processMessage(vack);
 
     await("view creator finishes").until(() -> vc.waiting);
-    NetView newView = gmsJoinLeave.getView();
+    GMSMembershipView newView = gmsJoinLeave.getView();
     System.out.println("new view is " + newView);
     assertTrue(newView.contains(mockMembers[1]));
     assertTrue(newView.getViewId() > preparedView.getViewId());
@@ -1335,14 +1339,14 @@ public class GMSJoinLeaveJUnitTest {
   @Test
   public void testPublicKeyForNewMemberFromPreparedViewIsInstalledInNewView() throws Exception {
     initMocks(false);
-    InternalDistributedMember newMember = mockMembers[1];
+    GMSMember newMember = mockMembers[1];
 
     prepareAndInstallView(gmsJoinLeaveMemberId,
         createMemberList(gmsJoinLeaveMemberId, mockMembers[0]));
     // a new member is joining
-    NetView preparedView =
-        new NetView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId() + 5);
-    for (InternalDistributedMember member : preparedView.getMembers()) {
+    GMSMembershipView preparedView =
+        new GMSMembershipView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId() + 5);
+    for (GMSMember member : preparedView.getMembers()) {
       preparedView.setPublicKey(member, member.toString());
     }
     newMember.setVmViewId(preparedView.getViewId());
@@ -1373,34 +1377,34 @@ public class GMSJoinLeaveJUnitTest {
     gmsJoinLeave.processMessage(vack);
 
     await("view creator finishes").until(() -> vc.waiting);
-    NetView newView = gmsJoinLeave.getView();
+    GMSMembershipView newView = gmsJoinLeave.getView();
     System.out.println("new view is " + newView);
     assertTrue(newView.contains(newMember));
     assertNotNull(newView.getPublicKey(newMember));
   }
 
-  private NetView createView() {
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
-    Set<InternalDistributedMember> shutdowns = new HashSet<>();
-    Set<InternalDistributedMember> crashes = new HashSet<>();
+  private GMSMembershipView createView() {
+    List<GMSMember> mbrs = new LinkedList<>();
+    Set<GMSMember> shutdowns = new HashSet<>();
+    Set<GMSMember> crashes = new HashSet<>();
     mbrs.add(mockMembers[0]);
     mbrs.add(mockMembers[1]);
     mbrs.add(mockMembers[2]);
     mbrs.add(gmsJoinLeaveMemberId);
 
     // prepare the view
-    NetView netView = new NetView(mockMembers[0], 1, mbrs, shutdowns, crashes);
+    GMSMembershipView netView = new GMSMembershipView(mockMembers[0], 1, mbrs, shutdowns, crashes);
     return netView;
   }
 
   @Test
   public void testCoordinatorFindRequestSuccess() throws Exception {
     initMocks(false);
-    HashSet<InternalDistributedMember> registrants = new HashSet<>();
+    HashSet<GMSMember> registrants = new HashSet<>();
     registrants.add(mockMembers[0]);
     FindCoordinatorResponse fcr = new FindCoordinatorResponse(mockMembers[0], mockMembers[0], false,
         null, registrants, false, true, null);
-    NetView view = createView();
+    GMSMembershipView view = createView();
 
     TcpClientWrapper tcpClientWrapper = mock(TcpClientWrapper.class);
     gmsJoinLeave.setTcpClientWrapper(tcpClientWrapper);
@@ -1417,11 +1421,11 @@ public class GMSJoinLeaveJUnitTest {
   public void testCoordinatorFindRequestFailure() throws Exception {
     try {
       initMocks(false);
-      HashSet<InternalDistributedMember> registrants = new HashSet<>();
+      HashSet<GMSMember> registrants = new HashSet<>();
       registrants.add(mockMembers[0]);
       FindCoordinatorResponse fcr = new FindCoordinatorResponse(mockMembers[0], mockMembers[0],
           false, null, registrants, false, true, null);
-      NetView view = createView();
+      GMSMembershipView view = createView();
       JoinResponseMessage jrm = new JoinResponseMessage(mockMembers[0], view, 0);
       gmsJoinLeave.setJoinResponseMessage(jrm);
 
@@ -1460,7 +1464,7 @@ public class GMSJoinLeaveJUnitTest {
     }
 
     @Override
-    boolean checkIfAvailable(InternalDistributedMember fmbr) {
+    boolean checkIfAvailable(GMSMember fmbr) {
       if (removeMember != null) {
         try {
           if (removeMember.equals(fmbr)) {
@@ -1534,7 +1538,7 @@ public class GMSJoinLeaveJUnitTest {
   private void installView() throws Exception {
     final int viewInstallationTime = 15000;
 
-    NetView oldView = null;
+    GMSMembershipView oldView = null;
     long giveup = System.currentTimeMillis() + viewInstallationTime;
     while (System.currentTimeMillis() < giveup && oldView == null) {
       Thread.sleep(500);
@@ -1542,19 +1546,19 @@ public class GMSJoinLeaveJUnitTest {
     }
     assertTrue(oldView != null); // it should have become coordinator and installed a view
 
-    NetView newView = new NetView(oldView, oldView.getViewId() + 1);
+    GMSMembershipView newView = new GMSMembershipView(oldView, oldView.getViewId() + 1);
     newView.add(mockMembers[0]);
     newView.add(mockMembers[1]);
     gmsJoinLeave.installView(newView);
   }
 
-  private void processJoinMessage(InternalDistributedMember coordinator,
-      InternalDistributedMember newMember, int port) {
+  private void processJoinMessage(GMSMember coordinator,
+      GMSMember newMember, int port) {
     JoinRequestMessage reqMsg = new JoinRequestMessage(coordinator, newMember, null, port, 0);
     gmsJoinLeave.processMessage(reqMsg);
   }
 
-  private void processRemoveMessage(InternalDistributedMember rMember) {
+  private void processRemoveMessage(GMSMember rMember) {
     RemoveMemberMessage msg =
         new RemoveMemberMessage(gmsJoinLeave.getMemberID(), rMember, "testing");
     msg.setSender(gmsJoinLeave.getMemberID());
@@ -1562,7 +1566,7 @@ public class GMSJoinLeaveJUnitTest {
     gmsJoinLeave.processMessage(msg);
   }
 
-  private void processLeaveMessage(InternalDistributedMember rMember) {
+  private void processLeaveMessage(GMSMember rMember) {
     LeaveRequestMessage msg =
         new LeaveRequestMessage(gmsJoinLeave.getMemberID(), rMember, "testing");
     msg.setSender(rMember);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
index 5a2a5c1..5309c91 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
@@ -31,8 +31,8 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
@@ -47,9 +47,9 @@ public class GMSEncryptJUnitTest {
 
   Services services;
 
-  InternalDistributedMember mockMembers[];
+  GMSMember mockMembers[];
 
-  NetView netView;
+  GMSMembershipView netView;
 
   @Rule
   public ConcurrencyRule concurrencyRule = new ConcurrencyRule();
@@ -70,18 +70,18 @@ public class GMSEncryptJUnitTest {
     services = mock(Services.class);
     when(services.getConfig()).thenReturn(serviceConfig);
 
-    mockMembers = new InternalDistributedMember[4];
+    mockMembers = new GMSMember[4];
     for (int i = 0; i < mockMembers.length; i++) {
-      mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
+      mockMembers[i] = new GMSMember("localhost", 8888 + i);
     }
     int viewId = 1;
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    List<GMSMember> mbrs = new LinkedList<>();
     mbrs.add(mockMembers[0]);
     mbrs.add(mockMembers[1]);
     mbrs.add(mockMembers[2]);
 
     // prepare the view
-    netView = new NetView(mockMembers[0], viewId, mbrs);
+    netView = new GMSMembershipView(mockMembers[0], viewId, mbrs);
 
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
index ea8a082..1820055 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
@@ -40,22 +40,22 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
 public class GMSQuorumCheckerJUnitTest {
 
-  private InternalDistributedMember[] mockMembers;
+  private GMSMember[] mockMembers;
   private JChannel channel;
   private JGAddress address;
 
   @Before
   public void initMocks() {
-    mockMembers = new InternalDistributedMember[12];
+    mockMembers = new GMSMember[12];
     for (int i = 0; i < mockMembers.length; i++) {
-      mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
+      mockMembers[i] = new GMSMember("localhost", 8888 + i);
     }
     channel = mock(JChannel.class);
     address = mock(JGAddress.class);
@@ -68,7 +68,7 @@ public class GMSQuorumCheckerJUnitTest {
 
   @Test
   public void testQuorumCheckerAllRespond() throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     Set<Integer> pongResponders = new HashSet<>();
     for (int i = 0; i < mockMembers.length; i++) {
       pongResponders.add(mockMembers[i].getPort());
@@ -88,7 +88,7 @@ public class GMSQuorumCheckerJUnitTest {
 
   @Test
   public void testQuorumCheckerMajorityRespond() throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     Set<Integer> pongResponders = new HashSet<>();
     for (int i = 0; i < mockMembers.length - 1; i++) {
       pongResponders.add(mockMembers[i].getPort());
@@ -105,7 +105,7 @@ public class GMSQuorumCheckerJUnitTest {
 
   @Test
   public void testQuorumCheckerNotEnoughWeightForQuorum() throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     Set<Integer> pongResponders = new HashSet<>();
     pongResponders.add(mockMembers[0].getPort());
     PingMessageAnswer answerer = new PingMessageAnswer(channel, pongResponders);
@@ -120,7 +120,7 @@ public class GMSQuorumCheckerJUnitTest {
 
   @Test
   public void testQuorumCheckerNoQuorumNoResponders() throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     Set<Integer> pongResponders = new HashSet<Integer>();
     PingMessageAnswer answerer = new PingMessageAnswer(channel, pongResponders);
     Mockito.doAnswer(answerer).when(channel).send(any(Message.class));
@@ -134,7 +134,7 @@ public class GMSQuorumCheckerJUnitTest {
 
   @Test
   public void testQuorumChecker10Servers2Locators4ServersLost() throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
     mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
 
@@ -161,7 +161,7 @@ public class GMSQuorumCheckerJUnitTest {
 
   @Test
   public void testQuorumChecker10Servers2Locators4ServersAnd1LocatorLost() throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
     mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
 
@@ -192,7 +192,7 @@ public class GMSQuorumCheckerJUnitTest {
   @Test
   public void testQuorumChecker10Servers2Locators5ServersAnd2LocatorsButNotLeadMemberLost()
       throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
     mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
 
@@ -225,7 +225,7 @@ public class GMSQuorumCheckerJUnitTest {
   @Test
   public void testQuorumChecker10Servers2Locators5ServerAnd1LocatorWithLeadMemberLost()
       throws Exception {
-    NetView view = prepareView();
+    GMSMembershipView view = prepareView();
     mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
     mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
 
@@ -256,7 +256,7 @@ public class GMSQuorumCheckerJUnitTest {
   @Test
   public void testQuorumChecker2Servers2LocatorsLeadMemberLost() throws Exception {
     int numMembers = 4;
-    NetView view = prepareView(numMembers);
+    GMSMembershipView view = prepareView(numMembers);
     mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
     mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
 
@@ -280,7 +280,7 @@ public class GMSQuorumCheckerJUnitTest {
   @Test
   public void testQuorumChecker2Servers2LocatorsLeadMemberAnd1LocatorLost() throws Exception {
     int numMembers = 4;
-    NetView view = prepareView(numMembers);
+    GMSMembershipView view = prepareView(numMembers);
     mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
     mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
 
@@ -302,19 +302,19 @@ public class GMSQuorumCheckerJUnitTest {
     assertSame(view.getMembers().size(), answerer.getPingCount());
   }
 
-  private NetView prepareView() {
+  private GMSMembershipView prepareView() {
     return prepareView(mockMembers.length);
   }
 
-  private NetView prepareView(int numMembers) {
+  private GMSMembershipView prepareView(int numMembers) {
     int viewId = 1;
-    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    List<GMSMember> mbrs = new LinkedList<>();
     for (int i = 0; i < numMembers; i++) {
       mbrs.add(mockMembers[i]);
     }
 
     // prepare the view
-    NetView netView = new NetView(mockMembers[0], viewId, mbrs);
+    GMSMembershipView netView = new GMSMembershipView(mockMembers[0], viewId, mbrs);
     return netView;
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index a4ac428..602bccb 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -47,14 +47,11 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import org.apache.commons.lang3.SerializationException;
 import org.jgroups.Address;
 import org.jgroups.Event;
 import org.jgroups.JChannel;
@@ -70,27 +67,24 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.ForcedDisconnectException;
 import org.apache.geode.GemFireIOException;
+import org.apache.geode.SerializationException;
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
-import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.SerialAckedMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
+import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Manager;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.MessageHandler;
 import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorRequest;
 import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
+import org.apache.geode.distributed.internal.membership.gms.messages.HeartbeatMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.InstallViewMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.JoinRequestMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.JoinResponseMessage;
@@ -101,9 +95,6 @@ import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
-import org.apache.geode.internal.alerting.AlertingAction;
-import org.apache.geode.internal.cache.DistributedCacheOperation;
-import org.apache.geode.internal.statistics.StatisticsRegistry;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
@@ -144,13 +135,17 @@ public class JGroupsMessengerJUnitTest {
     nonDefault.putAll(addProp);
     DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
     RemoteTransportConfig tconfig =
-        new RemoteTransportConfig(config, ClusterDistributionManager.NORMAL_DM_TYPE);
+        new RemoteTransportConfig(config, GMSMember.NORMAL_DM_TYPE);
 
     stopper = mock(Stopper.class);
     when(stopper.isCancelInProgress()).thenReturn(false);
 
     manager = mock(Manager.class);
     when(manager.isMulticastAllowed()).thenReturn(enableMcast);
+    when(manager.wrapMessage(any(Object.class)))
+        .thenAnswer(invocation -> invocation.getArgument(0));
+    when(manager.unwrapMessage(any(GMSMessage.class)))
+        .thenAnswer(invocation -> invocation.getArgument(0));
 
     healthMonitor = mock(HealthMonitor.class);
 
@@ -165,14 +160,7 @@ public class JGroupsMessengerJUnitTest {
     when(services.getManager()).thenReturn(manager);
     when(services.getJoinLeave()).thenReturn(joinLeave);
 
-    DistributionManager dm = mock(DistributionManager.class);
-    InternalDistributedSystem system =
-        new InternalDistributedSystem.BuilderForTesting(nonDefault)
-            .setDistributionManager(dm)
-            .setStatisticsManagerFactory(
-                (name, startTime, statsDisabled) -> new StatisticsRegistry(name, startTime))
-            .build();
-    when(services.getStatistics()).thenReturn(new DistributionStats(system, statsId));
+    when(services.getStatistics()).thenReturn(mock(DistributionStats.class));
 
     messenger = new JGroupsMessenger();
     messenger.init(services);
@@ -207,64 +195,70 @@ public class JGroupsMessengerJUnitTest {
   public void ioExceptionInitiatesSuspectProcessing() throws Exception {
     // see GEODE-634
     initMocks(false);
-    NetView v = createView();
+    GMSMembershipView v = createView();
     when(joinLeave.getView()).thenReturn(v);
     messenger.installView(v);
     messenger.handleJGroupsIOException(new IOException("je m'en fiche"),
         new JGAddress(v.getMembers().get(1)));
-    verify(healthMonitor).suspect(isA(InternalDistributedMember.class), isA(String.class));
+    verify(healthMonitor).suspect(isA(GMSMember.class), isA(String.class));
   }
 
   @Test
   public void ioExceptionDuringShutdownAvoidsSuspectProcessing() throws Exception {
     // see GEODE-634
     initMocks(false);
-    NetView v = createView();
+    GMSMembershipView v = createView();
     when(joinLeave.getView()).thenReturn(v);
     when(manager.shutdownInProgress()).thenReturn(true);
     messenger.installView(v);
     messenger.handleJGroupsIOException(new IOException("fichez-moi le camp"),
         new JGAddress(v.getMembers().get(1)));
-    verify(healthMonitor, never()).checkIfAvailable(isA(InternalDistributedMember.class),
+    verify(healthMonitor, never()).checkIfAvailable(isA(GMSMember.class),
         isA(String.class), isA(Boolean.class));
   }
 
-  private NetView createView() {
-    InternalDistributedMember sender = messenger.getMemberID();
-    List<InternalDistributedMember> mbrs = new ArrayList<>();
+  private GMSMembershipView createView() {
+    GMSMember sender = messenger.getMemberID();
+    List<GMSMember> mbrs = new ArrayList<>();
     mbrs.add(sender);
     mbrs.add(createAddress(100));
     mbrs.add(createAddress(101));
-    NetView v = new NetView(sender, 1, mbrs);
+    GMSMembershipView v = new GMSMembershipView(sender, 1, mbrs);
     return v;
   }
 
   @Test
-  public void alertMessagesBypassFlowControl() throws Exception {
+  public void normalMessagesUseFlowControl() throws Exception {
     initMocks(false);
     Message jgmsg = new Message();
-    DistributionMessage dmsg = mock(DistributionMessage.class);
-    when(dmsg.getProcessorType()).thenReturn(ClusterDistributionManager.SERIAL_EXECUTOR);
+    GMSMessage dmsg = mock(GMSMessage.class);
+    when(dmsg.isHighPriority()).thenReturn(false);
     messenger.setMessageFlags(dmsg, jgmsg);
-    assertFalse("expected no_fc to not be set in " + jgmsg.getFlags(),
+    assertFalse("expected flow-control to be used: " + jgmsg,
+        jgmsg.isFlagSet(Message.Flag.NO_FC));
+  }
+
+  @Test
+  public void highPriorityMessagesBypassFlowControl() throws Exception {
+    initMocks(false);
+    Message jgmsg = new Message();
+    GMSMessage dmsg = mock(GMSMessage.class);
+    when(dmsg.isHighPriority()).thenReturn(true);
+    messenger.setMessageFlags(dmsg, jgmsg);
+    assertTrue("expected flow-control to not be used: " + jgmsg,
         jgmsg.isFlagSet(Message.Flag.NO_FC));
-    AlertingAction.execute(() -> {
-      messenger.setMessageFlags(dmsg, jgmsg);
-      assertTrue("expected no_fc to be set in " + jgmsg.getFlags(),
-          jgmsg.isFlagSet(Message.Flag.NO_FC));
-    });
   }
 
   @Test
   public void testMemberWeightIsSerialized() throws Exception {
     HeapDataOutputStream out = new HeapDataOutputStream(500, Version.CURRENT);
-    InternalDistributedMember mbr = createAddress(8888);
-    ((GMSMember) mbr.getNetMember()).setMemberWeight((byte) 40);
+    GMSMember mbr = createAddress(8888);
+    mbr.setMemberWeight((byte) 40);
     mbr.toData(out);
     DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.toByteArray()));
-    mbr = new InternalDistributedMember();
+    mbr = new GMSMember();
     mbr.fromData(in);
-    assertEquals(40, mbr.getNetMember().getMemberWeight());
+    assertEquals(40, mbr.getMemberWeight());
   }
 
   @Test
@@ -272,25 +266,17 @@ public class JGroupsMessengerJUnitTest {
     for (int i = 0; i < 2; i++) {
       boolean enableMcast = (i == 1);
       initMocks(enableMcast);
-      InternalDistributedMember mbr = createAddress(8888);
-      DistributedCacheOperation.CacheOperationMessage msg =
-          mock(DistributedCacheOperation.CacheOperationMessage.class);
-      when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+      GMSMember mbr = createAddress(8888);
+      HeartbeatMessage msg =
+          mock(HeartbeatMessage.class);
+      when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
       when(msg.getMulticast()).thenReturn(enableMcast);
-      if (!enableMcast) {
-        // for non-mcast we send a message with a reply-processor
-        when(msg.getProcessorId()).thenReturn(1234);
-      } else {
-        // for mcast we send a direct-ack message and expect the messenger
-        // to register it
-        when(msg.isDirectAck()).thenReturn(true);
-      }
-      when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.PUT_ALL_MESSAGE);
+      when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.HEARTBEAT_RESPONSE);
 
       // for code coverage we need to test with both a SerializationException and
       // an IOException. The former is wrapped in a GemfireIOException while the
       // latter is not
-      doThrow(new SerializationException()).when(msg).toData(any(DataOutput.class));
+      doThrow(new SerializationException("")).when(msg).toData(any(DataOutput.class));
       try {
         messenger.send(msg);
         fail("expected a failure");
@@ -321,13 +307,11 @@ public class JGroupsMessengerJUnitTest {
       JChannel realChannel = messenger.myChannel;
       messenger.myChannel = mockChannel;
       try {
-        InternalDistributedMember mbr = createAddress(8888);
-        DistributedCacheOperation.CacheOperationMessage msg =
-            mock(DistributedCacheOperation.CacheOperationMessage.class);
-        when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+        GMSMember mbr = createAddress(8888);
+        HeartbeatMessage msg = mock(HeartbeatMessage.class);
+        when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
         when(msg.getMulticast()).thenReturn(enableMcast);
-        when(msg.getProcessorId()).thenReturn(1234);
-        when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.PUT_ALL_MESSAGE);
+        when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.HEARTBEAT_RESPONSE);
         try {
           messenger.send(msg);
           fail("expected a failure");
@@ -364,13 +348,11 @@ public class JGroupsMessengerJUnitTest {
       when(services.getShutdownCause()).thenReturn(shutdownCause);
 
       try {
-        InternalDistributedMember mbr = createAddress(8888);
-        DistributedCacheOperation.CacheOperationMessage msg =
-            mock(DistributedCacheOperation.CacheOperationMessage.class);
-        when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+        GMSMember mbr = createAddress(8888);
+        HeartbeatMessage msg = mock(HeartbeatMessage.class);
+        when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
         when(msg.getMulticast()).thenReturn(enableMcast);
-        when(msg.getProcessorId()).thenReturn(1234);
-        when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.PUT_ALL_MESSAGE);
+        when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.HEARTBEAT_RESPONSE);
         try {
           messenger.send(msg);
           fail("expected a failure");
@@ -401,12 +383,11 @@ public class JGroupsMessengerJUnitTest {
       JChannel realChannel = messenger.myChannel;
       messenger.myChannel = mockChannel;
       try {
-        InternalDistributedMember mbr = createAddress(8888);
-        DistributedCacheOperation.CacheOperationMessage msg =
-            mock(DistributedCacheOperation.CacheOperationMessage.class);
-        when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+        GMSMember mbr = createAddress(8888);
+        HeartbeatMessage msg =
+            mock(HeartbeatMessage.class);
+        when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
         when(msg.getMulticast()).thenReturn(false);
-        when(msg.getProcessorId()).thenReturn(1234);
         try {
           messenger.send(msg);
           fail("expected a failure");
@@ -425,20 +406,12 @@ public class JGroupsMessengerJUnitTest {
     for (int i = 0; i < 2; i++) {
       boolean enableMcast = (i == 1);
       initMocks(enableMcast);
-      InternalDistributedMember mbr = createAddress(8888);
-      DistributedCacheOperation.CacheOperationMessage msg =
-          mock(DistributedCacheOperation.CacheOperationMessage.class);
-      when(msg.getRecipients()).thenReturn(new InternalDistributedMember[] {mbr});
+      GMSMember mbr = createAddress(8888);
+      HeartbeatMessage msg =
+          mock(HeartbeatMessage.class);
+      when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
       when(msg.getMulticast()).thenReturn(enableMcast);
-      if (!enableMcast) {
-        // for non-mcast we send a message with a reply-processor
-        when(msg.getProcessorId()).thenReturn(1234);
-      } else {
-        // for mcast we send a direct-ack message and expect the messenger
-        // to register it
-        when(msg.isDirectAck()).thenReturn(true);
-      }
-      when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.PUT_ALL_MESSAGE);
+      when(msg.getDSFID()).thenReturn((int) DataSerializableFixedID.HEARTBEAT_RESPONSE);
       interceptor.collectMessages = true;
       try {
         messenger.sendUnreliably(msg);
@@ -470,23 +443,25 @@ public class JGroupsMessengerJUnitTest {
     MessageHandler mh = mock(MessageHandler.class);
     messenger.addHandler(JoinRequestMessage.class, mh);
 
-    InternalDistributedMember addr = messenger.getMemberID();
-    NetView v = new NetView(addr);
+    GMSMember addr = messenger.getMemberID();
+    GMSMembershipView v = new GMSMembershipView(addr);
     when(joinLeave.getView()).thenReturn(v);
 
 
-    InternalDistributedMember sender = createAddress(8888);
+    GMSMember sender = createAddress(8888);
 
     JoinRequestMessage msg = new JoinRequestMessage(messenger.localAddress, sender, null, -1, 0);
 
-    Message jmsg = messenger.createJGMessage(msg, messenger.jgAddress, Version.CURRENT_ORDINAL);
+    Message jmsg = messenger.createJGMessage(msg, messenger.jgAddress, messenger.localAddress,
+        Version.CURRENT_ORDINAL);
     interceptor.up(new Event(Event.MSG, jmsg));
 
     verify(mh, times(1)).processMessage(any(JoinRequestMessage.class));
 
     LeaveRequestMessage lmsg = new LeaveRequestMessage(messenger.localAddress, sender, "testing");
     when(joinLeave.getMemberID(any())).thenReturn(sender);
-    jmsg = messenger.createJGMessage(lmsg, messenger.jgAddress, Version.CURRENT_ORDINAL);
+    jmsg = messenger.createJGMessage(lmsg, messenger.jgAddress, messenger.localAddress,
+        Version.CURRENT_ORDINAL);
     interceptor.up(new Event(Event.MSG, jmsg));
 
     verify(manager).processMessage(any(LeaveRequestMessage.class));
@@ -495,91 +470,96 @@ public class JGroupsMessengerJUnitTest {
 
 
   @Test
-  public void testBigMessageIsFragmented() throws Exception {
+  public void testBigMessageIsFragmentedWhenSentPointToPoint() throws Exception {
     doTestBigMessageIsFragmented(false, false);
   }
 
   @Test
-  public void testBigMessageIsFragmentedMcast() throws Exception {
+  public void testBigMessageIsFragmentedWhenMulticast() throws Exception {
     doTestBigMessageIsFragmented(true, true);
   }
 
   @Test
-  public void testBroadcastUDPMessage() throws Exception {
+  public void testBigMessageIsFragmentedWhenBroadcast() throws Exception {
     doTestBigMessageIsFragmented(false, true);
   }
 
-  public void doTestBigMessageIsFragmented(boolean mcastEnabled, boolean mcastMsg)
+  public void doTestBigMessageIsFragmented(boolean mcastEnabled, boolean broadcastMessage)
       throws Exception {
     initMocks(mcastEnabled);
     MessageHandler mh = mock(MessageHandler.class);
     messenger.addHandler(JoinRequestMessage.class, mh);
 
-    InternalDistributedMember sender = messenger.getMemberID();
-    NetView v = new NetView(sender);
+    GMSMember sender = messenger.getMemberID();
+    GMSMembershipView v = new GMSMembershipView(sender);
     when(joinLeave.getView()).thenReturn(v);
     messenger.installView(v);
-    JoinRequestMessage msg = new JoinRequestMessage(messenger.localAddress, sender, null, -1, 0);
-    if (mcastMsg) {
-      msg.setMulticast(true);
-    }
-
-    messenger.send(msg);
-    int sentMessages = (mcastEnabled && mcastMsg) ? interceptor.mcastSentDataMessages
-        : interceptor.unicastSentDataMessages;
-    assertTrue("expected 1 message to be sent but found " + sentMessages, sentMessages == 1);
 
     // send a big message and expect fragmentation
-    msg = new JoinRequestMessage(messenger.localAddress, sender,
+    GMSMember recipient = broadcastMessage ? null : messenger.localAddress;
+    JoinRequestMessage msg = new JoinRequestMessage(recipient, sender,
         new byte[(int) (services.getConfig().getDistributionConfig().getUdpFragmentSize() * (1.5))],
         -1, 0);
 
+    msg.setMulticast(broadcastMessage);
+
     // configure an incoming message handler for JoinRequestMessage
     final JoinRequestMessage[] messageReceived = new JoinRequestMessage[1];
     messenger.addHandler(JoinRequestMessage.class, message -> messageReceived[0] = message);
 
     // configure the outgoing message interceptor
+    interceptor.mcastSentDataMessages = 0;
     interceptor.unicastSentDataMessages = 0;
     interceptor.collectMessages = true;
     interceptor.collectedMessages.clear();
 
     messenger.send(msg);
 
-    assertTrue("expected 2 messages to be sent but found " + interceptor.unicastSentDataMessages,
-        interceptor.unicastSentDataMessages == 2);
+    boolean jgroupsWillUseMulticast = mcastEnabled && broadcastMessage;
+    if (jgroupsWillUseMulticast) {
+      assertTrue(
+          "expected 2 messages to be broadcast but found " + interceptor.mcastSentDataMessages,
+          interceptor.mcastSentDataMessages == 2);
+    } else {
+      assertTrue("expected 2 messages to be sent but found " + interceptor.unicastSentDataMessages,
+          interceptor.unicastSentDataMessages == 2);
+    }
 
     List<Message> messages = new ArrayList<>(interceptor.collectedMessages);
     UUID fakeMember = new UUID(50, 50);
     short unicastHeaderId = ClassConfigurator.getProtocolId(UNICAST3.class);
     int seqno = 1;
     for (Message m : messages) {
-      m.setSrc(fakeMember);
-      UNICAST3.Header oldHeader = (UNICAST3.Header) m.getHeader(unicastHeaderId);
-      if (oldHeader == null)
-        continue;
-      UNICAST3.Header newHeader =
-          UNICAST3.Header.createDataHeader(seqno, oldHeader.connId(), seqno == 1);
-      seqno += 1;
-      m.putHeader(unicastHeaderId, newHeader);
+      if (jgroupsWillUseMulticast) {
+        m.setSrc(messenger.localAddress.getUUID());
+      } else {
+        m.setSrc(fakeMember);
+        UNICAST3.Header oldHeader = (UNICAST3.Header) m.getHeader(unicastHeaderId);
+        if (oldHeader == null)
+          continue;
+        UNICAST3.Header newHeader =
+            UNICAST3.Header.createDataHeader(seqno, oldHeader.connId(), seqno == 1);
+        seqno += 1;
+        m.putHeader(unicastHeaderId, newHeader);
+      }
       interceptor.up(new Event(Event.MSG, m));
     }
-    Thread.sleep(5000);
-    System.out.println("received message = " + messageReceived[0]);
+    assertNotNull(messageReceived[0]);
   }
 
   @Test
   public void testSendToMultipleMembers() throws Exception {
     initMocks(false);
-    InternalDistributedMember sender = messenger.getMemberID();
-    InternalDistributedMember other = createAddress(8888);
+    GMSMember sender = messenger.getMemberID();
+    GMSMember other = createAddress(8888);
 
-    NetView v = new NetView(sender);
+    GMSMembershipView v = new GMSMembershipView(sender);
     v.add(other);
     when(joinLeave.getView()).thenReturn(v);
     messenger.installView(v);
 
-    List<InternalDistributedMember> recipients = v.getMembers();
-    SerialAckedMessage msg = new SerialAckedMessage();
+    List<GMSMember> recipients = v.getMembers();
+    HeartbeatMessage msg = new HeartbeatMessage();
     msg.setRecipients(recipients);
 
     messenger.send(msg);
@@ -744,8 +724,8 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testMessageFiltering() throws Exception {
     initMocks(true);
-    InternalDistributedMember mbr = createAddress(8888);
-    NetView view = new NetView(mbr);
+    GMSMember mbr = createAddress(8888);
+    GMSMembershipView view = new GMSMembershipView(mbr);
 
     // the digest should be set in an outgoing join response
     JoinResponseMessage joinResponse = new JoinResponseMessage(mbr, view, 0);
@@ -775,7 +755,7 @@ public class JGroupsMessengerJUnitTest {
   public void testPingPong() throws Exception {
     initMocks(false);
     GMSPingPonger pinger = messenger.pingPonger;
-    InternalDistributedMember mbr = createAddress(8888);
+    GMSMember mbr = createAddress(8888);
     JGAddress addr = new JGAddress(mbr);
 
     Message pingMessage = pinger.createPingMessage(null, addr);
@@ -815,8 +795,8 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testJGroupsIOExceptionHandler() throws Exception {
     initMocks(false);
-    InternalDistributedMember mbr = createAddress(8888);
-    NetView v = new NetView(mbr);
+    GMSMember mbr = createAddress(8888);
+    GMSMembershipView v = new GMSMembershipView(mbr);
     v.add(messenger.getMemberID());
     messenger.installView(v);
 
@@ -828,48 +808,44 @@ public class JGroupsMessengerJUnitTest {
 
   @Test
   public void testReceiver() throws Exception {
-    try {
-      DistributionStats.enableClockStats = true;
-      initMocks(false);
-      JGroupsReceiver receiver = (JGroupsReceiver) messenger.myChannel.getReceiver();
-
-      // a zero-length message is ignored
-      Message msg = new Message(new JGAddress(messenger.getMemberID()));
-      Object result = messenger.readJGMessage(msg);
-      assertNull(result);
-
-      // for code coverage we need to pump this message through the receiver
-      receiver.receive(msg);
-
-      // for more code coverage we need to actually set a buffer in the message
-      msg.setBuffer(new byte[0]);
-      result = messenger.readJGMessage(msg);
-      assertNull(result);
-      receiver.receive(msg);
-
-      // now create a view and a real distribution-message
-      InternalDistributedMember myAddress = messenger.getMemberID();
-      InternalDistributedMember other = createAddress(8888);
-      NetView v = new NetView(myAddress);
-      v.add(other);
-      when(joinLeave.getView()).thenReturn(v);
-      messenger.installView(v);
-
-      List<InternalDistributedMember> recipients = v.getMembers();
-      SerialAckedMessage dmsg = new SerialAckedMessage();
-      dmsg.setRecipients(recipients);
-
-      // a message is ignored during manager shutdown
-      msg = messenger.createJGMessage(dmsg, new JGAddress(other), Version.CURRENT_ORDINAL);
-      when(manager.shutdownInProgress()).thenReturn(Boolean.TRUE);
-      receiver.receive(msg);
-      verify(manager, never()).processMessage(isA(DistributionMessage.class));
-
-      assertTrue("There should be UDPDispatchRequestTime stats",
-          services.getStatistics().getUDPDispatchRequestTime() > 0);
-    } finally {
-      DistributionStats.enableClockStats = false;
-    }
+    initMocks(false);
+    JGroupsReceiver receiver = (JGroupsReceiver) messenger.myChannel.getReceiver();
+    messenger.addHandler(HeartbeatMessage.class, message -> {
+    });
+
+    // a zero-length message is ignored
+    Message msg = new Message(new JGAddress(messenger.getMemberID()));
+    Object result = messenger.readJGMessage(msg);
+    assertNull(result);
+
+    // for code coverage we need to pump this message through the receiver
+    receiver.receive(msg);
+
+    // for more code coverage we need to actually set a buffer in the message
+    msg.setBuffer(new byte[0]);
+    result = messenger.readJGMessage(msg);
+    assertNull(result);
+    receiver.receive(msg);
+
+    // now create a view and a real distribution-message
+    GMSMember myAddress = messenger.getMemberID();
+    GMSMember other = createAddress(8888);
+    GMSMembershipView v = new GMSMembershipView(myAddress);
+    v.add(other);
+    when(joinLeave.getView()).thenReturn(v);
+    messenger.installView(v);
+
+    List<GMSMember> recipients = v.getMembers();
+    HeartbeatMessage dmsg = new HeartbeatMessage();
+    dmsg.setRecipients(recipients);
+
+    // a message is ignored during manager shutdown
+    msg = messenger.createJGMessage(dmsg, new JGAddress(other),
+        recipients.get(0), Version.CURRENT_ORDINAL);
+    when(manager.shutdownInProgress()).thenReturn(Boolean.TRUE);
+    receiver.receive(msg);
+    verify(manager, never()).processMessage(isA(GMSMessage.class));
+    verify(services.getStatistics(), times(3)).incUDPDispatchRequestTime(isA(Long.class));
   }
 
   @Test
@@ -877,7 +853,7 @@ public class JGroupsMessengerJUnitTest {
     initMocks(false);
     JChannel channel = messenger.myChannel;
     services.getConfig().getTransport().setOldDSMembershipInfo(new MembershipInformation(channel,
-        Collections.singleton(new InternalDistributedMember("localhost", 10000)),
+        Collections.singleton(new GMSMember("localhost", 10000)),
         new ConcurrentLinkedQueue<>()));
     JGroupsMessenger newMessenger = new JGroupsMessenger();
     newMessenger.init(services);
@@ -915,7 +891,7 @@ public class JGroupsMessengerJUnitTest {
   public void testWaitForMessageStateSucceeds() throws Exception {
     initMocks(true/* multicast */);
     JGroupsMessenger.MessageTracker tracker = mock(JGroupsMessenger.MessageTracker.class);
-    InternalDistributedMember mbr = createAddress(1234);
+    GMSMember mbr = createAddress(1234);
     messenger.scheduledMcastSeqnos.put(mbr, tracker);
     when(tracker.get()).thenReturn(0l, 2l, 49l, 50l, 80l);
     Map state = new HashMap();
@@ -941,7 +917,7 @@ public class JGroupsMessengerJUnitTest {
       // message 50 will never arrive
       Map state = new HashMap();
       state.put("JGroups.mcastState", Long.valueOf(50));
-      InternalDistributedMember mbr = createAddress(1234);
+      GMSMember mbr = createAddress(1234);
       messenger.scheduledMcastSeqnos.put(mbr, new JGroupsMessenger.MessageTracker(30));
       messenger.waitForMessageState(mbr, state);
       fail("expected a GemFireIOException to be thrown");
@@ -950,26 +926,26 @@ public class JGroupsMessengerJUnitTest {
     }
   }
 
-  private NetView createView(InternalDistributedMember otherMbr) {
-    InternalDistributedMember sender = messenger.getMemberID();
-    List<InternalDistributedMember> mbrs = new ArrayList<>();
+  private GMSMembershipView createView(GMSMember otherMbr) {
+    GMSMember sender = messenger.getMemberID();
+    List<GMSMember> mbrs = new ArrayList<>();
     mbrs.add(sender);
     mbrs.add(otherMbr);
-    NetView v = new NetView(sender, 1, mbrs);
+    GMSMembershipView v = new GMSMembershipView(sender, 1, mbrs);
     return v;
   }
 
   @Test
   public void testEncryptedFindCoordinatorRequest() throws Exception {
-    InternalDistributedMember otherMbr = new InternalDistributedMember("localhost", 8888);
+    GMSMember otherMbr = new GMSMember("localhost", 8888);
 
     Properties p = new Properties();
     final String udpDhalgo = "AES:128";
     p.put(ConfigurationProperties.SECURITY_UDP_DHALGO, udpDhalgo);
     initMocks(false, p);
 
-    NetView v = createView(otherMbr);
-    when(joinLeave.getMemberID(messenger.getMemberID().getNetMember()))
+    GMSMembershipView v = createView(otherMbr);
+    when(joinLeave.getMemberID(messenger.getMemberID()))
         .thenReturn(messenger.getMemberID());
     GMSEncrypt otherMbrEncrptor = new GMSEncrypt(services, udpDhalgo);
 
@@ -977,9 +953,9 @@ public class JGroupsMessengerJUnitTest {
     messenger.initClusterKey();
 
     FindCoordinatorRequest gfmsg = new FindCoordinatorRequest(messenger.getMemberID(),
-        new ArrayList<InternalDistributedMember>(2), 1,
+        new ArrayList<GMSMember>(2), 1,
         messenger.getPublicKey(messenger.getMemberID()), 1, "");
-    Set<InternalDistributedMember> recipients = new HashSet<>();
+    List<GMSMember> recipients = new ArrayList<>();
     recipients.add(otherMbr);
     gfmsg.setRecipients(recipients);
 
@@ -987,13 +963,13 @@ public class JGroupsMessengerJUnitTest {
 
     HeapDataOutputStream out = new HeapDataOutputStream(Version.CURRENT);
 
-    messenger.writeEncryptedMessage(gfmsg, version, out);
+    messenger.writeEncryptedMessage(gfmsg, otherMbr, version, out);
 
     byte[] requestBytes = out.toByteArray();
 
     DataInputStream dis = new DataInputStream(new ByteArrayInputStream(requestBytes));
 
-    DistributionMessage distributionMessage =
+    GMSMessage distributionMessage =
         messenger.readEncryptedMessage(dis, version, otherMbrEncrptor);
 
     assertEquals(gfmsg, distributionMessage);
@@ -1001,14 +977,14 @@ public class JGroupsMessengerJUnitTest {
 
   @Test
   public void testEncryptedFindCoordinatorResponse() throws Exception {
-    InternalDistributedMember otherMbr = new InternalDistributedMember("localhost", 8888);
+    GMSMember otherMbr = new GMSMember("localhost", 8888);
 
     Properties p = new Properties();
 
     p.put(ConfigurationProperties.SECURITY_UDP_DHALGO, AES_128);
     initMocks(false, p);
 
-    NetView v = createView(otherMbr);
+    GMSMembershipView v = createView(otherMbr);
 
     GMSEncrypt otherMbrEncrptor = new GMSEncrypt(services, AES_128);
     otherMbrEncrptor.setPublicKey(messenger.getPublicKey(messenger.getMemberID()),
@@ -1019,7 +995,7 @@ public class JGroupsMessengerJUnitTest {
 
     FindCoordinatorResponse gfmsg = new FindCoordinatorResponse(messenger.getMemberID(),
         messenger.getMemberID(), messenger.getClusterSecretKey(), 1);
-    Set<InternalDistributedMember> recipients = new HashSet<>();
+    List<GMSMember> recipients = new ArrayList<>();
     recipients.add(otherMbr);
     gfmsg.setRecipients(recipients);
 
@@ -1027,7 +1003,7 @@ public class JGroupsMessengerJUnitTest {
 
     HeapDataOutputStream out = new HeapDataOutputStream(Version.CURRENT);
 
-    messenger.writeEncryptedMessage(gfmsg, version, out);
+    messenger.writeEncryptedMessage(gfmsg, otherMbr, version, out);
 
     byte[] requestBytes = out.toByteArray();
 
@@ -1035,7 +1011,7 @@ public class JGroupsMessengerJUnitTest {
 
     messenger.addRequestId(1, messenger.getMemberID());
 
-    DistributionMessage distributionMessage =
+    GMSMessage distributionMessage =
         messenger.readEncryptedMessage(dis, version, otherMbrEncrptor);
 
     assertEquals(gfmsg, distributionMessage);
@@ -1043,13 +1019,13 @@ public class JGroupsMessengerJUnitTest {
 
   @Test
   public void testEncryptedJoinRequest() throws Exception {
-    InternalDistributedMember otherMbr = new InternalDistributedMember("localhost", 8888);
+    GMSMember otherMbr = new GMSMember("localhost", 8888);
 
     Properties p = new Properties();
     p.put(ConfigurationProperties.SECURITY_UDP_DHALGO, AES_128);
     initMocks(false, p);
 
-    NetView v = createView(otherMbr);
+    GMSMembershipView v = createView(otherMbr);
 
     GMSEncrypt otherMbrEncrptor = new GMSEncrypt(services, AES_128);
 
@@ -1063,13 +1039,13 @@ public class JGroupsMessengerJUnitTest {
 
     HeapDataOutputStream out = new HeapDataOutputStream(Version.CURRENT);
 
-    messenger.writeEncryptedMessage(gfmsg, version, out);
+    messenger.writeEncryptedMessage(gfmsg, otherMbr, version, out);
 
     byte[] requestBytes = out.toByteArray();
 
     DataInputStream dis = new DataInputStream(new ByteArrayInputStream(requestBytes));
 
-    DistributionMessage distributionMessage =
+    GMSMessage distributionMessage =
         messenger.readEncryptedMessage(dis, version, otherMbrEncrptor);
 
     assertEquals(gfmsg, distributionMessage);
@@ -1077,13 +1053,13 @@ public class JGroupsMessengerJUnitTest {
 
   @Test
   public void testEncryptedJoinResponse() throws Exception {
-    InternalDistributedMember otherMbr = new InternalDistributedMember("localhost", 8888);
+    GMSMember otherMbr = new GMSMember("localhost", 8888);
 
     Properties p = new Properties();
     p.put(ConfigurationProperties.SECURITY_UDP_DHALGO, AES_128);
     initMocks(false, p);
 
-    NetView v = createView(otherMbr);
+    GMSMembershipView v = createView(otherMbr);
 
     GMSEncrypt otherMbrEncrptor = new GMSEncrypt(services, AES_128);
     otherMbrEncrptor.setPublicKey(messenger.getPublicKey(messenger.getMemberID()),
@@ -1099,7 +1075,7 @@ public class JGroupsMessengerJUnitTest {
 
     HeapDataOutputStream out = new HeapDataOutputStream(Version.CURRENT);
 
-    messenger.writeEncryptedMessage(gfmsg, version, out);
+    messenger.writeEncryptedMessage(gfmsg, otherMbr, version, out);
 
     byte[] requestBytes = out.toByteArray();
 
@@ -1107,7 +1083,7 @@ public class JGroupsMessengerJUnitTest {
 
     messenger.addRequestId(1, messenger.getMemberID());
 
-    DistributionMessage gfMessageAtOtherMbr =
+    GMSMessage gfMessageAtOtherMbr =
         messenger.readEncryptedMessage(dis, version, otherMbrEncrptor);
 
     assertEquals(gfmsg, gfMessageAtOtherMbr);
@@ -1118,11 +1094,12 @@ public class JGroupsMessengerJUnitTest {
 
     out = new HeapDataOutputStream(Version.CURRENT);
 
-    messenger.writeEncryptedMessage(installViewMessage, version, out);
+    messenger.writeEncryptedMessage(installViewMessage, otherMbr, version, out);
 
     requestBytes = out.toByteArray();
 
-    otherMbrEncrptor.setClusterKey(((JoinResponseMessage) gfMessageAtOtherMbr).getSecretPk());
+    JoinResponseMessage joinResponseMessage = (JoinResponseMessage) gfMessageAtOtherMbr;
+    otherMbrEncrptor.setClusterKey(joinResponseMessage.getSecretPk());
 
     dis = new DataInputStream(new ByteArrayInputStream(requestBytes));
 
@@ -1132,18 +1109,11 @@ public class JGroupsMessengerJUnitTest {
 
   }
 
-  /**
-   * creates an InternalDistributedMember address that can be used with the doctored JGroups
-   * channel. This includes a logical (UUID) address and a physical (IpAddress) address.
-   *
-   * @param port the UDP port to use for the new address
-   */
-  private InternalDistributedMember createAddress(int port) {
+  private GMSMember createAddress(int port) {
     GMSMember gms = new GMSMember("localhost", port);
     gms.setUUID(UUID.randomUUID());
-    gms.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+    gms.setVmKind(GMSMember.NORMAL_DM_TYPE);
     gms.setVersionOrdinal(Version.CURRENT_ORDINAL);
-    return new InternalDistributedMember(gms);
+    return gms;
   }
-
 }
diff --git a/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt b/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
index b5a2761..5a8deb1 100644
--- a/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
+++ b/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
@@ -10,7 +10,7 @@ org/apache/geode/distributed/internal/DistributionConfigImpl
 org/apache/geode/distributed/internal/OverflowQueueWithDMStats
 org/apache/geode/distributed/internal/ThrottlingMemLinkedQueueWithDMStats
 org/apache/geode/distributed/internal/membership/gms/messenger/JGAddress
-org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager$BoundedLinkedHashMap
+org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager$BoundedLinkedHashMap
 org/apache/geode/distributed/internal/tcpserver/LocatorCancelException
 org/apache/geode/internal/AbstractConfig$SortedProperties
 org/apache/geode/internal/AvailablePort$Keeper
diff --git a/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
index 8acb88d..9212da8 100644
--- a/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/integrationTest/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
@@ -272,26 +272,30 @@ toData,16
 
 org/apache/geode/distributed/internal/membership/InternalDistributedMember,6
 fromData,38
-fromDataPre_GFE_7_1_0_0,281
-fromDataPre_GFE_9_0_0_0,281
+fromDataPre_GFE_7_1_0_0,277
+fromDataPre_GFE_9_0_0_0,277
 toData,34
-toDataPre_GFE_7_1_0_0,271
-toDataPre_GFE_9_0_0_0,266
+toDataPre_GFE_7_1_0_0,276
+toDataPre_GFE_9_0_0_0,271
 
-org/apache/geode/distributed/internal/membership/NetView,2
-fromData,109
-toData,60
+org/apache/geode/distributed/internal/membership/adapter/LocalViewMessage,2
+fromData,8
+toData,8
 
 org/apache/geode/distributed/internal/membership/gms/GMSMember,2
-fromData,62
-toData,62
+fromData,196
+toData,213
+
+org/apache/geode/distributed/internal/membership/gms/GMSMembershipView,2
+fromData,106
+toData,60
 
 org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest,2
-fromData,97
+fromData,91
 toData,114
 
 org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse,2
-fromData,105
+fromData,99
 toData,89
 
 org/apache/geode/distributed/internal/membership/gms/locator/GetViewRequest,2
@@ -303,7 +307,7 @@ fromData,12
 toData,9
 
 org/apache/geode/distributed/internal/membership/gms/messages/FinalCheckPassedMessage,2
-fromData,12
+fromData,9
 toData,9
 
 org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatMessage,2
@@ -311,40 +315,40 @@ fromData,11
 toData,11
 
 org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatRequestMessage,2
-fromData,22
+fromData,19
 toData,19
 
 org/apache/geode/distributed/internal/membership/gms/messages/InstallViewMessage,2
-fromData,49
-toData,45
+fromData,44
+toData,40
 
 org/apache/geode/distributed/internal/membership/gms/messages/JoinRequestMessage,2
-fromData,48
-toData,45
+fromData,42
+toData,42
 
 org/apache/geode/distributed/internal/membership/gms/messages/JoinResponseMessage,2
-fromData,47
+fromData,44
 toData,41
 
 org/apache/geode/distributed/internal/membership/gms/messages/LeaveRequestMessage,2
-fromData,20
+fromData,17
 toData,17
 
+org/apache/geode/distributed/internal/membership/gms/messages/NetworkPartitionMessage,2
+fromData,1
+toData,1
+
 org/apache/geode/distributed/internal/membership/gms/messages/RemoveMemberMessage,2
-fromData,20
+fromData,17
 toData,17
 
 org/apache/geode/distributed/internal/membership/gms/messages/SuspectMembersMessage,2
-fromData,53
+fromData,50
 toData,81
 
 org/apache/geode/distributed/internal/membership/gms/messages/ViewAckMessage,2
-fromData,37
-toData,34
-
-org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage,2
-fromData,8
-toData,8
+fromData,32
+toData,29
 
 org/apache/geode/distributed/internal/streaming/StreamingOperation$RequestStreamingMessage,2
 fromData,16
@@ -496,11 +500,11 @@ toData,16
 
 org/apache/geode/internal/admin/remote/ClientHealthStats,6
 fromData,73
-fromDataPre_GFE_8_0_0_0,68
 fromDataPre_GEODE_1_9_0_0,14
+fromDataPre_GFE_8_0_0_0,68
 toData,73
-toDataPre_GFE_8_0_0_0,68
 toDataPre_GEODE_1_9_0_0,14
+toDataPre_GFE_8_0_0_0,68
 
 org/apache/geode/internal/admin/remote/CompactRequest,2
 fromData,6
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
index e64d89d..3e56bf1 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
@@ -66,7 +66,7 @@ import org.apache.geode.distributed.internal.membership.DistributedMembershipLis
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MemberFactory;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.MembershipView;
 import org.apache.geode.distributed.internal.membership.adapter.auth.GMSAuthenticator;
 import org.apache.geode.distributed.internal.membership.gms.messages.ViewAckMessage;
 import org.apache.geode.internal.Assert;
@@ -1107,7 +1107,7 @@ public class ClusterDistributionManager implements DistributionManager {
     try {
 
       // And the distinguished guests today are...
-      NetView v = membershipManager.getView();
+      MembershipView v = membershipManager.getView();
       logger.info("Initial (distribution manager) view, {}",
           String.valueOf(v));
 
@@ -1514,11 +1514,6 @@ public class ClusterDistributionManager implements DistributionManager {
   }
 
   @Override
-  public long getMembershipPort() {
-    return localAddress.getPort();
-  }
-
-  @Override
   public Set<InternalDistributedMember> putOutgoing(final DistributionMessage msg) {
     try {
       DistributionMessageObserver observer = DistributionMessageObserver.getInstance();
@@ -2687,7 +2682,7 @@ public class ClusterDistributionManager implements DistributionManager {
     addMemberEvent(new MemberSuspectEvent(suspect, whoSuspected, reason));
   }
 
-  private void handleViewInstalled(NetView view) {
+  private void handleViewInstalled(MembershipView view) {
     addMemberEvent(new ViewInstalledEvent(view));
   }
 
@@ -3394,12 +3389,12 @@ public class ClusterDistributionManager implements DistributionManager {
     }
 
     @Override
-    public void newMemberConnected(InternalDistributedMember member) {
+    public void newMemberConnected(DistributedMember member) {
       // Do not elect the elder here as surprise members invoke this callback
       // without holding the view lock. That can cause a race condition and
       // subsequent deadlock (#45566). Elder selection is now done when a view
       // is installed.
-      dm.addNewMember(member);
+      dm.addNewMember((InternalDistributedMember) member);
     }
 
     @Override
@@ -3430,7 +3425,7 @@ public class ClusterDistributionManager implements DistributionManager {
     }
 
     @Override
-    public void viewInstalled(NetView view) {
+    public void viewInstalled(MembershipView view) {
       dm.handleViewInstalled(view);
     }
 
@@ -3612,9 +3607,9 @@ public class ClusterDistributionManager implements DistributionManager {
   }
 
   private static class ViewInstalledEvent extends MemberEvent {
-    NetView view;
+    MembershipView view;
 
-    ViewInstalledEvent(NetView view) {
+    ViewInstalledEvent(MembershipView view) {
       super(null);
       this.view = view;
     }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
index 2e52900..57f5243 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
@@ -169,13 +169,6 @@ public interface DistributionManager extends ReplySender {
   ElderState getElderState(boolean force) throws InterruptedException;
 
   /**
-   * Returns the membership port of the underlying distribution manager used for communication.
-   *
-   * @since GemFire 3.0
-   */
-  long getMembershipPort();
-
-  /**
    * Sends a message
    *
    * @return recipients who did not receive the message
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java
index 825618a..21151a0 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java
@@ -259,6 +259,7 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
     this.multicast = false;
   }
 
+
   /**
    * Returns the intended recipient(s) of this message. If the message is intended to delivered to
    * all distribution managers, then the array will contain ALL_RECIPIENTS. If the recipients have
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 7564870..96ae52d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -73,8 +73,8 @@ import org.apache.geode.distributed.internal.locks.GrantorRequestProcessor;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.distributed.internal.membership.QuorumChecker;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.distributed.internal.membership.gms.messenger.MembershipInformation;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.DSFIDFactory;
 import org.apache.geode.internal.InternalDataSerializer;
@@ -773,14 +773,8 @@ public class InternalDistributedSystem extends DistributedSystem
       Assert.assertTrue(dm != null);
       Assert.assertTrue(dm.getSystem() == this);
 
-      try {
-        id = dm.getMembershipPort();
-      } catch (DistributedSystemDisconnectedException e) {
-        // bug #48144 - The dm's channel threw an NPE. It now throws this exception
-        // but during startup we should instead throw a SystemConnectException
-        throw new SystemConnectException(
-            "Distributed system has disconnected during startup.",
-            e);
+      if (dm != null && dm.getMembershipManager() != null) {
+        id = dm.getMembershipManager().getLocalMember().getPort();
       }
 
       synchronized (isConnectedMutex) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 5a2b21c..16f04cf 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -58,8 +58,9 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.InternalDistributedSystem.ConnectListener;
 import org.apache.geode.distributed.internal.membership.MemberFactory;
+import org.apache.geode.distributed.internal.membership.NetLocator;
 import org.apache.geode.distributed.internal.membership.QuorumChecker;
-import org.apache.geode.distributed.internal.membership.gms.NetLocator;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
 import org.apache.geode.distributed.internal.membership.gms.locator.PeerLocatorRequest;
 import org.apache.geode.distributed.internal.tcpserver.LocatorCancelException;
 import org.apache.geode.distributed.internal.tcpserver.TcpClient;
@@ -700,7 +701,9 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
           (InternalDistributedSystem) DistributedSystem.connect(distributedSystemProperties);
 
       if (peerLocator) {
-        netLocator.setMembershipManager(internalDistributedSystem.getDM().getMembershipManager());
+        netLocator.setServices(
+            ((GMSMembershipManager) internalDistributedSystem.getDM().getMembershipManager())
+                .getServices());
       }
 
       internalDistributedSystem.addDisconnectListener(sys -> stop(false, false, false));
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
index 248ff96..7dbda8d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
@@ -257,11 +257,6 @@ public class LonerDistributionManager implements DistributionManager {
   }
 
   @Override
-  public long getMembershipPort() {
-    return 0;
-  }
-
-  @Override
   public InternalDistributedSystem getSystem() {
     return this.system;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java
index 03ef3b4..486d102 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java
@@ -29,6 +29,7 @@ import org.apache.geode.InternalGemFireException;
 import org.apache.geode.annotations.internal.MakeNotStatic;
 import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.UnsupportedVersionException;
+import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.internal.deadlock.MessageDependencyMonitor;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -1118,7 +1119,8 @@ public class ReplyProcessor21 implements MembershipListener {
           cause);
     } else if (suspectThem) {
       if (suspectMembers != null && suspectMembers.size() > 0) {
-        getDistributionManager().getMembershipManager().suspectMembers(suspectMembers,
+        getDistributionManager().getMembershipManager().suspectMembers(
+            (Set<DistributedMember>) (Set<?>) suspectMembers,
             "Failed to respond within ack-wait-threshold");
       }
     }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java
index f3b6d20..365ba7c 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java
@@ -17,13 +17,14 @@ package org.apache.geode.distributed.internal.membership;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.direct.DirectChannelListener;
 
 public interface DistributedMembershipListener extends DirectChannelListener {
 
   /** this method is invoked when the processing of a new view is completed */
-  void viewInstalled(NetView view);
+  void viewInstalled(MembershipView view);
 
   /**
    * this is invoked when there has been a loss of quorum and enable-network-partition-detection is
@@ -37,7 +38,7 @@ public interface DistributedMembershipListener extends DirectChannelListener {
    *
    * @param m the new member
    */
-  void newMemberConnected(InternalDistributedMember m);
+  void newMemberConnected(DistributedMember m);
 
   /**
    * Event indicating that a member has left the system
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java
index 94b1ddb..bd14909 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java
@@ -42,7 +42,6 @@ import org.apache.geode.distributed.DurableClientAttributes;
 import org.apache.geode.distributed.Role;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionAdvisor.ProfileId;
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.DataSerializableFixedID;
@@ -59,11 +58,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     DataSerializableFixedID, ProfileId, VersionSource<DistributedMember> {
   private static final long serialVersionUID = -2785249969777296507L;
 
-  /** whether to show NetMember components in toString() */
-  private static final boolean SHOW_NETMEMBER =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "show_netmembers");
-
-  protected NetMember netMbr; // the underlying member object, e.g. from JGroups
+  protected NetMember netMbr; // the underlying member object
 
   /**
    * whether this is a partial member ID (without roles, durable attributes). We use partial IDs in
@@ -98,6 +93,10 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   /** product version bit flag */
   private static final int VERSION_BIT = 0x8;
 
+  public int getVmPid() {
+    return netMbr.getProcessId();
+  }
+
   @FunctionalInterface
   public interface HostnameResolver {
     InetAddress getInetAddress(ServerLocation location) throws UnknownHostException;
@@ -112,11 +111,6 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   private static HostnameResolver hostnameResolver =
       (location) -> InetAddress.getByName(location.getHostName());
 
-  /**
-   * Representing the host name of this member.
-   */
-  private String hostName = null;
-
   private transient Version versionObj = Version.CURRENT;
 
   /** The versions in which this message was modified */
@@ -127,9 +121,9 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     netMbr.setProcessId(OSProcess.getId());
     try {
       if (SocketCreator.resolve_dns) {
-        this.hostName = SocketCreator.getHostName(SocketCreator.getLocalHost());
+        netMbr.setHostName(SocketCreator.getHostName(SocketCreator.getLocalHost()));
       } else {
-        this.hostName = SocketCreator.getLocalHost().getHostAddress();
+        netMbr.setHostName(SocketCreator.getLocalHost().getHostAddress());
       }
     } catch (UnknownHostException ee) {
       throw new InternalGemFireError(ee);
@@ -157,10 +151,11 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   public InternalDistributedMember(InetAddress i, int p, boolean splitBrainEnabled,
       boolean canBeCoordinator, MemberAttributes attr) {
 
-    this.netMbr = MemberFactory.newNetMember(i, p, splitBrainEnabled, canBeCoordinator,
-        Version.CURRENT_ORDINAL, attr);
+    String hostName = SocketCreator.resolve_dns ? SocketCreator.getHostName(i) : i.getHostAddress();
 
-    this.hostName = SocketCreator.resolve_dns ? SocketCreator.getHostName(i) : i.getHostAddress();
+    this.netMbr = MemberFactory.newNetMember(i, hostName, p, splitBrainEnabled, canBeCoordinator,
+        Version.CURRENT_ORDINAL,
+        attr);
 
     short version = netMbr.getVersionOrdinal();
     try {
@@ -174,16 +169,16 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
 
   /**
    * Construct a InternalDistributedMember based on the given NetMember.
-   * <p>
-   * This is not the preferred way of creating an instance since the NetMember may not have all
-   * required information (e.g., a JGroups address without direct-port and other information).
    *
    */
   public InternalDistributedMember(NetMember m) {
     netMbr = m;
 
-    this.hostName = SocketCreator.resolve_dns ? SocketCreator.getHostName(m.getInetAddress())
-        : m.getInetAddress().getHostAddress();
+    if (netMbr.getHostName() == null) {
+      String hostName = SocketCreator.resolve_dns ? SocketCreator.getHostName(m.getInetAddress())
+          : m.getInetAddress().getHostAddress();
+      netMbr.setHostName(hostName);
+    }
 
     short version = m.getVersionOrdinal();
     try {
@@ -192,7 +187,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       this.versionObj = Version.CURRENT;
     }
     cachedToString = null;
-    this.isPartial = true;
+    this.isPartial = false;
   }
 
   /**
@@ -220,7 +215,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
    * @throws UnknownHostException if the given hostname cannot be resolved
    */
   public InternalDistributedMember(String i, int p) {
-    this(i, p, Version.CURRENT);
+    this(MemberFactory.newNetMember(i, p));
   }
 
   /**
@@ -231,7 +226,6 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
    */
 
   public InternalDistributedMember(ServerLocation location) {
-    this.hostName = location.getHostName();
     final InetAddress addr;
     try {
       addr = hostnameResolver.getInetAddress(location);
@@ -239,44 +233,10 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       throw new ServerConnectivityException("Unable to resolve server location " + location, e);
     }
 
-    netMbr = MemberFactory.newNetMember(addr, location.getPort());
-    netMbr.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+    netMbr =
+        MemberFactory.newNetMember(addr, location.getHostName(), location.getPort(), false, true,
+            Version.CURRENT_ORDINAL, MemberAttributes.DEFAULT);
     versionObj = Version.CURRENT;
-    netMbr.setVersion(versionObj);
-  }
-
-  /**
-   * Create a InternalDistributedMember referring to the current host (as defined by the given
-   * string).
-   * <p>
-   *
-   * <b> THIS METHOD IS FOR TESTING ONLY. DO NOT USE IT TO CREATE IDs FOR USE IN THE PRODUCT. IT
-   * DOES NOT PROPERLY INITIALIZE ATTRIBUTES NEEDED FOR P2P FUNCTIONALITY. </b>
-   *
-   *
-   * @param i the hostname, must be for the current host
-   * @param p the membership listening port
-   * @param version the version of this member
-   * @throws UnknownHostException if the given hostname cannot be resolved
-   */
-  public InternalDistributedMember(String i, int p, Version version) {
-    this(i, p, version, MemberFactory.newNetMember(i, p));
-  }
-
-  /**
-   * Create a InternalDistributedMember referring to the current host (as defined by the given
-   * string).
-   * <p>
-   *
-   * <b> THIS METHOD IS FOR TESTING ONLY. DO NOT USE IT TO CREATE IDs FOR USE IN THE PRODUCT. IT
-   * DOES NOT PROPERLY INITIALIZE ATTRIBUTES NEEDED FOR P2P FUNCTIONALITY. </b>
-   **/
-  public InternalDistributedMember(String i, int p, Version version, NetMember netMember) {
-    netMbr = netMember;
-    defaultToCurrentHost();
-    netMember.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
-    this.versionObj = version;
-    netMember.setVersion(version);
   }
 
   /**
@@ -304,14 +264,13 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     MemberAttributes mattr = new MemberAttributes(p, org.apache.geode.internal.OSProcess.getId(),
         vmKind, -1, n, groups, attr);
     InetAddress addr = SocketCreator.toInetAddress(host);
-    netMbr = MemberFactory.newNetMember(addr, p, false, true, Version.CURRENT_ORDINAL, mattr);
+    netMbr = MemberFactory.newNetMember(addr, host, p, false, true, Version.CURRENT_ORDINAL, mattr);
     defaultToCurrentHost();
     netMbr.setName(n);
     this.uniqueTag = u;
     netMbr.setVmKind(vmKind);
     netMbr.setDirectPort(p);
     netMbr.setDurableClientAttributes(attr);
-    this.hostName = host;
     netMbr.setGroups(groups);
   }
 
@@ -446,41 +405,14 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     }
   }
 
-  public void setVmKind(int p) {
-    netMbr.setVmKind(p);
-    cachedToString = null;
-  }
-
   public void setVmViewId(int p) {
     netMbr.setVmViewId(p);
     cachedToString = null;
   }
 
   /**
-   * [GemStone] Returns the process id of the VM that hosts the distribution manager with this
-   * address.
-   *
-   * @since GemFire 4.0
-   */
-  public int getVmPid() {
-    return netMbr.getProcessId();
-  }
-
-  /**
-   * [GemStone] Sets the process id of the VM that hosts the distribution manager with this address.
-   *
-   * @since GemFire 4.0
-   */
-  public void setVmPid(int p) {
-    netMbr.setProcessId(p);
-    cachedToString = null;
-  }
-
-  /**
    * Returns the name of this member's distributed system connection or null if no name was
    * specified.
-   *
-   * @see org.apache.geode.distributed.DistributedSystem#getName
    */
   @Override
   public String getName() {
@@ -529,7 +461,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       return 0;
     }
     // obligatory type check
-    if ((o == null) || !(o instanceof InternalDistributedMember))
+    if (!(o instanceof InternalDistributedMember))
       throw new ClassCastException(
           "InternalDistributedMember.compareTo(): comparison between different classes");
     InternalDistributedMember other = (InternalDistributedMember) o;
@@ -636,7 +568,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       return true;
     }
     // GemStone fix for 29125
-    if ((obj == null) || !(obj instanceof InternalDistributedMember)) {
+    if (!(obj instanceof InternalDistributedMember)) {
       return false;
     }
     InternalDistributedMember other = (InternalDistributedMember) obj;
@@ -722,10 +654,6 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
         sb.append("(version:").append(Version.toString(version)).append(')');
       }
 
-      if (SHOW_NETMEMBER) {
-        sb.append("[[").append(this.netMbr).append("]]");
-      }
-
       // leave out Roles on purpose
 
       // if (netMbr instanceof GMSMember) {
@@ -747,8 +675,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     if (add.isMulticastAddress())
       host = add.getHostAddress();
     else {
-      // host = shortName(add.getHostName());
-      host = SocketCreator.resolve_dns ? shortName(this.hostName) : this.hostName;
+      String hostName = netMbr.getHostName();
+      host = SocketCreator.resolve_dns ? shortName(hostName) : hostName;
     }
 
     sb.append(host);
@@ -767,7 +695,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       }
 
       if (vmPid > 0)
-        sb.append(Integer.toString(vmPid));
+        sb.append(vmPid);
 
       String vmStr = "";
       switch (vmKind) {
@@ -845,7 +773,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     out.write(address);
     out.writeInt(getPort());
 
-    DataSerializer.writeString(this.hostName, out);
+    DataSerializer.writeString(netMbr.getHostName(), out);
 
     int flags = 0;
     if (netMbr.isNetworkPartitionDetectionEnabled())
@@ -888,7 +816,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     InetAddress inetAddr = InetAddress.getByAddress(addr);
     int port = in.readInt();
 
-    this.hostName = DataSerializer.readString(in);
+    String hostName = DataSerializer.readString(in);
 
     int flags = in.readUnsignedByte();
     boolean sbEnabled = (flags & NPD_ENABLED_BIT) != 0;
@@ -910,7 +838,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
 
     short version = readVersion(flags, in);
 
-    netMbr = MemberFactory.newNetMember(inetAddr, port, sbEnabled, elCoord, version,
+    netMbr = MemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord, version,
         new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups,
             durableClientAttributes));
     if (version >= Version.GFE_90.ordinal()) {
@@ -946,7 +874,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     DataSerializer.writeInetAddress(getInetAddress(), out);
     out.writeInt(getPort());
 
-    DataSerializer.writeString(this.hostName, out);
+    DataSerializer.writeString(netMbr.getHostName(), out);
 
     int flags = 0;
     if (netMbr.isNetworkPartitionDetectionEnabled())
@@ -998,7 +926,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     DataSerializer.writeInetAddress(getInetAddress(), out);
     out.writeInt(getPort());
 
-    DataSerializer.writeString(this.hostName, out);
+    DataSerializer.writeString(netMbr.getHostName(), out);
 
     int flags = 0;
     if (netMbr.isNetworkPartitionDetectionEnabled())
@@ -1047,9 +975,9 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     InetAddress inetAddr = DataSerializer.readInetAddress(in);
     int port = in.readInt();
 
-    this.hostName = DataSerializer.readString(in);
+    String hostName = DataSerializer.readString(in);
 
-    this.hostName = SocketCreator.resolve_dns
+    hostName = SocketCreator.resolve_dns
         ? SocketCreator.getCanonicalHostName(inetAddr, hostName) : inetAddr.getHostAddress();
 
     int flags = in.readUnsignedByte();
@@ -1082,7 +1010,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
 
     MemberAttributes attr = new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups,
         durableClientAttributes);
-    netMbr = MemberFactory.newNetMember(inetAddr, port, sbEnabled, elCoord, version, attr);
+    netMbr =
+        MemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord, version, attr);
 
     Assert.assertTrue(netMbr.getVmKind() > 0);
     // Assert.assertTrue(getPort() > 0);
@@ -1092,9 +1021,9 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     InetAddress inetAddr = DataSerializer.readInetAddress(in);
     int port = in.readInt();
 
-    this.hostName = DataSerializer.readString(in);
+    String hostName = DataSerializer.readString(in);
 
-    this.hostName = SocketCreator.resolve_dns
+    hostName = SocketCreator.resolve_dns
         ? SocketCreator.getCanonicalHostName(inetAddr, hostName) : inetAddr.getHostAddress();
 
     int flags = in.readUnsignedByte();
@@ -1127,7 +1056,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
 
     MemberAttributes attr = new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups,
         durableClientAttributes);
-    netMbr = MemberFactory.newNetMember(inetAddr, port, sbEnabled, elCoord, version, attr);
+    netMbr =
+        MemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord, version, attr);
 
     Assert.assertTrue(netMbr.getVmKind() > 0);
   }
@@ -1145,7 +1075,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     InetAddress inetAddr = DataSerializer.readInetAddress(in);
     int port = in.readInt();
 
-    this.hostName =
+    String hostName =
         SocketCreator.resolve_dns ? SocketCreator.getHostName(inetAddr) : inetAddr.getHostAddress();
 
     int flags = in.readUnsignedByte();
@@ -1167,7 +1097,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     String name = DataSerializer.readString(in);
 
     MemberAttributes attr = new MemberAttributes(-1, -1, vmKind, vmViewId, name, null, null);
-    netMbr = MemberFactory.newNetMember(inetAddr, port, sbEnabled, elCoord,
+    netMbr = MemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord,
         InternalDataSerializer.getVersionForDataStream(in).ordinal(), attr);
 
     if (InternalDataSerializer.getVersionForDataStream(in).compareTo(Version.GFE_90) == 0) {
@@ -1210,13 +1140,6 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   }
 
   /**
-   * [GemStone] Set the direct channel port
-   */
-  public void setDirectChannelPort(int p) {
-    netMbr.setDirectPort(p);
-  }
-
-  /**
    * Set the membership port. This is done in loner systems using client/server connection
    * information to help form a unique ID
    */
@@ -1252,10 +1175,6 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       sb.append("(version:").append(Version.toString(version)).append(')');
     }
 
-    if (SHOW_NETMEMBER) {
-      sb.append("[[").append(netMbr.getUniqueId()).append("]]");
-    }
-
     return sb.toString();
   }
 
@@ -1279,35 +1198,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   }
 
   @VisibleForTesting
-  void setIsPartial(boolean value) {
+  public void setIsPartial(boolean value) {
     isPartial = value;
   }
 
-  public static class InternalDistributedMemberWrapper {
-    InternalDistributedMember mbr;
-
-    public InternalDistributedMemberWrapper(InternalDistributedMember m) {
-      this.mbr = m;
-    }
-
-    public InternalDistributedMember getMbr() {
-      return mbr;
-    }
-
-    @Override
-    public int hashCode() {
-      return mbr.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      InternalDistributedMember other = ((InternalDistributedMemberWrapper) obj).mbr;
-      return mbr.compareTo(other, false, false) == 0;
-    }
-
-    @Override
-    public String toString() {
-      return "InternalDistributedMemberWrapper [mbr=" + mbr + "]";
-    }
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberFactory.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberFactory.java
index 84bf1b3..0c8f3f8 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberFactory.java
@@ -21,8 +21,7 @@ import org.apache.geode.annotations.Immutable;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.LocatorStats;
-import org.apache.geode.distributed.internal.membership.gms.GMSMemberFactory;
-import org.apache.geode.distributed.internal.membership.gms.NetLocator;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMemberFactory;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Authenticator;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 
@@ -48,9 +47,12 @@ public class MemberFactory {
    * @param payload the payload for this member
    * @return the new NetMember
    */
-  public static NetMember newNetMember(InetAddress i, int p, boolean splitBrainEnabled,
-      boolean canBeCoordinator, short version, MemberAttributes payload) {
-    return services.newNetMember(i, p, splitBrainEnabled, canBeCoordinator, payload, version);
+  public static NetMember newNetMember(InetAddress i, String hostName, int p,
+      boolean splitBrainEnabled,
+      boolean canBeCoordinator, short version,
+      MemberAttributes payload) {
+    return services.newNetMember(i, hostName, p, splitBrainEnabled, canBeCoordinator, payload,
+        version);
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberServices.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberServices.java
index 5e21234..82f3d74 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberServices.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberServices.java
@@ -20,7 +20,6 @@ import java.nio.file.Path;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.LocatorStats;
-import org.apache.geode.distributed.internal.membership.gms.NetLocator;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Authenticator;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 
@@ -36,6 +35,7 @@ public interface MemberServices {
    *
    * @param i the name of the host for the specified NetMember, the current host (hopefully) if
    *        there are any problems.
+   * @param hostName the associated host name
    * @param port the membership port
    * @param splitBrainEnabled whether the member has this feature enabled
    * @param canBeCoordinator whether the member can be membership coordinator
@@ -43,7 +43,8 @@ public interface MemberServices {
    * @param version TODO
    * @return the new NetMember
    */
-  NetMember newNetMember(InetAddress i, int port, boolean splitBrainEnabled,
+  NetMember newNetMember(InetAddress i, String hostName, int port,
+      boolean splitBrainEnabled,
       boolean canBeCoordinator, MemberAttributes payload, short version);
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipManager.java
index b5c49c0..1377327 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipManager.java
@@ -48,7 +48,7 @@ public interface MembershipManager {
    *
    * @return list of members
    */
-  NetView getView();
+  MembershipView getView();
 
   /**
    * Returns an object that is used to sync access to the view. While this lock is held the view
@@ -83,6 +83,21 @@ public interface MembershipManager {
 
 
   /**
+   * test method for simulating a sick/dead member
+   */
+  void beSick();
+
+  /**
+   * test method for simulating a sick/dead member
+   */
+  void playDead();
+
+  /**
+   * test method for simulating a sick/dead member
+   */
+  void beHealthy();
+
+  /**
    * A test hook for healthiness tests
    */
   boolean isBeingSick();
@@ -107,7 +122,7 @@ public interface MembershipManager {
   /**
    * A shutdown message has been received from another member
    */
-  void shutdownMessageReceived(InternalDistributedMember id, String reason);
+  void shutdownMessageReceived(DistributedMember id, String reason);
 
   /**
    * Stall the current thread until we are ready to accept view events
@@ -214,7 +229,7 @@ public interface MembershipManager {
    *
    * @return true if membership is confirmed, else timeout and false
    */
-  boolean waitForNewMember(InternalDistributedMember remoteId);
+  boolean waitForNewMember(DistributedMember remoteId);
 
   /**
    * Release critical resources, avoiding any possibility of deadlock
@@ -224,6 +239,12 @@ public interface MembershipManager {
   void emergencyClose();
 
   /**
+   * Notifies the manager that a member has contacted us who is not in the current membership view
+   *
+   */
+  void addSurpriseMemberForTesting(DistributedMember mbr, long birthTime);
+
+  /**
    * Request the current membership coordinator to remove the given member
    */
   boolean requestMemberRemoval(DistributedMember member, String reason);
@@ -238,20 +259,20 @@ public interface MembershipManager {
    * @param reason why the check is being done (must not be blank/null)
    * @return true if the member checks out
    */
-  boolean verifyMember(InternalDistributedMember mbr, String reason);
+  boolean verifyMember(DistributedMember mbr, String reason);
 
 
   /**
    * Initiate SUSPECT processing for the given members. This may be done if the members have not
    * been responsive. If they fail SUSPECT processing, they will be removed from membership.
    */
-  void suspectMembers(Set<InternalDistributedMember> members, String reason);
+  void suspectMembers(Set<DistributedMember> members, String reason);
 
   /**
    * Initiate SUSPECT processing for the given member. This may be done if the member has not been
    * responsive. If it fails SUSPECT processing, it will be removed from membership.
    */
-  void suspectMember(InternalDistributedMember member, String reason);
+  void suspectMember(DistributedMember member, String reason);
 
   /**
    * if the manager initiated shutdown, this will return the cause of abnormal termination of
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java
new file mode 100644
index 0000000..9371cae
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java
@@ -0,0 +1,226 @@
+/*
+ * 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.distributed.internal.membership;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+
+/**
+ * The MembershipView class represents a membership view. Note that this class is not synchronized,
+ * so take that under advisement if you decide to modify a view with add() or remove().
+ */
+public class MembershipView {
+
+  private int viewId;
+  private List<InternalDistributedMember> members;
+  private Set<InternalDistributedMember> shutdownMembers;
+  private Set<InternalDistributedMember> crashedMembers;
+  private InternalDistributedMember creator;
+  private Set<InternalDistributedMember> hashedMembers;
+
+
+  public MembershipView() {
+    viewId = 0;
+    members = new ArrayList<>(4);
+    this.hashedMembers = new HashSet<>(members);
+    shutdownMembers = Collections.emptySet();
+    crashedMembers = new HashSet<>();
+    creator = null;
+  }
+
+  public MembershipView(InternalDistributedMember creator, int viewId,
+      List<InternalDistributedMember> members) {
+    this.viewId = viewId;
+    this.members = new ArrayList<>(members);
+    hashedMembers = new HashSet<>(this.members);
+    shutdownMembers = new HashSet<>();
+    crashedMembers = Collections.emptySet();
+    this.creator = creator;
+  }
+
+  /**
+   * Create a new view with the contents of the given view and the specified view ID
+   */
+  public MembershipView(MembershipView other, int viewId) {
+    this.creator = other.creator;
+    this.viewId = viewId;
+    this.members = new ArrayList<>(other.members);
+    this.hashedMembers = new HashSet<>(other.members);
+    this.shutdownMembers = new HashSet<>(other.shutdownMembers);
+    this.crashedMembers = new HashSet<>(other.crashedMembers);
+  }
+
+  public MembershipView(InternalDistributedMember creator, int viewId,
+      List<InternalDistributedMember> mbrs, Set<InternalDistributedMember> shutdowns,
+      Set<InternalDistributedMember> crashes) {
+    this.creator = creator;
+    this.viewId = viewId;
+    this.members = mbrs;
+    this.hashedMembers = new HashSet<>(mbrs);
+    this.shutdownMembers = shutdowns;
+    this.crashedMembers = crashes;
+  }
+
+  public int getViewId() {
+    return this.viewId;
+  }
+
+  public InternalDistributedMember getCreator() {
+    return this.creator;
+  }
+
+  public void setCreator(InternalDistributedMember creator) {
+    this.creator = creator;
+  }
+
+  public void setViewId(int viewId) {
+    this.viewId = viewId;
+  }
+
+
+
+  public List<InternalDistributedMember> getMembers() {
+    return Collections.unmodifiableList(this.members);
+  }
+
+  /**
+   * return members that are i this view but not the given old view
+   */
+  public List<InternalDistributedMember> getNewMembers(MembershipView olderView) {
+    List<InternalDistributedMember> result = new ArrayList<>(members);
+    result.removeAll(olderView.getMembers());
+    return result;
+  }
+
+  public Object get(int i) {
+    return this.members.get(i);
+  }
+
+  public void add(InternalDistributedMember mbr) {
+    this.hashedMembers.add(mbr);
+    this.members.add(mbr);
+  }
+
+  public boolean remove(InternalDistributedMember mbr) {
+    this.hashedMembers.remove(mbr);
+    return this.members.remove(mbr);
+  }
+
+  public void removeAll(Collection<InternalDistributedMember> ids) {
+    this.hashedMembers.removeAll(ids);
+    ids.forEach(this::remove);
+  }
+
+  public boolean contains(DistributedMember mbr) {
+    assert mbr instanceof InternalDistributedMember;
+    return this.hashedMembers.contains(mbr);
+  }
+
+  public int size() {
+    return this.members.size();
+  }
+
+  public InternalDistributedMember getLeadMember() {
+    for (InternalDistributedMember mbr : this.members) {
+      if (mbr.getVmKind() == ClusterDistributionManager.NORMAL_DM_TYPE) {
+        return mbr;
+      }
+    }
+    return null;
+  }
+
+  public InternalDistributedMember getCoordinator() {
+    for (InternalDistributedMember addr : members) {
+      if (addr.getNetMember().preferredForCoordinator()) {
+        return addr;
+      }
+    }
+    if (members.size() > 0) {
+      return members.get(0);
+    }
+    return null;
+  }
+
+  public Set<InternalDistributedMember> getShutdownMembers() {
+    return this.shutdownMembers;
+  }
+
+  public Set<InternalDistributedMember> getCrashedMembers() {
+    return this.crashedMembers;
+  }
+
+  public String toString() {
+    InternalDistributedMember lead = getLeadMember();
+
+    StringBuilder sb = new StringBuilder(200);
+    sb.append("View[").append(creator).append('|').append(viewId).append("] members: [");
+    boolean first = true;
+    for (InternalDistributedMember mbr : this.members) {
+      if (!first)
+        sb.append(", ");
+      sb.append(mbr);
+      if (mbr == lead) {
+        sb.append("{lead}");
+      }
+      first = false;
+    }
+    if (!this.shutdownMembers.isEmpty()) {
+      sb.append("]  shutdown: [");
+      first = true;
+      for (InternalDistributedMember mbr : this.shutdownMembers) {
+        if (!first)
+          sb.append(", ");
+        sb.append(mbr);
+        first = false;
+      }
+    }
+    if (!this.crashedMembers.isEmpty()) {
+      sb.append("]  crashed: [");
+      first = true;
+      for (InternalDistributedMember mbr : this.crashedMembers) {
+        if (!first)
+          sb.append(", ");
+        sb.append(mbr);
+        first = false;
+      }
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+
+  @Override
+  public synchronized boolean equals(Object other) {
+    if (other == this) {
+      return true;
+    }
+    if (other instanceof MembershipView) {
+      return this.members.equals(((MembershipView) other).getMembers());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    return this.members.hashCode();
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/NetLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetLocator.java
similarity index 85%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/NetLocator.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetLocator.java
index 9739f27..de91d9d 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/NetLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetLocator.java
@@ -12,9 +12,9 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.distributed.internal.membership.gms;
+package org.apache.geode.distributed.internal.membership;
 
-import org.apache.geode.distributed.internal.membership.MembershipManager;
+import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
 
 public interface NetLocator extends TcpHandler {
@@ -25,6 +25,6 @@ public interface NetLocator extends TcpHandler {
    *
    * @return true if the membership manager was accepted
    */
-  boolean setMembershipManager(MembershipManager mgr);
+  boolean setServices(Services pservices);
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java
index c6da0b3..920db7c 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java
@@ -23,24 +23,18 @@ import org.apache.geode.distributed.DurableClientAttributes;
 import org.apache.geode.internal.Version;
 
 /**
- * This is the SPI for the basic element of membership provided in the GemFire system.
+ * This is the SPI for the basic element of membership provided in Geode.
  *
  *
  */
 public interface NetMember extends Comparable<NetMember> {
 
-  void setAttributes(MemberAttributes args);
-
-  MemberAttributes getAttributes();
-
   InetAddress getInetAddress();
 
   int getPort();
 
   void setPort(int p);
 
-  boolean isMulticastAddress();
-
   short getVersionOrdinal();
 
   /**
@@ -110,9 +104,9 @@ public interface NetMember extends Comparable<NetMember> {
   /** compare data that is not known to DistributedMember instances */
   int compareAdditionalData(NetMember other);
 
-  /**
-   * Return a unique string id for this member which is immutable and will not change during the
-   * life of this member.
-   */
-  String getUniqueId();
+  void setDurableTimeout(int newValue);
+
+  void setHostName(String hostName);
+
+  String getHostName();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/QuorumChecker.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/QuorumChecker.java
index 2fafbae..4563563 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/QuorumChecker.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/QuorumChecker.java
@@ -54,8 +54,4 @@ public interface QuorumChecker {
    */
   MembershipInformation getMembershipInfo();
 
-  /**
-   * Returns the membership view that is being used to establish a quorum
-   */
-  NetView getView();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSLocatorAdapter.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSLocatorAdapter.java
new file mode 100644
index 0000000..b0667fe
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSLocatorAdapter.java
@@ -0,0 +1,98 @@
+/*
+ * 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.distributed.internal.membership.adapter;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.file.Path;
+
+import org.apache.geode.cache.GemFireCache;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.LocatorStats;
+import org.apache.geode.distributed.internal.membership.NetLocator;
+import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.interfaces.Locator;
+import org.apache.geode.distributed.internal.membership.gms.locator.GMSLocator;
+import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
+import org.apache.geode.distributed.internal.tcpserver.TcpServer;
+
+public class GMSLocatorAdapter implements TcpHandler, NetLocator {
+
+  private final GMSLocator gmsLocator;
+
+  /**
+   * @param bindAddress network address that TcpServer will bind to
+   * @param locatorString location of other locators (bootstrapping, failover)
+   * @param usePreferredCoordinators true if the membership coordinator should be a Locator
+   * @param networkPartitionDetectionEnabled true if network partition detection is enabled
+   * @param locatorStats the locator statistics object
+   * @param securityUDPDHAlgo DF algorithm
+   * @param workingDirectory directory to use for view file (defaults to "user.dir")
+   */
+  public GMSLocatorAdapter(InetAddress bindAddress, String locatorString,
+      boolean usePreferredCoordinators,
+      boolean networkPartitionDetectionEnabled, LocatorStats locatorStats,
+      String securityUDPDHAlgo, Path workingDirectory) {
+    gmsLocator =
+        new GMSLocator(bindAddress, locatorString, usePreferredCoordinators,
+            networkPartitionDetectionEnabled,
+            locatorStats, securityUDPDHAlgo, workingDirectory);
+  }
+
+  @Override
+  public Object processRequest(Object request) throws IOException {
+    return gmsLocator.processRequest(request);
+  }
+
+  @Override
+  public void endRequest(Object request, long startTime) {
+    gmsLocator.endRequest(request, startTime);
+  }
+
+  @Override
+  public void endResponse(Object request, long startTime) {
+    gmsLocator.endResponse(request, startTime);
+  }
+
+  @Override
+  public void shutDown() {
+    gmsLocator.shutDown();
+  }
+
+  @Override
+  public void restarting(DistributedSystem ds, GemFireCache cache,
+      InternalConfigurationPersistenceService sharedConfig) {
+    gmsLocator.setServices(
+        ((GMSMembershipManager) ((InternalDistributedSystem) ds).getDM().getMembershipManager())
+            .getGMSManager()
+            .getServices());
+  }
+
+  @Override
+  public void init(TcpServer tcpServer) {
+    gmsLocator.init("" + tcpServer.getPort());
+  }
+
+  @Override
+  public boolean setServices(Services pservices) {
+    return gmsLocator.setServices(pservices);
+  }
+
+  public Locator getGMSLocator() {
+    return gmsLocator;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberAdapter.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberAdapter.java
new file mode 100644
index 0000000..978fdef
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberAdapter.java
@@ -0,0 +1,225 @@
+/*
+ * 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.distributed.internal.membership.adapter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetAddress;
+
+import org.apache.geode.distributed.DurableClientAttributes;
+import org.apache.geode.distributed.internal.membership.NetMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.internal.Version;
+
+/**
+ * GMSMemberAdapter implements the NetMember interface required by InternalDistributedMember
+ * to supply it with membership details. GMSMemberAdapter delegates to a GMSMember, which
+ * is the identifier class for group membership services.
+ */
+public class GMSMemberAdapter implements NetMember {
+
+  private GMSMember gmsMember;
+  private DurableClientAttributes durableClientAttributes;
+
+  public GMSMemberAdapter(GMSMember gmsMember) {
+    this.gmsMember = gmsMember;
+    String durableId = gmsMember.getDurableId();
+    if (durableId != null) {
+      this.durableClientAttributes =
+          new DurableClientAttributes(durableId, gmsMember.getDurableTimeout());
+    }
+  }
+
+  public GMSMember getGmsMember() {
+    return gmsMember;
+  }
+
+  @Override
+  public InetAddress getInetAddress() {
+    return gmsMember.getInetAddress();
+  }
+
+  @Override
+  public int getPort() {
+    return gmsMember.getPort();
+  }
+
+  @Override
+  public void setPort(int p) {
+    gmsMember.setPort(p);
+  }
+
+  @Override
+  public short getVersionOrdinal() {
+    return gmsMember.getVersionOrdinal();
+  }
+
+  @Override
+  public boolean isNetworkPartitionDetectionEnabled() {
+    return gmsMember.isNetworkPartitionDetectionEnabled();
+  }
+
+  @Override
+  public void setNetworkPartitionDetectionEnabled(boolean enabled) {
+    gmsMember.setNetworkPartitionDetectionEnabled(enabled);
+  }
+
+  @Override
+  public boolean preferredForCoordinator() {
+    return gmsMember.preferredForCoordinator();
+  }
+
+  @Override
+  public void setPreferredForCoordinator(boolean preferred) {
+    gmsMember.setPreferredForCoordinator(preferred);
+  }
+
+  @Override
+  public byte getMemberWeight() {
+    return gmsMember.getMemberWeight();
+  }
+
+  @Override
+  public void setVersion(Version v) {
+    gmsMember.setVersion(v);
+  }
+
+  @Override
+  public int getProcessId() {
+    return gmsMember.getProcessId();
+  }
+
+  @Override
+  public void setProcessId(int id) {
+    gmsMember.setProcessId(id);
+  }
+
+  @Override
+  public byte getVmKind() {
+    return gmsMember.getVmKind();
+  }
+
+  @Override
+  public void setVmKind(int kind) {
+    gmsMember.setVmKind(kind);
+  }
+
+  @Override
+  public int getVmViewId() {
+    return gmsMember.getVmViewId();
+  }
+
+  @Override
+  public void setVmViewId(int id) {
+    gmsMember.setVmViewId(id);
+  }
+
+  @Override
+  public int getDirectPort() {
+    return gmsMember.getDirectPort();
+  }
+
+  @Override
+  public void setDirectPort(int port) {
+    gmsMember.setDirectPort(port);
+  }
+
+  @Override
+  public String getName() {
+    return gmsMember.getName();
+  }
+
+  @Override
+  public void setName(String name) {
+    gmsMember.setName(name);
+  }
+
+  @Override
+  public DurableClientAttributes getDurableClientAttributes() {
+    return this.durableClientAttributes;
+  }
+
+  @Override
+  public void setDurableTimeout(int newValue) {
+    if (durableClientAttributes != null) {
+      durableClientAttributes.updateTimeout(newValue);
+      gmsMember.setDurableTimeout(newValue);
+    }
+  }
+
+  @Override
+  public void setHostName(String hostName) {
+    gmsMember.setHostName(hostName);
+  }
+
+  @Override
+  public String getHostName() {
+    return gmsMember.getHostName();
+  }
+
+  @Override
+  public void setDurableClientAttributes(DurableClientAttributes attributes) {
+    durableClientAttributes = attributes;
+    if (attributes != null) {
+      gmsMember.setDurableId(attributes.getId());
+      gmsMember.setDurableTimeout(attributes.getTimeout());
+    }
+  }
+
+  @Override
+  public String[] getGroups() {
+    return gmsMember.getGroups();
+  }
+
+  @Override
+  public void setGroups(String[] groups) {
+    gmsMember.setGroups(groups);
+  }
+
+  @Override
+  public boolean hasAdditionalData() {
+    return gmsMember.hasAdditionalData();
+  }
+
+  @Override
+  public void writeAdditionalData(DataOutput out) throws IOException {
+    gmsMember.writeAdditionalData(out);
+  }
+
+  @Override
+  public void readAdditionalData(DataInput in) throws ClassNotFoundException, IOException {
+    gmsMember.readAdditionalData(in);
+  }
+
+  @Override
+  public int compareAdditionalData(NetMember other) {
+    return gmsMember.compareAdditionalData(((GMSMemberAdapter) other).getGmsMember());
+  }
+
+  @Override
+  public int compareTo(NetMember o) {
+    return gmsMember.compareTo(((GMSMemberAdapter) o).getGmsMember());
+  }
+
+  @Override
+  public String toString() {
+    return "adapter(" + gmsMember + ")";
+  }
+
+  public void setGmsMember(GMSMember canonicalID) {
+    gmsMember = canonicalID;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberFactory.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberFactory.java
similarity index 76%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberFactory.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberFactory.java
index 6fca0a0..993b71b 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberFactory.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.distributed.internal.membership.gms;
+package org.apache.geode.distributed.internal.membership.adapter;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -20,6 +20,7 @@ import java.nio.file.Path;
 
 import org.apache.geode.GemFireConfigException;
 import org.apache.geode.SystemConnectException;
+import org.apache.geode.distributed.DurableClientAttributes;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionException;
@@ -28,9 +29,11 @@ import org.apache.geode.distributed.internal.membership.DistributedMembershipLis
 import org.apache.geode.distributed.internal.membership.MemberAttributes;
 import org.apache.geode.distributed.internal.membership.MemberServices;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
+import org.apache.geode.distributed.internal.membership.NetLocator;
 import org.apache.geode.distributed.internal.membership.NetMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Authenticator;
-import org.apache.geode.distributed.internal.membership.gms.locator.GMSLocator;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.net.SocketCreator;
@@ -49,17 +52,27 @@ public class GMSMemberFactory implements MemberServices {
    *
    * @param i the name of the host for the specified NetMember, the current host (hopefully) if
    *        there are any problems.
+   * @param p the membership port
    * @param splitBrainEnabled whether the member has this feature enabled
    * @param canBeCoordinator whether the member can be membership coordinator
-   * @param p the membership port
    * @param attr the MemberAttributes
    * @return the new NetMember
    */
   @Override
-  public NetMember newNetMember(InetAddress i, int p, boolean splitBrainEnabled,
+  public NetMember newNetMember(InetAddress i, String hostName, int p,
+      boolean splitBrainEnabled,
       boolean canBeCoordinator, MemberAttributes attr, short version) {
-    GMSMember result =
-        new GMSMember(attr, i, p, splitBrainEnabled, canBeCoordinator, version, 0, 0);
+    DurableClientAttributes durableClientAttributes = attr.getDurableClientAttributes();
+    String durableId = null;
+    int durableTimeout = 0;
+    if (durableClientAttributes != null) {
+      durableId = durableClientAttributes.getId();
+      durableTimeout = durableClientAttributes.getTimeout();
+    }
+    GMSMemberAdapter result =
+        new GMSMemberAdapter(new GMSMember(i, hostName, p, attr.getVmPid(), (byte) attr.getVmKind(),
+            attr.getPort(), attr.getVmViewId(), attr.getName(), attr.getGroups(),
+            durableId, durableTimeout, splitBrainEnabled, canBeCoordinator, version, 0, 0));
     return result;
   }
 
@@ -73,8 +86,8 @@ public class GMSMemberFactory implements MemberServices {
    */
   @Override
   public NetMember newNetMember(InetAddress i, int p) {
-    return new GMSMember(MemberAttributes.DEFAULT, i, p, false, true, Version.CURRENT_ORDINAL, 0,
-        0);
+    return newNetMember(i, i.getHostName(), p, false, true, MemberAttributes.DEFAULT,
+        Version.CURRENT_ORDINAL);
   }
 
   /**
@@ -102,7 +115,9 @@ public class GMSMemberFactory implements MemberServices {
       final RemoteTransportConfig transport, DMStats stats,
       final Authenticator authenticator,
       final DistributionConfig config) throws DistributionException {
-    Services services = new Services(listener, transport, stats, authenticator, config);
+    GMSMembershipManager gmsMembershipManager = new GMSMembershipManager(listener);
+    Services services =
+        new Services(gmsMembershipManager.getGMSManager(), transport, stats, authenticator, config);
     try {
       services.init();
       services.start();
@@ -116,7 +131,7 @@ public class GMSMemberFactory implements MemberServices {
       Services.getLogger().error("Unexpected problem starting up membership services", e);
       throw new SystemConnectException("Problem starting up membership services", e);
     }
-    return (MembershipManager) services.getManager();
+    return gmsMembershipManager;
   }
 
   @Override
@@ -124,7 +139,7 @@ public class GMSMemberFactory implements MemberServices {
       boolean usePreferredCoordinators, boolean networkPartitionDetectionEnabled,
       LocatorStats stats, String securityUDPDHAlgo, Path workingDirectory) {
 
-    return new GMSLocator(bindAddress, locatorString, usePreferredCoordinators,
+    return new GMSLocatorAdapter(bindAddress, locatorString, usePreferredCoordinators,
         networkPartitionDetectionEnabled, stats, securityUDPDHAlgo, workingDirectory);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java
similarity index 81%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java
index d0feb1c..881866c 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.distributed.internal.membership.gms.mgr;
+package org.apache.geode.distributed.internal.membership.adapter;
 
 import java.io.NotSerializableException;
 import java.util.ArrayList;
@@ -70,15 +70,19 @@ import org.apache.geode.distributed.internal.membership.DistributedMembershipLis
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.distributed.internal.membership.MembershipTestHook;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.MembershipView;
 import org.apache.geode.distributed.internal.membership.QuorumChecker;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.SuspectMember;
 import org.apache.geode.distributed.internal.membership.gms.fd.GMSHealthMonitor;
+import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Manager;
 import org.apache.geode.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import org.apache.geode.distributed.internal.membership.gms.messenger.GMSQuorumChecker;
 import org.apache.geode.internal.Assert;
+import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
@@ -90,7 +94,7 @@ import org.apache.geode.internal.tcp.ConnectExceptions;
 import org.apache.geode.internal.tcp.MemberShunnedException;
 import org.apache.geode.internal.util.Breadcrumbs;
 
-public class GMSMembershipManager implements MembershipManager, Manager {
+public class GMSMembershipManager implements MembershipManager {
   private static final Logger logger = Services.getLogger();
 
   /** product version to use for multicast serialization */
@@ -122,6 +126,12 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   private final ThreadLocal<Boolean> forceUseUDPMessaging =
       ThreadLocal.withInitial(() -> Boolean.FALSE);
 
+
+  private final ManagerImpl gmsManager;
+
+  private long ackSevereAlertThreshold;
+  private long ackWaitThreshold;
+
   /**
    * Trick class to make the startup synch more visible in stack traces
    *
@@ -149,7 +159,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     // Miscellaneous state depending on the kind of event
     InternalDistributedMember member;
     DistributionMessage dmsg;
-    NetView gmsView;
+    MembershipView gmsView;
 
     @Override
     public String toString() {
@@ -197,7 +207,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
      *
      * @param v the new view
      */
-    StartupEvent(NetView v) {
+    StartupEvent(MembershipView v) {
       this.kind = VIEW;
       this.gmsView = v;
     }
@@ -245,7 +255,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
    *
    * All accesses to this object are protected via {@link #latestViewLock}
    */
-  private NetView latestView = new NetView();
+  private MembershipView latestView = new MembershipView();
 
   /**
    * This is the lock for protecting access to latestView
@@ -398,7 +408,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     @Override
     public void messageReceived(DistributionMessage msg) {
       // bug 36851 - notify failure detection that we've had contact from a member
-      services.getHealthMonitor().contactedBy(msg.getSender());
+      services.getHealthMonitor()
+          .contactedBy(((GMSMemberAdapter) msg.getSender().getNetMember()).getGmsMember());
       handleOrDeferMessage(msg);
     }
 
@@ -413,7 +424,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   /**
    * Analyze a given view object, generate events as appropriate
    */
-  protected void processView(long newViewId, NetView newView) {
+  protected void processView(long newViewId, MembershipView newView) {
     // Sanity check...
     if (logger.isDebugEnabled()) {
       StringBuilder msg = new StringBuilder(200);
@@ -455,12 +466,12 @@ public class GMSMembershipManager implements MembershipManager, Manager {
       }
 
       // Save previous view, for delta analysis
-      NetView priorView = latestView;
+      MembershipView priorView = latestView;
 
       // update the view to reflect our changes, so that
       // callbacks will see the new (updated) view.
       latestViewId = newViewId;
-      latestView = new NetView(newView, newView.getViewId());
+      latestView = new MembershipView(newView, newView.getViewId());
 
       // look for additions
       for (int i = 0; i < newView.getMembers().size(); i++) { // additions
@@ -630,11 +641,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   private boolean tcpDisabled;
 
 
-  @Override
-  public boolean isMulticastAllowed() {
-    return !disableMulticastForRollingUpgrade;
-  }
-
   /**
    * Joins the distributed system
    *
@@ -658,8 +664,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
               + "Operation either timed out, was stopped or Locator does not exist.");
         }
 
-        NetView initialView = services.getJoinLeave().getView();
-        latestView = new NetView(initialView, initialView.getViewId());
+        MembershipView initialView = createGeodeView(services.getJoinLeave().getView());
+        latestView = new MembershipView(initialView, initialView.getViewId());
         listener.viewInstalled(latestView);
 
       } catch (RuntimeException ex) {
@@ -679,128 +685,73 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     }
   }
 
-
-  public GMSMembershipManager(DistributedMembershipListener listener) {
-    Assert.assertTrue(listener != null);
-    this.listener = listener;
+  private MembershipView createGeodeView(GMSMembershipView view) {
+    return createGeodeView(view.getCreator(), view.getViewId(), view.getMembers(),
+        view.getShutdownMembers(),
+        view.getCrashedMembers());
   }
 
-  @Override
-  public void init(Services services) {
-    this.services = services;
-
-    Assert.assertTrue(services != null);
-
-    DistributionConfig config = services.getConfig().getDistributionConfig();
-    RemoteTransportConfig transport = services.getConfig().getTransport();
-
-    this.membershipCheckTimeout = config.getSecurityPeerMembershipTimeout();
-    this.wasReconnectingSystem = transport.getIsReconnectingDS();
-
-    // cache these settings for use in send()
-    this.mcastEnabled = transport.isMcastEnabled();
-    this.tcpDisabled = transport.isTcpDisabled();
-
-    if (!this.tcpDisabled) {
-      dcReceiver = new MyDCReceiver(listener);
+  private MembershipView createGeodeView(GMSMember gmsCreator, int viewId,
+      List<GMSMember> gmsMembers,
+      Set<GMSMember> gmsShutdowns, Set<GMSMember> gmsCrashes) {
+    InternalDistributedMember geodeCreator =
+        new InternalDistributedMember(new GMSMemberAdapter(gmsCreator));
+    List<InternalDistributedMember> geodeMembers = new ArrayList<>(gmsMembers.size());
+    for (GMSMember member : gmsMembers) {
+      geodeMembers.add(new InternalDistributedMember(new GMSMemberAdapter(member)));
     }
-
-    surpriseMemberTimeout =
-        Math.max(20 * DistributionConfig.DEFAULT_MEMBER_TIMEOUT, 20 * config.getMemberTimeout());
-    surpriseMemberTimeout =
-        Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "surprise-member-timeout",
-            surpriseMemberTimeout).intValue();
-
+    Set<InternalDistributedMember> geodeShutdownMembers =
+        gmsMemberCollectionToInternalDistributedMemberSet(gmsShutdowns);
+    Set<InternalDistributedMember> geodeCrashedMembers =
+        gmsMemberCollectionToInternalDistributedMemberSet(gmsCrashes);
+    return new MembershipView(geodeCreator, viewId, geodeMembers, geodeShutdownMembers,
+        geodeCrashedMembers);
   }
 
-  @Override
-  public void start() {
-    DistributionConfig config = services.getConfig().getDistributionConfig();
-
-    int dcPort = 0;
-    if (!tcpDisabled) {
-      directChannel = new DirectChannel(this, dcReceiver, config);
-      dcPort = directChannel.getPort();
+  private Set<InternalDistributedMember> gmsMemberCollectionToInternalDistributedMemberSet(
+      Collection<GMSMember> gmsMembers) {
+    if (gmsMembers.size() == 0) {
+      return Collections.emptySet();
+    } else if (gmsMembers.size() == 1) {
+      return Collections.singleton(
+          new InternalDistributedMember(new GMSMemberAdapter(gmsMembers.iterator().next())));
+    } else {
+      Set<InternalDistributedMember> idmMembers = new HashSet<>(gmsMembers.size());
+      for (GMSMember member : gmsMembers) {
+        idmMembers.add(new InternalDistributedMember(new GMSMemberAdapter((member))));
+      }
+      return idmMembers;
     }
-
-
-    services.getMessenger().getMemberID().setDirectChannelPort(dcPort);
-
   }
 
 
-  @Override
-  public void joinDistributedSystem() {
-    long startTime = System.currentTimeMillis();
-
-    try {
-      join();
-    } catch (RuntimeException e) {
-      if (directChannel != null) {
-        directChannel.disconnect(e);
+  private List<InternalDistributedMember> gmsMemberListToInternalDistributedMemberList(
+      List<GMSMember> gmsMembers) {
+    if (gmsMembers.size() == 0) {
+      return Collections.emptyList();
+    } else if (gmsMembers.size() == 1) {
+      return Collections
+          .singletonList(new InternalDistributedMember(new GMSMemberAdapter(gmsMembers.get(0))));
+    } else {
+      List<InternalDistributedMember> idmMembers = new ArrayList<>(gmsMembers.size());
+      for (GMSMember member : gmsMembers) {
+        idmMembers.add(new InternalDistributedMember(new GMSMemberAdapter((member))));
       }
-      throw e;
+      return idmMembers;
     }
-
-    this.address = services.getMessenger().getMemberID();
-
-    if (directChannel != null) {
-      directChannel.setLocalAddr(address);
-    }
-
-    this.hasJoined = true;
-
-    // in order to debug startup issues we need to announce the membership
-    // ID as soon as we know it
-    logger.info("Finished joining (took {}ms).",
-        "" + (System.currentTimeMillis() - startTime));
-
   }
 
-  @Override
-  public void started() {
-    startCleanupTimer();
-  }
-
-
-  /** this is invoked by JoinLeave when there is a loss of quorum in the membership system */
-  @Override
-  public void quorumLost(Collection<InternalDistributedMember> failures, NetView view) {
-    // notify of quorum loss if split-brain detection is enabled (meaning we'll shut down) or
-    // if the loss is more than one member
-
-    boolean notify = failures.size() > 1;
-    if (!notify) {
-      notify = services.getConfig().isNetworkPartitionDetectionEnabled();
-    }
-
-    if (notify) {
-      List<InternalDistributedMember> remaining = new ArrayList<>(view.getMembers());
-      remaining.removeAll(failures);
-
-      if (inhibitForceDisconnectLogging) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("<ExpectedException action=add>Possible loss of quorum</ExpectedException>");
-        }
-      }
-      logger.fatal("Possible loss of quorum due to the loss of {} cache processes: {}",
-          failures.size(), failures);
-      if (inhibitForceDisconnectLogging) {
-        if (logger.isDebugEnabled()) {
-          logger.debug(
-              "<ExpectedException action=remove>Possible loss of quorum</ExpectedException>");
-        }
-      }
 
 
-      try {
-        this.listener.quorumLost(new HashSet<>(failures), remaining);
-      } catch (CancelException e) {
-        // safe to ignore - a forced disconnect probably occurred
-      }
-    }
+  public GMSMembershipManager(DistributedMembershipListener listener) {
+    Assert.assertTrue(listener != null);
+    this.listener = listener;
+    this.gmsManager = new ManagerImpl();
   }
 
+  public Manager getGMSManager() {
+    return this.gmsManager;
+  }
 
   @Override
   public boolean testMulticast() {
@@ -945,7 +896,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
         // should ensure it is not chosen as an elder.
         // This will get corrected when the member finally shows up in the
         // view.
-        NetView newMembers = new NetView(latestView, latestView.getViewId());
+        MembershipView newMembers = new MembershipView(latestView, latestView.getViewId());
         newMembers.add(member);
         latestView = newMembers;
       }
@@ -1041,11 +992,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     logger.warn("Membership: disregarding shunned member <{}>", m);
   }
 
-  @Override
-  public void processMessage(DistributionMessage msg) {
-    handleOrDeferMessage(msg);
-  }
-
   /**
    * Logic for processing a distribution message.
    * <p>
@@ -1058,12 +1004,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     InternalDistributedMember m = msg.getSender();
     boolean shunned = false;
 
-    // UDP messages received from surprise members will have partial IDs.
-    // Attempt to replace these with full IDs from the MembershipManager's view.
-    if (msg.getSender().isPartial()) {
-      replacePartialIdentifierInMessage(msg);
-    }
-
     // If this member is shunned or new, grab the latestViewWriteLock: update the appropriate data
     // structure.
     if (isShunnedOrNew(m)) {
@@ -1111,8 +1051,12 @@ public class GMSMembershipManager implements MembershipManager, Manager {
    */
   public void replacePartialIdentifierInMessage(DistributionMessage msg) {
     InternalDistributedMember sender = msg.getSender();
-    sender = this.services.getJoinLeave().getMemberID(sender.getNetMember());
-    if (sender.isPartial()) {
+    GMSMember oldID = ((GMSMemberAdapter) sender.getNetMember()).getGmsMember();
+    GMSMember newID = this.services.getJoinLeave().getMemberID(oldID);
+    if (newID != null && newID != oldID) {
+      sender.setNetMember(new GMSMemberAdapter(newID));
+      sender.setIsPartial(false);
+    } else {
       // the DM's view also has surprise members, so let's check it as well
       sender = this.dcReceiver.getDM().getCanonicalId(sender);
     }
@@ -1127,7 +1071,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
    *
    * @param viewArg the new view
    */
-  protected void handleOrDeferViewEvent(NetView viewArg) {
+  protected void handleOrDeferViewEvent(MembershipView viewArg) {
     if (this.isJoining) {
       // bug #44373 - queue all view messages while joining.
       // This is done under the latestViewLock, but we can't block here because
@@ -1156,11 +1100,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     }
   }
 
-  @Override
-  public void memberSuspected(InternalDistributedMember initiator,
-      InternalDistributedMember suspect, String reason) {
-    SuspectMember s = new SuspectMember(initiator, suspect, reason);
-    handleOrDeferSuspect(s);
+  private InternalDistributedMember gmsMemberToDMember(GMSMember gmsMember) {
+    return new InternalDistributedMember(new GMSMemberAdapter(gmsMember));
   }
 
   /**
@@ -1176,8 +1117,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
           return;
         }
       }
-      InternalDistributedMember suspect = suspectInfo.suspectedMember;
-      InternalDistributedMember who = suspectInfo.whoSuspected;
+      InternalDistributedMember suspect = gmsMemberToDMember(suspectInfo.suspectedMember);
+      InternalDistributedMember who = gmsMemberToDMember(suspectInfo.whoSuspected);
       this.suspectedMembers.put(suspect, Long.valueOf(System.currentTimeMillis()));
       try {
         listener.memberSuspect(suspect, who, suspectInfo.reason);
@@ -1336,41 +1277,21 @@ public class GMSMembershipManager implements MembershipManager, Manager {
    * {@link DistributedMember}s)
    */
   @Override
-  public NetView getView() {
+  public MembershipView getView() {
     // Grab the latest view under a mutex...
-    NetView v;
+    MembershipView v;
 
     latestViewReadLock.lock();
     v = latestView;
     latestViewReadLock.unlock();
 
-    NetView result = new NetView(v, v.getViewId());
+    MembershipView result = new MembershipView(v, v.getViewId());
 
     v.getMembers().stream().filter(this::isShunned).forEachOrdered(result::remove);
 
     return result;
   }
 
-  /**
-   * test hook
-   * <p>
-   * The lead member is the eldest member with partition detection enabled.
-   * <p>
-   * If no members have partition detection enabled, there will be no lead member and this method
-   * will return null.
-   *
-   * @return the lead member associated with the latest view
-   */
-  @Override
-  public DistributedMember getLeadMember() {
-    latestViewReadLock.lock();
-    try {
-      return latestView == null ? null : latestView.getLeadMember();
-    } finally {
-      latestViewReadLock.unlock();
-    }
-  }
-
   private boolean isJoining() {
     return this.isJoining;
   }
@@ -1393,7 +1314,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   @Override
   public boolean memberExists(DistributedMember m) {
     latestViewReadLock.lock();
-    NetView v = latestView;
+    MembershipView v = latestView;
     latestViewReadLock.unlock();
     return v.contains(m);
   }
@@ -1472,14 +1393,17 @@ public class GMSMembershipManager implements MembershipManager, Manager {
    * not the same as a SHUNNED member.
    */
   @Override
-  public void shutdownMessageReceived(InternalDistributedMember id, String reason) {
+  public void shutdownMessageReceived(DistributedMember id, String reason) {
     if (logger.isDebugEnabled()) {
       logger.debug("Membership: recording shutdown status of {}", id);
     }
     synchronized (this.shutdownMembers) {
       this.shutdownMembers.put(id, id);
-      services.getHealthMonitor().memberShutdown(id, reason);
-      services.getJoinLeave().memberShutdown(id, reason);
+      services.getHealthMonitor()
+          .memberShutdown(
+              ((GMSMemberAdapter) ((InternalDistributedMember) id).getNetMember()).getGmsMember(),
+              reason);
+      services.getJoinLeave().memberShutdown(getGMSMember((InternalDistributedMember) id), reason);
     }
   }
 
@@ -1490,38 +1414,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   }
 
   @Override
-  public void stop() {
-
-    // [bruce] Do not null out the channel w/o adding appropriate synchronization
-
-    logger.debug("MembershipManager closing");
-
-    if (directChannel != null) {
-      directChannel.disconnect(null);
-
-      if (address != null) {
-        // Make sure that channel information is consistent
-        // Probably not important in this particular case, but just
-        // to be consistent...
-        latestViewWriteLock.lock();
-        try {
-          destroyMember(address, "orderly shutdown");
-        } finally {
-          latestViewWriteLock.unlock();
-        }
-      }
-    }
-
-    if (cleanupTimer != null) {
-      cleanupTimer.cancel();
-    }
-
-    if (logger.isDebugEnabled()) {
-      logger.debug("Membership: channel closed");
-    }
-  }
-
-  @Override
   public void uncleanShutdown(String reason, final Exception e) {
     inhibitForcedDisconnectLogging(false);
 
@@ -1570,7 +1462,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     logger.warn("Membership: requesting removal of {}. Reason={}",
         new Object[] {mbr, reason});
     try {
-      services.getJoinLeave().remove((InternalDistributedMember) mbr, reason);
+      services.getJoinLeave().remove(getGMSMember((InternalDistributedMember) mbr), reason);
     } catch (RuntimeException e) {
       Throwable problem = e;
       if (services.getShutdownCause() != null) {
@@ -1600,14 +1492,14 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   }
 
   @Override
-  public void suspectMembers(Set<InternalDistributedMember> members, String reason) {
-    for (final InternalDistributedMember member : members) {
+  public void suspectMembers(Set<DistributedMember> members, String reason) {
+    for (final DistributedMember member : members) {
       verifyMember(member, reason);
     }
   }
 
   @Override
-  public void suspectMember(InternalDistributedMember mbr, String reason) {
+  public void suspectMember(DistributedMember mbr, String reason) {
     if (!this.shutdownInProgress && !this.shutdownMembers.containsKey(mbr)) {
       verifyMember(mbr, reason);
     }
@@ -1625,9 +1517,10 @@ public class GMSMembershipManager implements MembershipManager, Manager {
    * @return true if the member checks out
    */
   @Override
-  public boolean verifyMember(InternalDistributedMember mbr, String reason) {
+  public boolean verifyMember(DistributedMember mbr, String reason) {
     return mbr != null && memberExists(mbr)
-        && this.services.getHealthMonitor().checkIfAvailable(mbr, reason, false);
+        && this.services.getHealthMonitor()
+            .checkIfAvailable(getGMSMember((InternalDistributedMember) mbr), reason, false);
   }
 
   /**
@@ -1661,9 +1554,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
 
     int sentBytes;
     try {
-      sentBytes = directChannel.send(this, keys, content,
-          this.services.getConfig().getDistributionConfig().getAckWaitThreshold(),
-          this.services.getConfig().getDistributionConfig().getAckSevereAlertThreshold());
+      sentBytes =
+          directChannel.send(this, keys, content, ackWaitThreshold, ackSevereAlertThreshold);
 
       if (theStats != null) {
         theStats.incSentBytes(sentBytes);
@@ -1702,7 +1594,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
       // of the view, we have a serious error (bug36202).
 
       // grab a recent view, excluding shunned members
-      NetView view = services.getJoinLeave().getView();
+      GMSMembershipView view = services.getJoinLeave().getView();
 
       // Iterate through members and causes in tandem :-(
       Iterator it_mem = members.iterator();
@@ -1711,7 +1603,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
         InternalDistributedMember member = (InternalDistributedMember) it_mem.next();
         Throwable th = (Throwable) it_causes.next();
 
-        if (!view.contains(member) || (th instanceof ShunnedMemberException)) {
+        if (!view.contains(getGMSMember(member)) || (th instanceof ShunnedMemberException)) {
           continue;
         }
         logger.fatal(String.format("Failed to send message <%s> to member <%s> view, %s",
@@ -1732,6 +1624,13 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     return null;
   }
 
+  /**
+   * retrieve the GMS member ID held in a Geode InternalDistributedMember
+   */
+  private GMSMember getGMSMember(InternalDistributedMember member) {
+    return ((GMSMemberAdapter) member.getNetMember()).getGmsMember();
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -1742,27 +1641,18 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     return (this.hasJoined && !this.shutdownInProgress);
   }
 
-  /**
-   * Returns true if the distributed system is in the process of auto-reconnecting. Otherwise
-   * returns false.
-   */
-  @Override
-  public boolean isReconnectingDS() {
-    return this.wasReconnectingSystem && !this.reconnectCompleted;
-  }
-
   @Override
   public QuorumChecker getQuorumChecker() {
     if (!(services.isShutdownDueToForcedDisconnect())) {
       return null;
     }
-    if (this.quorumChecker != null) {
-      return this.quorumChecker;
+    if (quorumChecker != null) {
+      return quorumChecker;
     }
 
-    QuorumChecker impl = services.getMessenger().getQuorumChecker();
-    this.quorumChecker = impl;
-    return impl;
+    GMSQuorumChecker impl = services.getMessenger().getQuorumChecker();
+    quorumChecker = new GMSQuorumCheckerAdapter(impl);
+    return quorumChecker;
   }
 
   @Override
@@ -1840,7 +1730,11 @@ public class GMSMembershipManager implements MembershipManager, Manager {
 
     if (useMcast || tcpDisabled || sendViaMessenger) {
       checkAddressesForUUIDs(destinations);
-      result = services.getMessenger().send(msg);
+      Set<GMSMember> failures = services.getMessenger().send(new GMSMessageAdapter(msg));
+      if (failures == null || failures.size() == 0) {
+        return Collections.emptySet();
+      }
+      return gmsMemberCollectionToInternalDistributedMemberSet(failures);
     } else {
       result = directChannelSend(destinations, msg, theStats);
     }
@@ -1854,17 +1748,14 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   }
 
   void checkAddressesForUUIDs(InternalDistributedMember[] addresses) {
+    GMSMembershipView view = services.getJoinLeave().getView();
     for (int i = 0; i < addresses.length; i++) {
       InternalDistributedMember m = addresses[i];
       if (m != null) {
-        GMSMember id = (GMSMember) m.getNetMember();
+        GMSMemberAdapter adapter = (GMSMemberAdapter) m.getNetMember();
+        GMSMember id = adapter.getGmsMember();
         if (!id.hasUUID()) {
-          latestViewReadLock.lock();
-          try {
-            addresses[i] = latestView.getCanonicalID(addresses[i]);
-          } finally {
-            latestViewReadLock.unlock();
-          }
+          adapter.setGmsMember(view.getCanonicalID(id));
         }
       }
     }
@@ -1893,15 +1784,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     latestViewWriteLock.unlock();
   }
 
-  @Override
-  public boolean shutdownInProgress() {
-    // Impossible condition (bug36329): make sure that we check DM's
-    // view of shutdown here
-    ClusterDistributionManager dm = listener.getDM();
-    return shutdownInProgress || (dm != null && dm.shutdownInProgress());
-  }
-
-
   /**
    * Clean up and create consistent new view with member removed. No uplevel events are generated.
    *
@@ -1913,7 +1795,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     latestViewWriteLock.lock();
     try {
       if (latestView.contains(member)) {
-        NetView newView = new NetView(latestView, latestView.getViewId());
+        MembershipView newView = new MembershipView(latestView, latestView.getViewId());
         newView.remove(member);
         latestView = newView;
       }
@@ -2148,11 +2030,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     this.reconnectCompleted = reconnectCompleted;
   }
 
-  @Override
-  public boolean isReconnectCompleted() {
-    return reconnectCompleted;
-  }
-
 
   /*
    * non-thread-owned serial channels and high priority channels are not included
@@ -2164,7 +2041,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     if (dc != null) {
       dc.getChannelStates(member, result);
     }
-    services.getMessenger().getMessageState((InternalDistributedMember) member, result,
+    services.getMessenger().getMessageState(getGMSMember((InternalDistributedMember) member),
+        result,
         includeMulticast);
     return result;
   }
@@ -2178,7 +2056,8 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     if (dc != null) {
       dc.waitForChannelState(otherMember, state);
     }
-    services.getMessenger().waitForMessageState((InternalDistributedMember) otherMember, state);
+    services.getMessenger().waitForMessageState(
+        getGMSMember((InternalDistributedMember) otherMember), state);
 
     if (services.getConfig().isMulticastEnabled()
         && !services.getConfig().getDistributionConfig().getDisableTcp()) {
@@ -2294,10 +2173,18 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     return result;
   }
 
+  @Override
+  public boolean shutdownInProgress() {
+    // Impossible condition (bug36329): make sure that we check DM's
+    // view of shutdown here
+    ClusterDistributionManager dm = listener.getDM();
+    return shutdownInProgress || (dm != null && dm.shutdownInProgress());
+  }
+
 
   // TODO GEODE-1752 rewrite this to get rid of the latches, which are currently a memory leak
   @Override
-  public boolean waitForNewMember(InternalDistributedMember remoteId) {
+  public boolean waitForNewMember(DistributedMember remoteId) {
     boolean foundRemoteId = false;
     CountDownLatch currentLatch = null;
     // ARB: preconditions
@@ -2475,72 +2362,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     }
   }
 
-  @Override
-  public void stopped() {}
-
-  @Override
-  public void installView(NetView v) {
-    if (latestViewId < 0 && !isConnected()) {
-      latestViewId = v.getViewId();
-      latestView = v;
-      logger.debug("MembershipManager: initial view is {}", latestView);
-    } else {
-      handleOrDeferViewEvent(v);
-    }
-  }
-
-  @Override
-  public Set<InternalDistributedMember> send(DistributionMessage m)
-      throws NotSerializableException {
-    return send(m.getRecipients(), m, this.services.getStatistics());
-  }
-
-  @Override
-  public void forceDisconnect(final String reason) {
-    if (GMSMembershipManager.this.shutdownInProgress || isJoining()) {
-      return; // probably a race condition
-    }
-
-    setShutdown();
-
-    final Exception shutdownCause = new ForcedDisconnectException(reason);
-
-    // cache the exception so it can be appended to ShutdownExceptions
-    services.setShutdownCause(shutdownCause);
-    services.getCancelCriterion().cancel(reason);
-
-    AlertAppender.getInstance().stopSession();
-
-    if (!inhibitForceDisconnectLogging) {
-      logger.fatal(
-          String.format("Membership service failure: %s", reason),
-          shutdownCause);
-    }
-
-    if (this.isReconnectingDS()) {
-      logger.info("Reconnecting system failed to connect");
-      uncleanShutdown(reason,
-          new ForcedDisconnectException("reconnecting system failed to connect"));
-      return;
-    }
-
-    listener.saveConfig();
-
-    Thread reconnectThread = new LoggingThread("DisconnectThread", false, () -> {
-      // stop server locators immediately since they may not have correct
-      // information. This has caused client failures in bridge/wan
-      // network-down testing
-      InternalLocator loc = (InternalLocator) Locator.getLocator();
-      if (loc != null) {
-        loc.stop(true, !services.getConfig().getDistributionConfig().getDisableAutoReconnect(),
-            false);
-      }
-      uncleanShutdown(reason, shutdownCause);
-    });
-    reconnectThread.start();
-  }
-
-
   public void disableDisconnectOnQuorumLossForTesting() {
     services.getJoinLeave().disableDisconnectOnQuorumLossForTesting();
   }
@@ -2569,12 +2390,6 @@ public class GMSMembershipManager implements MembershipManager, Manager {
 
 
   @Override
-  public boolean isShutdownStarted() {
-    ClusterDistributionManager dm = listener.getDM();
-    return shutdownInProgress || (dm != null && dm.isCloseInProgress());
-  }
-
-  @Override
   public void disconnect(boolean beforeJoined) {
     if (beforeJoined) {
       uncleanShutdown("Failed to start distribution", null);
@@ -2582,4 +2397,320 @@ public class GMSMembershipManager implements MembershipManager, Manager {
       shutdown();
     }
   }
+
+
+  class ManagerImpl implements Manager {
+
+    @Override
+    public Services getServices() {
+      return services;
+    }
+
+    @Override
+    /* Service interface */
+    public void init(Services services) {
+      GMSMembershipManager.this.services = services;
+
+      Assert.assertTrue(services != null);
+
+      DistributionConfig config = services.getConfig().getDistributionConfig();
+      RemoteTransportConfig transport = services.getConfig().getTransport();
+
+      membershipCheckTimeout = config.getSecurityPeerMembershipTimeout();
+      wasReconnectingSystem = transport.getIsReconnectingDS();
+
+      // cache these settings for use in send()
+      mcastEnabled = transport.isMcastEnabled();
+      tcpDisabled = transport.isTcpDisabled();
+      ackSevereAlertThreshold = config.getAckSevereAlertThreshold();
+      ackWaitThreshold = config.getAckWaitThreshold();
+
+      if (!tcpDisabled) {
+        dcReceiver = new MyDCReceiver(listener);
+      }
+
+      surpriseMemberTimeout =
+          Math.max(20 * DistributionConfig.DEFAULT_MEMBER_TIMEOUT, 20 * config.getMemberTimeout());
+      surpriseMemberTimeout =
+          Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "surprise-member-timeout",
+              surpriseMemberTimeout).intValue();
+
+    }
+
+    /* Service interface */
+    @Override
+    public void start() {
+      DistributionConfig config = services.getConfig().getDistributionConfig();
+
+      int dcPort = 0;
+      if (!tcpDisabled) {
+        directChannel = new DirectChannel(GMSMembershipManager.this, dcReceiver, config);
+        dcPort = directChannel.getPort();
+      }
+      services.getMessenger().getMemberID().setDirectPort(dcPort);
+    }
+
+    /* Service interface */
+    @Override
+    public void started() {
+      startCleanupTimer();
+      // see if a locator was started and put it in GMS Services
+      InternalLocator l = (InternalLocator) org.apache.geode.distributed.Locator.getLocator();
+      if (l != null && l.getLocatorHandler() != null) {
+        if (l.getLocatorHandler().setServices(services)) {
+          services.setLocator(((GMSLocatorAdapter) l.getLocatorHandler()).getGMSLocator());
+        }
+      }
+    }
+
+    /* Service interface */
+    @Override
+    public void stop() {
+      // [bruce] Do not null out the channel w/o adding appropriate synchronization
+
+      logger.debug("MembershipManager closing");
+
+      if (directChannel != null) {
+        directChannel.disconnect(null);
+
+        if (address != null) {
+          // Make sure that channel information is consistent
+          // Probably not important in this particular case, but just
+          // to be consistent...
+          latestViewWriteLock.lock();
+          try {
+            destroyMember(address, "orderly shutdown");
+          } finally {
+            latestViewWriteLock.unlock();
+          }
+        }
+      }
+
+      if (cleanupTimer != null) {
+        cleanupTimer.cancel();
+      }
+
+      if (logger.isDebugEnabled()) {
+        logger.debug("Membership: channel closed");
+      }
+    }
+
+    /* Service interface */
+    @Override
+    public void stopped() {}
+
+    /* Service interface */
+    @Override
+    public void installView(GMSMembershipView v) {
+      if (latestViewId < 0 && !isConnected()) {
+        latestViewId = v.getViewId();
+        latestView = createGeodeView(v);
+        logger.debug("MembershipManager: initial view is {}", latestView);
+      } else {
+        handleOrDeferViewEvent(createGeodeView(v));
+      }
+    }
+
+    @Override
+    public void beSick() {
+      // no-op
+    }
+
+    @Override
+    public void playDead() {
+      // no-op
+    }
+
+    @Override
+    public void beHealthy() {
+      // no-op
+    }
+
+    @Override
+    public void emergencyClose() {
+      // no-op
+    }
+
+
+    @Override
+    public void joinDistributedSystem() {
+      long startTime = System.currentTimeMillis();
+
+      try {
+        join();
+      } catch (RuntimeException e) {
+        if (directChannel != null) {
+          directChannel.disconnect(e);
+        }
+        throw e;
+      }
+
+      GMSMembershipManager.this.address =
+          new InternalDistributedMember(
+              new GMSMemberAdapter(services.getMessenger().getMemberID()));
+
+      if (directChannel != null) {
+        directChannel.setLocalAddr(address);
+      }
+
+      GMSMembershipManager.this.hasJoined = true;
+
+      // in order to debug startup issues we need to announce the membership
+      // ID as soon as we know it
+      logger.info("Finished joining (took {}ms).",
+          "" + (System.currentTimeMillis() - startTime));
+
+    }
+
+    @Override
+    public void memberSuspected(GMSMember initiator,
+        GMSMember suspect, String reason) {
+      SuspectMember s = new SuspectMember(initiator, suspect, reason);
+      handleOrDeferSuspect(s);
+    }
+
+
+    @Override
+    public void forceDisconnect(final String reason) {
+      if (GMSMembershipManager.this.shutdownInProgress || isJoining()) {
+        return; // probably a race condition
+      }
+
+      setShutdown();
+
+      final Exception shutdownCause = new ForcedDisconnectException(reason);
+
+      // cache the exception so it can be appended to ShutdownExceptions
+      services.setShutdownCause(shutdownCause);
+      services.getCancelCriterion().cancel(reason);
+
+      AlertAppender.getInstance().stopSession();
+
+      if (!inhibitForceDisconnectLogging) {
+        logger.fatal(
+            String.format("Membership service failure: %s", reason),
+            shutdownCause);
+      }
+
+      if (this.isReconnectingDS()) {
+        logger.info("Reconnecting system failed to connect");
+        uncleanShutdown(reason,
+            new ForcedDisconnectException("reconnecting system failed to connect"));
+        return;
+      }
+
+      listener.saveConfig();
+
+      Thread reconnectThread = new LoggingThread("DisconnectThread", false, () -> {
+        // stop server locators immediately since they may not have correct
+        // information. This has caused client failures in bridge/wan
+        // network-down testing
+        InternalLocator loc = (InternalLocator) Locator.getLocator();
+        if (loc != null) {
+          loc.stop(true, !services.getConfig().getDistributionConfig().getDisableAutoReconnect(),
+              false);
+        }
+        uncleanShutdown(reason, shutdownCause);
+      });
+      reconnectThread.start();
+    }
+
+
+    /** this is invoked by JoinLeave when there is a loss of quorum in the membership system */
+    @Override
+    public void quorumLost(Collection<GMSMember> failures, GMSMembershipView view) {
+      // notify of quorum loss if split-brain detection is enabled (meaning we'll shut down) or
+      // if the loss is more than one member
+
+      boolean notify = failures.size() > 1;
+      if (!notify) {
+        notify = services.getConfig().isNetworkPartitionDetectionEnabled();
+      }
+
+      if (notify) {
+        List<InternalDistributedMember> remaining =
+            gmsMemberListToInternalDistributedMemberList(view.getMembers());
+        remaining.removeAll(failures);
+
+        if (inhibitForceDisconnectLogging) {
+          if (logger.isDebugEnabled()) {
+            logger
+                .debug("<ExpectedException action=add>Possible loss of quorum</ExpectedException>");
+          }
+        }
+        logger.fatal("Possible loss of quorum due to the loss of {} cache processes: {}",
+            failures.size(), failures);
+        if (inhibitForceDisconnectLogging) {
+          if (logger.isDebugEnabled()) {
+            logger.debug(
+                "<ExpectedException action=remove>Possible loss of quorum</ExpectedException>");
+          }
+        }
+
+
+        try {
+          listener.quorumLost(
+              gmsMemberCollectionToInternalDistributedMemberSet(failures),
+              remaining);
+        } catch (CancelException e) {
+          // safe to ignore - a forced disconnect probably occurred
+        }
+      }
+    }
+
+    @Override
+    public void processMessage(GMSMessage msg) {
+      DistributionMessage distributionMessage =
+          (DistributionMessage) ((GMSMessageAdapter) msg).getGeodeMessage();
+      // UDP messages received from surprise members will have partial IDs.
+      // Attempt to replace these with full IDs from the MembershipManager's view.
+      if (distributionMessage.getSender().isPartial()) {
+        replacePartialIdentifierInMessage(distributionMessage);
+      }
+
+      handleOrDeferMessage(distributionMessage);
+    }
+
+    @Override
+    public boolean isMulticastAllowed() {
+      return !disableMulticastForRollingUpgrade;
+    }
+
+    @Override
+    public boolean shutdownInProgress() {
+      // Impossible condition (bug36329): make sure that we check DM's
+      // view of shutdown here
+      ClusterDistributionManager dm = listener.getDM();
+      return shutdownInProgress || (dm != null && dm.shutdownInProgress());
+    }
+
+    @Override
+    public boolean isReconnectingDS() {
+      return wasReconnectingSystem && !reconnectCompleted;
+    }
+
+    @Override
+    public boolean isShutdownStarted() {
+      ClusterDistributionManager dm = listener.getDM();
+      return shutdownInProgress || (dm != null && dm.isCloseInProgress());
+    }
+
+    @Override
+    public GMSMessage wrapMessage(Object receivedMessage) {
+      if (receivedMessage instanceof GMSMessage) {
+        return (GMSMessage) receivedMessage;
+      }
+      // Geode's DistributionMessage class isn't known by GMS classes
+      return new GMSMessageAdapter((DistributionMessage) receivedMessage);
+    }
+
+    @Override
+    public DataSerializableFixedID unwrapMessage(GMSMessage messageToSend) {
+      if (messageToSend instanceof GMSMessageAdapter) {
+        return ((GMSMessageAdapter) messageToSend).getGeodeMessage();
+      }
+      return (DataSerializableFixedID) messageToSend;
+    }
+
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java
new file mode 100644
index 0000000..fc0178c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java
@@ -0,0 +1,128 @@
+/*
+ * 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.distributed.internal.membership.adapter;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
+import org.apache.geode.internal.DataSerializableFixedID;
+import org.apache.geode.internal.cache.DirectReplyMessage;
+
+/**
+ * GMSMessageAdapter wraps a Geode DistributionMessage to be sent via the GMS Messenger
+ */
+
+public class GMSMessageAdapter implements GMSMessage {
+  DistributionMessage geodeMessage;
+
+  public GMSMessageAdapter(DistributionMessage geodeMessage) {
+    this.geodeMessage = geodeMessage;
+  }
+
+  @Override
+  public void setRecipient(GMSMember member) {
+    geodeMessage.setRecipient(new InternalDistributedMember(new GMSMemberAdapter(member)));
+  }
+
+  @Override
+  public void setRecipients(List<GMSMember> recipients) {
+    geodeMessage.setRecipients(
+        recipients.stream().map(GMSMemberAdapter::new).map(InternalDistributedMember::new).collect(
+            Collectors.toList()));
+  }
+
+  @Override
+  public boolean isHighPriority() {
+    return geodeMessage instanceof HighPriorityDistributionMessage ||
+        geodeMessage.getProcessorType() == ClusterDistributionManager.HIGH_PRIORITY_EXECUTOR;
+  }
+
+  @Override
+  public void registerProcessor() {
+    if (geodeMessage instanceof DirectReplyMessage) {
+      ((DirectReplyMessage) geodeMessage).registerProcessor();
+    }
+  }
+
+  @Override
+  public List<GMSMember> getRecipients() {
+    InternalDistributedMember[] recipients = geodeMessage.getRecipients();
+    if (recipients == null
+        || recipients.length == 1 && recipients[0] == DistributionMessage.ALL_RECIPIENTS) {
+      return Collections.singletonList(null);
+    }
+    return Arrays.asList(recipients).stream()
+        .map(recipient -> (GMSMemberAdapter) recipient.getNetMember())
+        .map(GMSMemberAdapter::getGmsMember).collect(
+            Collectors.toList());
+  }
+
+  @Override
+  public int getDSFID() {
+    return geodeMessage.getDSFID();
+  }
+
+  @Override
+  public boolean forAll() {
+    return geodeMessage.forAll();
+  }
+
+  @Override
+  public boolean getMulticast() {
+    return geodeMessage.getMulticast();
+  }
+
+  @Override
+  public void setMulticast(boolean useMulticast) {
+    geodeMessage.setMulticast(useMulticast);
+  }
+
+  @Override
+  public void setSender(GMSMember sender) {
+    geodeMessage.setSender(new InternalDistributedMember(new GMSMemberAdapter(sender)));
+  }
+
+  @Override
+  public GMSMember getSender() {
+    return ((GMSMemberAdapter) geodeMessage.getSender().getNetMember()).getGmsMember();
+  }
+
+  @Override
+  public void resetTimestamp() {
+    geodeMessage.resetTimestamp();
+  }
+
+  @Override
+  public void setBytesRead(int amount) {
+    geodeMessage.setBytesRead(amount);
+  }
+
+  @Override
+  public String toString() {
+    return geodeMessage.toString();
+  }
+
+  public DataSerializableFixedID getGeodeMessage() {
+    return geodeMessage;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSQuorumCheckerAdapter.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSQuorumCheckerAdapter.java
new file mode 100644
index 0000000..6eb707d
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSQuorumCheckerAdapter.java
@@ -0,0 +1,53 @@
+/*
+ * 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.distributed.internal.membership.adapter;
+
+import org.apache.geode.distributed.internal.membership.QuorumChecker;
+import org.apache.geode.distributed.internal.membership.gms.messenger.GMSQuorumChecker;
+import org.apache.geode.distributed.internal.membership.gms.messenger.MembershipInformation;
+
+public class GMSQuorumCheckerAdapter implements QuorumChecker {
+  private final GMSQuorumChecker quorumChecker;
+
+  public GMSQuorumCheckerAdapter(GMSQuorumChecker delegate) {
+
+    quorumChecker = delegate;
+  }
+
+  @Override
+  public boolean checkForQuorum(long timeoutMS) throws InterruptedException {
+    return quorumChecker.checkForQuorum(timeoutMS);
+  }
+
+  @Override
+  public void suspend() {
+    quorumChecker.suspend();
+  }
+
+  @Override
+  public void resume() {
+    quorumChecker.resume();
+  }
+
+  @Override
+  public void close() {
+    quorumChecker.close();
+  }
+
+  @Override
+  public MembershipInformation getMembershipInfo() {
+    return quorumChecker.getMembershipInfo();
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/LocalViewMessage.java
similarity index 92%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/LocalViewMessage.java
index 0c419b4..3d482bf 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/LocalViewMessage.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.distributed.internal.membership.gms.mgr;
+package org.apache.geode.distributed.internal.membership.adapter;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.MembershipView;
 
 
 /**
@@ -35,9 +35,9 @@ public class LocalViewMessage extends SerialDistributionMessage {
 
   private GMSMembershipManager manager;
   private long viewId;
-  private NetView view;
+  private MembershipView view;
 
-  public LocalViewMessage(InternalDistributedMember addr, long viewId, NetView view,
+  public LocalViewMessage(InternalDistributedMember addr, long viewId, MembershipView view,
       GMSMembershipManager manager) {
     super();
     this.sender = addr;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java
index e49a048..4eb6cf4 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java
@@ -23,8 +23,9 @@ import java.util.Properties;
 import org.apache.commons.lang3.StringUtils;
 
 import org.apache.geode.LogWriter;
-import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Authenticator;
 import org.apache.geode.internal.cache.tier.sockets.Handshake;
 import org.apache.geode.internal.security.CallbackInstantiator;
@@ -61,14 +62,14 @@ public class GMSAuthenticator implements Authenticator {
    *         failure message
    */
   @Override
-  public String authenticate(InternalDistributedMember member, Properties credentials) {
+  public String authenticate(GMSMember member, Properties credentials) {
     return authenticate(member, credentials, this.securityProps);
   }
 
   /**
    * Method is package protected to be used in testing.
    */
-  String authenticate(DistributedMember member, Properties credentials, Properties secProps) {
+  String authenticate(GMSMember member, Properties credentials, Properties secProps) {
 
     // For older systems, locator might be started without cache, so secureService may not be
     // initialized here. We need to check if the passed in secProps has peer authenticator or not at
@@ -104,7 +105,7 @@ public class GMSAuthenticator implements Authenticator {
   /**
    * Method is package protected to be used in testing.
    */
-  Principal invokeAuthenticator(Properties securityProps, DistributedMember member,
+  Principal invokeAuthenticator(Properties securityProps, GMSMember member,
       Properties credentials) throws AuthenticationFailedException {
     String authMethod = securityProps.getProperty(SECURITY_PEER_AUTHENTICATOR);
     org.apache.geode.security.Authenticator auth = null;
@@ -115,7 +116,8 @@ public class GMSAuthenticator implements Authenticator {
       // this.securityProps contains security-ldap-basedn but security-ldap-baseDomainName is
       // expected
       auth.init(this.securityProps, logWriter, securityLogWriter);
-      return auth.authenticate(credentials, member);
+      return auth.authenticate(credentials,
+          new InternalDistributedMember(new GMSMemberAdapter(member)));
 
     } catch (GemFireSecurityException gse) {
       throw gse;
@@ -137,7 +139,7 @@ public class GMSAuthenticator implements Authenticator {
    * @return the credentials
    */
   @Override
-  public Properties getCredentials(InternalDistributedMember member) {
+  public Properties getCredentials(GMSMember member) {
     try {
       return getCredentials(member, securityProps);
 
@@ -153,9 +155,11 @@ public class GMSAuthenticator implements Authenticator {
   /**
    * For testing only.
    */
-  Properties getCredentials(DistributedMember member, Properties secProps) {
+  Properties getCredentials(GMSMember member, Properties secProps) {
     String authMethod = secProps.getProperty(SECURITY_PEER_AUTH_INIT);
-    return Handshake.getCredentials(authMethod, secProps, member, true,
+    return Handshake.getCredentials(authMethod, secProps,
+        new InternalDistributedMember(new GMSMemberAdapter(member)),
+        true,
         logWriter,
         securityLogWriter);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java
index dabb952..bacae83 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java
@@ -19,31 +19,34 @@ import java.io.DataOutput;
 import java.io.EOFException;
 import java.io.IOException;
 import java.net.InetAddress;
-import java.net.UnknownHostException;
 
 import org.jgroups.util.UUID;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.distributed.internal.membership.MemberAttributes;
-import org.apache.geode.distributed.internal.membership.NetMember;
+import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
+import org.apache.geode.internal.net.SocketCreator;
 
 /**
- * This is the fundamental representation of a member of a GemFire distributed system.
- *
- * Unfortunately, this class serves two distinct functions. First, it is the fundamental element of
- * membership in the GemFire distributed system. As such, it is used in enumerations and properly
- * responds to hashing and equals() comparisons.
- *
- * Second, it is used as a cheap way of representing an address. This is unfortunate, because as a
- * NetMember, it holds two separate port numbers: the "membership" descriptor as well as a direct
- * communication channel.
- *
+ * GMSMember is the membership identifier class for Group Membership Services.
  */
-public class GMSMember implements NetMember, DataSerializableFixedID {
+public class GMSMember implements DataSerializableFixedID {
+  /** The type for regular members */
+  public static final int NORMAL_DM_TYPE = 10;
+
+  /** The DM type for locator members */
+  public static final int LOCATOR_DM_TYPE = 11;
+
+  /** The DM type for deprecated admin-only members */
+  public static final int ADMIN_ONLY_DM_TYPE = 12;
+
+  /** The DM type for stand-alone members (usually clients) */
+  public static final int LONER_DM_TYPE = 13;
+
+  private String hostName;
+
   private int udpPort = 0;
   private boolean preferredForCoordinator;
   private boolean networkPartitionDetectionEnabled;
@@ -54,37 +57,31 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
   private int vmViewId = -1;
   private int directPort;
   private String name;
-  private DurableClientAttributes durableClientAttributes;
   private String[] groups;
   private short versionOrdinal = Version.CURRENT_ORDINAL;
   private long uuidLSBs;
   private long uuidMSBs;
-
+  private String durableId;
+  private int durableTimeout;
 
 
   // Used only by Externalization
   public GMSMember() {}
 
-  @Override
-  public MemberAttributes getAttributes() {
-    return new MemberAttributes(directPort, processId, vmKind, vmViewId, name, groups,
-        durableClientAttributes);
+  @VisibleForTesting
+  public GMSMember(String localhost, int udpPort, Version version) {
+    this.hostName = localhost;
+    this.inetAddr = SocketCreator.toInetAddress(localhost);
+    this.udpPort = udpPort;
+    this.versionOrdinal = version.ordinal();
+    this.vmKind = NORMAL_DM_TYPE;
+    this.preferredForCoordinator = true;
+    this.vmViewId = -1;
+    this.processId = -1;
+    this.directPort = -1;
+    setUUID(UUID.randomUUID());
   }
 
-  @Override
-  public void setAttributes(MemberAttributes p_attr) {
-    MemberAttributes attr = p_attr;
-    if (attr == null) {
-      attr = MemberAttributes.INVALID;
-    }
-    processId = attr.getVmPid();
-    vmKind = (byte) attr.getVmKind();
-    directPort = attr.getPort();
-    vmViewId = attr.getVmViewId();
-    name = attr.getName();
-    groups = attr.getGroups();
-    durableClientAttributes = attr.getDurableClientAttributes();
-  }
 
   /**
    * Create a CacheMember referring to the current host (as defined by the given string).
@@ -92,13 +89,9 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
    * @param i the hostname, must be for the current host
    * @param p the membership listening port
    */
+  @VisibleForTesting
   public GMSMember(String i, int p) {
-    udpPort = p;
-    try {
-      inetAddr = InetAddress.getByName(i);
-    } catch (UnknownHostException e) {
-      // oops
-    }
+    this(i, p, Version.CURRENT);
   }
 
   /**
@@ -113,12 +106,23 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
    * @param msbs - most significant bytes of UUID
    * @param lsbs - least significant bytes of UUID
    */
-  public GMSMember(MemberAttributes attr, InetAddress i, int p,
+  public GMSMember(InetAddress i, String hostName, int p, int processId, byte vmKind,
+      int directPort, int vmViewId,
+      String name, String[] groups,
+      String durableId, int durableTimeout,
       boolean networkPartitionDetectionEnabled, boolean preferredForCoordinator, short version,
       long msbs, long lsbs) {
-    setAttributes(attr);
     this.inetAddr = i;
+    this.hostName = hostName;
     this.udpPort = p;
+    this.processId = processId;
+    this.vmKind = vmKind;
+    this.directPort = directPort;
+    this.vmViewId = vmViewId;
+    this.name = name;
+    this.groups = groups;
+    this.durableId = durableId;
+    this.durableTimeout = durableTimeout;
     this.networkPartitionDetectionEnabled = networkPartitionDetectionEnabled;
     this.preferredForCoordinator = preferredForCoordinator;
     this.versionOrdinal = version;
@@ -128,6 +132,7 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
 
   public GMSMember(InetAddress i, int p, short version, long msbs, long lsbs, int viewId) {
     this.inetAddr = i;
+    this.hostName = i.getHostName();
     this.udpPort = p;
     this.versionOrdinal = version;
     this.uuidMSBs = msbs;
@@ -142,6 +147,7 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
    * @param other the member to create a copy of
    */
   public GMSMember(GMSMember other) {
+    this.hostName = other.hostName;
     this.udpPort = other.udpPort;
     this.preferredForCoordinator = other.preferredForCoordinator;
     this.networkPartitionDetectionEnabled = other.networkPartitionDetectionEnabled;
@@ -152,39 +158,43 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     this.vmViewId = other.vmViewId;
     this.directPort = other.directPort;
     this.name = other.name;
-    this.durableClientAttributes = other.durableClientAttributes;
+    this.durableId = other.durableId;
+    this.durableTimeout = other.durableTimeout;
     this.groups = other.groups;
     this.versionOrdinal = other.versionOrdinal;
     this.uuidLSBs = other.uuidLSBs;
     this.uuidMSBs = other.uuidMSBs;
   }
 
-  @Override
+
   public int getPort() {
     return this.udpPort;
   }
 
-  @Override
-  public boolean isMulticastAddress() {
-    return false;
-  }
 
-  @Override
   public boolean preferredForCoordinator() {
     return this.preferredForCoordinator;
   }
 
-  @Override
+
   public void setPreferredForCoordinator(boolean preferred) {
     this.preferredForCoordinator = preferred;
   }
 
-  @Override
+
+  public String getDurableId() {
+    return durableId;
+  }
+
+  public int getDurableTimeout() {
+    return durableTimeout;
+  }
+
   public InetAddress getInetAddress() {
     return this.inetAddr;
   }
 
-  @Override
+
   public short getVersionOrdinal() {
     return this.versionOrdinal;
   }
@@ -229,18 +239,22 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
    * @exception java.lang.ClassCastException - if the specified object's type prevents it from being
    * compared to this Object.
    */
-  @Override
-  public int compareTo(NetMember o) {
+
+  public int compareTo(GMSMember o) {
+    return compareTo(o, true);
+  }
+
+  public int compareTo(GMSMember o, boolean compareUUIDs) {
     if (o == this) {
       return 0;
     }
     // obligatory type check
-    if (o == null || !(o instanceof GMSMember)) {
+    if (o == null) {
       throw new ClassCastException(
-          "NetMember.compareTo(): comparison between different classes");
+          "GMSMember.compareTo(): comparison between different classes");
     }
     byte[] myAddr = inetAddr.getAddress();
-    GMSMember his = (GMSMember) o;
+    GMSMember his = o;
     byte[] hisAddr = his.inetAddr.getAddress();
     if (myAddr != hisAddr) {
       for (int idx = 0; idx < myAddr.length; idx++) {
@@ -273,7 +287,7 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
         result = 1;
       }
     }
-    if (result == 0 && this.uuidMSBs != 0 && his.uuidMSBs != 0) {
+    if (compareUUIDs && result == 0 && this.uuidMSBs != 0 && his.uuidMSBs != 0) {
       if (this.uuidMSBs < his.uuidMSBs) {
         result = -1;
       } else if (his.uuidMSBs < this.uuidMSBs) {
@@ -287,9 +301,8 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     return result;
   }
 
-  @Override
-  public int compareAdditionalData(NetMember other) {
-    GMSMember his = (GMSMember) other;
+
+  public int compareAdditionalData(GMSMember his) {
     int result = 0;
     if (this.uuidMSBs != 0 && his.uuidMSBs != 0) {
       if (this.uuidMSBs < his.uuidMSBs) {
@@ -328,31 +341,19 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     String uuid = formatUUID();
 
     sb.append("GMSMember[addr=").append(inetAddr).append(";port=").append(udpPort)
-        .append(";processId=").append(processId).append(";name=").append(name).append(uuid)
+        .append(";kind=").append(vmKind).append(";processId=").append(";viewId=").append(vmViewId)
+        .append(processId).append(";v").append(versionOrdinal).append(";name=")
+        .append(name).append(uuid).append(";weight=").append(memberWeight)
         .append("]");
     return sb.toString();
   }
 
-  @Override
-  public String getUniqueId() {
-    StringBuilder sb = new StringBuilder(100);
-    sb.append("GMSMember[addr=").append(inetAddr);
-    sb.append(";processId=").append(processId);
-    sb.append(";name=").append(name);
-    sb.append(formatUUID()).append("]");
-    return sb.toString();
-  }
-
-  public int getUdpPort() {
-    return udpPort;
-  }
 
-  @Override
   public boolean isNetworkPartitionDetectionEnabled() {
     return networkPartitionDetectionEnabled;
   }
 
-  @Override
+
   public byte getMemberWeight() {
     return memberWeight;
   }
@@ -361,40 +362,36 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     return inetAddr;
   }
 
-  @Override
+
   public int getProcessId() {
     return processId;
   }
 
-  @Override
+
   public byte getVmKind() {
     return vmKind;
   }
 
-  @Override
+
   public int getVmViewId() {
     return vmViewId;
   }
 
-  @Override
+
   public void setVmViewId(int id) {
     this.vmViewId = id;
   }
 
-  @Override
+
   public int getDirectPort() {
     return directPort;
   }
 
-  @Override
+
   public String getName() {
     return name;
   }
 
-  @Override
-  public DurableClientAttributes getDurableClientAttributes() {
-    return durableClientAttributes;
-  }
 
   public String[] getRoles() {
     return groups;
@@ -404,7 +401,7 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     this.udpPort = udpPort;
   }
 
-  @Override
+
   public void setNetworkPartitionDetectionEnabled(boolean networkPartitionDetectionEnabled) {
     this.networkPartitionDetectionEnabled = networkPartitionDetectionEnabled;
   }
@@ -417,17 +414,17 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     this.inetAddr = inetAddr;
   }
 
-  @Override
+
   public void setProcessId(int processId) {
     this.processId = processId;
   }
 
-  @Override
+
   public void setVmKind(int vmKind) {
     this.vmKind = (byte) vmKind;
   }
 
-  @Override
+
   public void setVersion(Version v) {
     this.versionOrdinal = v.ordinal();
   }
@@ -436,32 +433,27 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     this.vmViewId = birthViewId;
   }
 
-  @Override
+
   public void setDirectPort(int directPort) {
     this.directPort = directPort;
   }
 
-  @Override
+
   public void setName(String name) {
     this.name = name;
   }
 
-  @Override
-  public void setDurableClientAttributes(DurableClientAttributes durableClientAttributes) {
-    this.durableClientAttributes = durableClientAttributes;
-  }
 
-  @Override
   public String[] getGroups() {
     return groups;
   }
 
-  @Override
+
   public void setGroups(String[] groups) {
     this.groups = groups;
   }
 
-  @Override
+
   public void setPort(int p) {
     this.udpPort = p;
   }
@@ -485,17 +477,49 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
 
   static final int NPD_ENABLED_BIT = 0x01;
   static final int PREFERRED_FOR_COORD_BIT = 0x02;
+  static final int VERSION_BIT = 0x8;
+
+  static final int LONER_VM_TYPE = 13; // from ClusterDistributionManager
 
   @Override
   public void toData(DataOutput out) throws IOException {
-    writeEssentialData(out);
-    out.writeInt(directPort);
-    out.writeByte(memberWeight);
+    DataSerializer.writeInetAddress(getInetAddress(), out);
+    out.writeInt(getPort());
+
+    DataSerializer.writeString(hostName, out);
+
+    int flags = 0;
+    if (isNetworkPartitionDetectionEnabled())
+      flags |= NPD_ENABLED_BIT;
+    if (preferredForCoordinator())
+      flags |= PREFERRED_FOR_COORD_BIT;
+    // always write product version but enable reading from older versions
+    // that do not have it
+    flags |= VERSION_BIT;
+
+    out.writeByte((byte) (flags & 0xff));
+
+    out.writeInt(getDirectPort());
+    out.writeInt(getProcessId());
+    int vmKind = getVmKind();
     out.writeByte(vmKind);
-    out.writeInt(processId);
+    DataSerializer.writeStringArray(getGroups(), out);
+
+    DataSerializer.writeString(getName(), out);
+    if (vmKind == LONER_VM_TYPE) {
+      DataSerializer.writeString("", out);
+    } else { // added in 6.5 for unique identifiers in P2P
+      DataSerializer.writeString(String.valueOf(getVmViewId()), out);
+    }
+    DataSerializer
+        .writeString(durableId == null ? "" : durableId, out);
+    DataSerializer.writeInteger(durableId == null ? 300 : durableTimeout, out);
 
-    DataSerializer.writeString(name, out);
-    DataSerializer.writeStringArray(groups, out);
+    Version.writeOrdinal(out, versionOrdinal, true);
+
+    if (versionOrdinal >= Version.GFE_90.ordinal()) {
+      writeAdditionalData(out);
+    }
   }
 
   public void writeEssentialData(DataOutput out) throws IOException {
@@ -520,14 +544,60 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
 
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    readEssentialData(in);
-    this.directPort = in.readInt();
-    this.memberWeight = in.readByte();
-    this.vmKind = in.readByte();
-    this.processId = in.readInt();
+    inetAddr = DataSerializer.readInetAddress(in);
+    udpPort = in.readInt();
 
-    this.name = DataSerializer.readString(in);
-    this.groups = DataSerializer.readStringArray(in);
+    this.hostName = DataSerializer.readString(in);
+
+    int flags = in.readUnsignedByte();
+    preferredForCoordinator = (flags & PREFERRED_FOR_COORD_BIT) != 0;
+    this.networkPartitionDetectionEnabled = (flags & NPD_ENABLED_BIT) != 0;
+
+    directPort = in.readInt();
+    processId = in.readInt();
+    vmKind = (byte) in.readUnsignedByte();
+    groups = DataSerializer.readStringArray(in);
+    vmViewId = -1;
+
+    name = DataSerializer.readString(in);
+    if (vmKind == LONER_DM_TYPE) {
+      DataSerializer.readString(in);
+    } else {
+      String str = DataSerializer.readString(in);
+      if (str != null) { // backward compatibility from earlier than 6.5
+        vmViewId = Integer.parseInt(str);
+      }
+    }
+
+    durableId = DataSerializer.readString(in);
+    durableTimeout = in.readInt();
+
+    versionOrdinal = readVersion(flags, in);
+
+    if (versionOrdinal >= Version.GFE_90.ordinal()) {
+      readAdditionalData(in);
+    }
+  }
+
+  private short readVersion(int flags, DataInput in) throws IOException {
+    if ((flags & VERSION_BIT) != 0) {
+      return Version.readOrdinal(in);
+    } else {
+      // prior to 7.1 member IDs did not serialize their version information
+      Version v = InternalDataSerializer.getVersionForDataStreamOrNull(in);
+      if (v != null) {
+        return v.ordinal();
+      }
+      return Version.CURRENT_ORDINAL;
+    }
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  public void setHostName(String hostName) {
+    this.hostName = hostName;
   }
 
   public void readEssentialData(DataInput in) throws IOException, ClassNotFoundException {
@@ -538,6 +608,11 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     this.preferredForCoordinator = (flags & PREFERRED_FOR_COORD_BIT) != 0;
 
     this.inetAddr = DataSerializer.readInetAddress(in);
+    if (this.inetAddr != null) {
+      this.hostName =
+          SocketCreator.resolve_dns ? SocketCreator.getHostName(inetAddr)
+              : inetAddr.getHostAddress();
+    }
     this.udpPort = in.readInt();
     this.vmViewId = in.readInt();
     this.uuidMSBs = in.readLong();
@@ -547,19 +622,19 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
     }
   }
 
-  @Override
+
   public boolean hasAdditionalData() {
     return uuidMSBs != 0 || uuidLSBs != 0 || memberWeight != 0;
   }
 
-  @Override
+
   public void writeAdditionalData(DataOutput out) throws IOException {
     out.writeLong(uuidMSBs);
     out.writeLong(uuidLSBs);
     out.write(memberWeight);
   }
 
-  @Override
+
   public void readAdditionalData(DataInput in) throws ClassNotFoundException, IOException {
     try {
       this.uuidMSBs = in.readLong();
@@ -571,6 +646,49 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
   }
 
   private String formatUUID() {
-    return ";uuid=" + getUUID().toStringLong();
+    UUID uuid = getUUID();
+    return ";uuid=" + (uuid == null ? "none" : getUUID().toStringLong());
+  }
+
+  public void setDurableTimeout(int newValue) {
+    durableTimeout = newValue;
+  }
+
+  public void setDurableId(String id) {
+    durableId = id;
   }
+
+
+  public static class GMSMemberWrapper {
+    GMSMember mbr;
+
+    public GMSMemberWrapper(GMSMember m) {
+      this.mbr = m;
+    }
+
+    public GMSMember getMbr() {
+      return mbr;
+    }
+
+    @Override
+    public int hashCode() {
+      return mbr.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == null || !(obj instanceof GMSMemberWrapper)) {
+        return false;
+      }
+      GMSMember other = ((GMSMemberWrapper) obj).mbr;
+      return mbr.compareTo(other) == 0;
+    }
+
+    @Override
+    public String toString() {
+      return "GMSMemberWrapper [mbr=" + mbr + "]";
+    }
+  }
+
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java
similarity index 67%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java
index 1ddb1d6..6336838 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.distributed.internal.membership;
+package org.apache.geode.distributed.internal.membership.gms;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -34,35 +34,35 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.annotations.Immutable;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
+import org.apache.geode.internal.logging.LogService;
 
 /**
- * The NetView class represents a membership view. Note that this class is not synchronized, so take
+ * The GMSMembershipView class represents a membership view. Note that this class is not
+ * synchronized, so take
  * that under advisement if you decide to modify a view with add() or remove().
  *
- * @since GemFire 5.5
  */
-public class NetView implements DataSerializableFixedID {
+public class GMSMembershipView implements DataSerializableFixedID {
+  private static final Logger logger = LogService.getLogger();
 
   private int viewId;
-  private List<InternalDistributedMember> members;
+  private List<GMSMember> members;
   // TODO this should be a List
-  private final Map<InternalDistributedMember, Object> publicKeys = new ConcurrentHashMap<>();
+  private final Map<GMSMember, Object> publicKeys = new ConcurrentHashMap<>();
   private int[] failureDetectionPorts = new int[10];
-  private Set<InternalDistributedMember> shutdownMembers;
-  private Set<InternalDistributedMember> crashedMembers;
-  private InternalDistributedMember creator;
-  private Set<InternalDistributedMember> hashedMembers;
+  private Set<GMSMember> shutdownMembers;
+  private Set<GMSMember> crashedMembers;
+  private GMSMember creator;
+  private Set<GMSMember> hashedMembers;
   private final Object membersLock = new Object();
   @Immutable
   public static final Random RANDOM = new Random();
 
 
-  public NetView() {
+  public GMSMembershipView() {
     viewId = 0;
     members = new ArrayList<>(4);
     this.hashedMembers = new HashSet<>(members);
@@ -72,7 +72,7 @@ public class NetView implements DataSerializableFixedID {
     Arrays.fill(failureDetectionPorts, -1);
   }
 
-  public NetView(InternalDistributedMember creator) {
+  public GMSMembershipView(GMSMember creator) {
     viewId = 0;
     members = new ArrayList<>(4);
     members.add(creator);
@@ -83,8 +83,8 @@ public class NetView implements DataSerializableFixedID {
     Arrays.fill(failureDetectionPorts, -1);
   }
 
-  public NetView(InternalDistributedMember creator, int viewId,
-      List<InternalDistributedMember> members) {
+  public GMSMembershipView(GMSMember creator, int viewId,
+      List<GMSMember> members) {
     this.viewId = viewId;
     this.members = new ArrayList<>(members);
     hashedMembers = new HashSet<>(this.members);
@@ -101,7 +101,7 @@ public class NetView implements DataSerializableFixedID {
    * @param size size of the view, used for presizing collections
    * @param viewId the ID of the view
    */
-  public NetView(int size, long viewId) {
+  public GMSMembershipView(int size, long viewId) {
     this.viewId = (int) viewId;
     members = new ArrayList<>(size);
     this.hashedMembers = new HashSet<>();
@@ -114,7 +114,7 @@ public class NetView implements DataSerializableFixedID {
   /**
    * Create a new view with the contents of the given view and the specified view ID
    */
-  public NetView(NetView other, int viewId) {
+  public GMSMembershipView(GMSMembershipView other, int viewId) {
     this.creator = other.creator;
     this.viewId = viewId;
     this.members = new ArrayList<>(other.members);
@@ -127,9 +127,9 @@ public class NetView implements DataSerializableFixedID {
     this.publicKeys.putAll(other.publicKeys);
   }
 
-  public NetView(InternalDistributedMember creator, int viewId,
-      List<InternalDistributedMember> mbrs, Set<InternalDistributedMember> shutdowns,
-      Set<InternalDistributedMember> crashes) {
+  public GMSMembershipView(GMSMember creator, int viewId,
+      List<GMSMember> mbrs, Set<GMSMember> shutdowns,
+      Set<GMSMember> crashes) {
     this.creator = creator;
     this.viewId = viewId;
     this.members = mbrs;
@@ -140,29 +140,31 @@ public class NetView implements DataSerializableFixedID {
     Arrays.fill(this.failureDetectionPorts, -1);
   }
 
+
   public int getViewId() {
     return this.viewId;
   }
 
-  public InternalDistributedMember getCreator() {
+
+  public GMSMember getCreator() {
     return this.creator;
   }
 
-  public void setCreator(InternalDistributedMember creator) {
+  public void setCreator(GMSMember creator) {
     this.creator = creator;
   }
 
-  public Object getPublicKey(InternalDistributedMember mbr) {
+  public Object getPublicKey(GMSMember mbr) {
     return publicKeys.get(mbr);
   }
 
-  public void setPublicKey(InternalDistributedMember mbr, Object key) {
+  public void setPublicKey(GMSMember mbr, Object key) {
     if (mbr != null && key != null) {
       publicKeys.put(mbr, key);
     }
   }
 
-  public void setPublicKeys(NetView otherView) {
+  public void setPublicKeys(GMSMembershipView otherView) {
     if (otherView.publicKeys != null) {
       this.publicKeys.putAll(otherView.publicKeys);
     }
@@ -178,7 +180,7 @@ public class NetView implements DataSerializableFixedID {
     return this.failureDetectionPorts;
   }
 
-  public int getFailureDetectionPort(InternalDistributedMember mbr) {
+  public int getFailureDetectionPort(GMSMember mbr) {
     int idx = members.indexOf(mbr);
     if (idx < 0 || idx >= failureDetectionPorts.length) {
       return -1;
@@ -187,7 +189,7 @@ public class NetView implements DataSerializableFixedID {
   }
 
 
-  public void setFailureDetectionPort(InternalDistributedMember mbr, int port) {
+  public void setFailureDetectionPort(GMSMember mbr, int port) {
     int idx = members.indexOf(mbr);
     if (idx < 0) {
       throw new IllegalArgumentException("element not found in members list:" + mbr);
@@ -199,12 +201,12 @@ public class NetView implements DataSerializableFixedID {
   /**
    * Transfer the failure-detection ports from another view to this one
    */
-  public void setFailureDetectionPorts(NetView otherView) {
+  public void setFailureDetectionPorts(GMSMembershipView otherView) {
     int[] ports = otherView.getFailureDetectionPorts();
     if (ports != null) {
       int idx = 0;
       int portsSize = ports.length;
-      for (InternalDistributedMember mbr : otherView.getMembers()) {
+      for (GMSMember mbr : otherView.getMembers()) {
         if (contains(mbr)) {
           // unit tests create views w/o failure detection ports, so we must check the length
           // of the array
@@ -233,15 +235,27 @@ public class NetView implements DataSerializableFixedID {
     }
   }
 
-  public List<InternalDistributedMember> getMembers() {
-    return Collections.unmodifiableList(this.members);
+  public Object get(int i) {
+    return this.members.get(i);
+  }
+
+  public void add(GMSMember mbr) {
+    this.hashedMembers.add(mbr);
+    this.members.add(mbr);
+    int idx = members.size() - 1;
+    ensureFDCapacity(idx);
+    this.failureDetectionPorts[idx] = -1;
+  }
+
+  public void addCrashedMembers(Set<GMSMember> mbr) {
+    this.crashedMembers.addAll(mbr);
   }
 
   /**
    * return members that are i this view but not the given old view
    */
-  public List<InternalDistributedMember> getNewMembers(NetView olderView) {
-    List<InternalDistributedMember> result = new ArrayList<>(members);
+  public List<GMSMember> getNewMembers(GMSMembershipView olderView) {
+    List<GMSMember> result = new ArrayList<>(members);
     result.removeAll(olderView.getMembers());
     return result;
   }
@@ -249,30 +263,14 @@ public class NetView implements DataSerializableFixedID {
   /**
    * return members added in this view
    */
-  public List<InternalDistributedMember> getNewMembers() {
-    List<InternalDistributedMember> result = new ArrayList<>(5);
+  public List<GMSMember> getNewMembers() {
+    List<GMSMember> result = new ArrayList<>(5);
     result.addAll(this.members.stream().filter(mbr -> mbr.getVmViewId() == this.viewId)
         .collect(Collectors.toList()));
     return result;
   }
 
-  public Object get(int i) {
-    return this.members.get(i);
-  }
-
-  public void add(InternalDistributedMember mbr) {
-    this.hashedMembers.add(mbr);
-    this.members.add(mbr);
-    int idx = members.size() - 1;
-    ensureFDCapacity(idx);
-    this.failureDetectionPorts[idx] = -1;
-  }
-
-  public void addCrashedMembers(Set<InternalDistributedMember> mbr) {
-    this.crashedMembers.addAll(mbr);
-  }
-
-  public boolean remove(InternalDistributedMember mbr) {
+  public boolean remove(GMSMember mbr) {
     this.hashedMembers.remove(mbr);
     int idx = this.members.indexOf(mbr);
     if (idx >= 0) {
@@ -283,13 +281,12 @@ public class NetView implements DataSerializableFixedID {
     return this.members.remove(mbr);
   }
 
-  public void removeAll(Collection<InternalDistributedMember> ids) {
+  public void removeAll(Collection<GMSMember> ids) {
     this.hashedMembers.removeAll(ids);
     ids.forEach(this::remove);
   }
 
-  public boolean contains(DistributedMember mbr) {
-    assert mbr instanceof InternalDistributedMember;
+  public boolean contains(GMSMember mbr) {
     return this.hashedMembers.contains(mbr);
   }
 
@@ -297,19 +294,19 @@ public class NetView implements DataSerializableFixedID {
     return this.members.size();
   }
 
-  public InternalDistributedMember getLeadMember() {
-    for (InternalDistributedMember mbr : this.members) {
-      if (mbr.getVmKind() == ClusterDistributionManager.NORMAL_DM_TYPE) {
+  public GMSMember getLeadMember() {
+    for (GMSMember mbr : this.members) {
+      if (mbr.getVmKind() == GMSMember.NORMAL_DM_TYPE) {
         return mbr;
       }
     }
     return null;
   }
 
-  public InternalDistributedMember getCoordinator() {
+  public GMSMember getCoordinator() {
     synchronized (membersLock) {
-      for (InternalDistributedMember addr : members) {
-        if (addr.getNetMember().preferredForCoordinator()) {
+      for (GMSMember addr : members) {
+        if (addr.preferredForCoordinator()) {
           return addr;
         }
       }
@@ -323,18 +320,18 @@ public class NetView implements DataSerializableFixedID {
   /**
    * Returns the coordinator of this view, rejecting any in the given collection of IDs
    */
-  public InternalDistributedMember getCoordinator(
-      Collection<InternalDistributedMember> rejections) {
+  public GMSMember getCoordinator(
+      Collection<GMSMember> rejections) {
     if (rejections == null) {
       return getCoordinator();
     }
     synchronized (membersLock) {
-      for (InternalDistributedMember addr : members) {
-        if (addr.getNetMember().preferredForCoordinator() && !rejections.contains(addr)) {
+      for (GMSMember addr : members) {
+        if (addr.preferredForCoordinator() && !rejections.contains(addr)) {
           return addr;
         }
       }
-      for (InternalDistributedMember addr : members) {
+      for (GMSMember addr : members) {
         if (!rejections.contains(addr)) {
           return addr;
         }
@@ -353,18 +350,18 @@ public class NetView implements DataSerializableFixedID {
    * @param maxNumberDesired number of preferred coordinators to return
    * @return list of preferred coordinators
    */
-  public List<InternalDistributedMember> getPreferredCoordinators(
-      Set<InternalDistributedMember> filter, InternalDistributedMember localAddress,
+  public List<GMSMember> getPreferredCoordinators(
+      Set<GMSMember> filter, GMSMember localAddress,
       int maxNumberDesired) {
-    List<InternalDistributedMember> results = new ArrayList<>();
-    List<InternalDistributedMember> notPreferredCoordinatorList = new ArrayList<>();
+    List<GMSMember> results = new ArrayList<>();
+    List<GMSMember> notPreferredCoordinatorList = new ArrayList<>();
 
     synchronized (membersLock) {
-      for (InternalDistributedMember addr : members) {
+      for (GMSMember addr : members) {
         if (addr.equals(localAddress)) {
           continue;// this is must to add
         }
-        if (addr.getNetMember().preferredForCoordinator() && !filter.contains(addr)) {
+        if (addr.preferredForCoordinator() && !filter.contains(addr)) {
           results.add(addr);
           if (results.size() >= maxNumberDesired) {
             break;
@@ -377,7 +374,7 @@ public class NetView implements DataSerializableFixedID {
       results.add(localAddress);// to add local address
 
       if (results.size() < maxNumberDesired && notPreferredCoordinatorList.size() > 0) {
-        Iterator<InternalDistributedMember> it = notPreferredCoordinatorList.iterator();
+        Iterator<GMSMember> it = notPreferredCoordinatorList.iterator();
         while (it.hasNext() && results.size() < maxNumberDesired) {
           results.add(it.next());
         }
@@ -387,21 +384,32 @@ public class NetView implements DataSerializableFixedID {
     return results;
   }
 
-  public Set<InternalDistributedMember> getShutdownMembers() {
+  /* NetView implementation method */
+
+  public List<GMSMember> getGMSMembers() {
+    return (List<GMSMember>) (List<?>) Collections.unmodifiableList(this.members);
+  }
+
+
+  public List<GMSMember> getMembers() {
+    return Collections.unmodifiableList(this.members);
+  }
+
+  public Set<GMSMember> getShutdownMembers() {
     return this.shutdownMembers;
   }
 
-  public Set<InternalDistributedMember> getCrashedMembers() {
+  public Set<GMSMember> getCrashedMembers() {
     return this.crashedMembers;
   }
 
   /** check to see if the given address is next in line to be coordinator */
-  public boolean shouldBeCoordinator(InternalDistributedMember who) {
-    Iterator<InternalDistributedMember> it = this.members.iterator();
-    InternalDistributedMember firstNonPreferred = null;
+  public boolean shouldBeCoordinator(GMSMember who) {
+    Iterator<GMSMember> it = this.members.iterator();
+    GMSMember firstNonPreferred = null;
     while (it.hasNext()) {
-      InternalDistributedMember mbr = it.next();
-      if (mbr.getNetMember().preferredForCoordinator()) {
+      GMSMember mbr = it.next();
+      if (mbr.preferredForCoordinator()) {
         return mbr.equals(who);
       } else if (firstNonPreferred == null) {
         firstNonPreferred = mbr;
@@ -415,20 +423,20 @@ public class NetView implements DataSerializableFixedID {
    */
   public int memberWeight() {
     int result = 0;
-    InternalDistributedMember lead = getLeadMember();
-    for (InternalDistributedMember mbr : this.members) {
-      result += mbr.getNetMember().getMemberWeight();
+    GMSMember lead = getLeadMember();
+    for (GMSMember mbr : this.members) {
+      result += mbr.getMemberWeight();
       switch (mbr.getVmKind()) {
-        case ClusterDistributionManager.NORMAL_DM_TYPE:
+        case GMSMember.NORMAL_DM_TYPE:
           result += 10;
           if (lead != null && mbr.equals(lead)) {
             result += 5;
           }
           break;
-        case ClusterDistributionManager.LOCATOR_DM_TYPE:
+        case GMSMember.LOCATOR_DM_TYPE:
           result += 3;
           break;
-        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
+        case GMSMember.ADMIN_ONLY_DM_TYPE:
           break;
         default:
           throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -441,25 +449,25 @@ public class NetView implements DataSerializableFixedID {
    * returns the weight of crashed members in this membership view with respect to the given
    * previous view
    */
-  public int getCrashedMemberWeight(NetView oldView) {
+  public int getCrashedMemberWeight(GMSMembershipView oldView) {
     int result = 0;
-    InternalDistributedMember lead = oldView.getLeadMember();
-    for (InternalDistributedMember mbr : this.crashedMembers) {
+    GMSMember lead = oldView.getLeadMember();
+    for (GMSMember mbr : this.crashedMembers) {
       if (!oldView.contains(mbr)) {
         continue;
       }
-      result += mbr.getNetMember().getMemberWeight();
+      result += mbr.getMemberWeight();
       switch (mbr.getVmKind()) {
-        case ClusterDistributionManager.NORMAL_DM_TYPE:
+        case GMSMember.NORMAL_DM_TYPE:
           result += 10;
           if (lead != null && mbr.equals(lead)) {
             result += 5;
           }
           break;
-        case ClusterDistributionManager.LOCATOR_DM_TYPE:
+        case GMSMember.LOCATOR_DM_TYPE:
           result += 3;
           break;
-        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
+        case GMSMember.ADMIN_ONLY_DM_TYPE:
           break;
         default:
           throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -472,10 +480,10 @@ public class NetView implements DataSerializableFixedID {
    * returns the members of this views crashedMembers collection that were members of the given
    * view. Admin-only members are not counted
    */
-  public Set<InternalDistributedMember> getActualCrashedMembers(NetView oldView) {
-    Set<InternalDistributedMember> result = new HashSet<>(this.crashedMembers.size());
+  public Set<GMSMember> getActualCrashedMembers(GMSMembershipView oldView) {
+    Set<GMSMember> result = new HashSet<>(this.crashedMembers.size());
     result.addAll(this.crashedMembers.stream()
-        .filter(mbr -> (mbr.getVmKind() != ClusterDistributionManager.ADMIN_ONLY_DM_TYPE))
+        .filter(mbr -> (mbr.getVmKind() != GMSMember.ADMIN_ONLY_DM_TYPE))
         .filter(mbr -> oldView == null || oldView.contains(mbr)).collect(Collectors.toList()));
     return result;
   }
@@ -483,25 +491,25 @@ public class NetView implements DataSerializableFixedID {
   /**
    * logs the weight of failed members wrt the given previous view
    */
-  public void logCrashedMemberWeights(NetView oldView, Logger log) {
-    InternalDistributedMember lead = oldView.getLeadMember();
-    for (InternalDistributedMember mbr : this.crashedMembers) {
+  public void logCrashedMemberWeights(GMSMembershipView oldView, Logger log) {
+    GMSMember lead = oldView.getLeadMember();
+    for (GMSMember mbr : this.crashedMembers) {
       if (!oldView.contains(mbr)) {
         continue;
       }
-      int mbrWeight = mbr.getNetMember().getMemberWeight();
+      int mbrWeight = mbr.getMemberWeight();
       switch (mbr.getVmKind()) {
-        case ClusterDistributionManager.NORMAL_DM_TYPE:
+        case GMSMember.NORMAL_DM_TYPE:
           if (lead != null && mbr.equals(lead)) {
             mbrWeight += 15;
           } else {
             mbrWeight += 10;
           }
           break;
-        case ClusterDistributionManager.LOCATOR_DM_TYPE:
+        case GMSMember.LOCATOR_DM_TYPE:
           mbrWeight += 3;
           break;
-        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
+        case GMSMember.ADMIN_ONLY_DM_TYPE:
           break;
         default:
           throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -511,12 +519,12 @@ public class NetView implements DataSerializableFixedID {
   }
 
   public String toString() {
-    InternalDistributedMember lead = getLeadMember();
+    GMSMember lead = getLeadMember();
 
     StringBuilder sb = new StringBuilder(200);
     sb.append("View[").append(creator).append('|').append(viewId).append("] members: [");
     boolean first = true;
-    for (InternalDistributedMember mbr : this.members) {
+    for (GMSMember mbr : this.members) {
       if (!first)
         sb.append(", ");
       sb.append(mbr);
@@ -528,7 +536,7 @@ public class NetView implements DataSerializableFixedID {
     if (!this.shutdownMembers.isEmpty()) {
       sb.append("]  shutdown: [");
       first = true;
-      for (InternalDistributedMember mbr : this.shutdownMembers) {
+      for (GMSMember mbr : this.shutdownMembers) {
         if (!first)
           sb.append(", ");
         sb.append(mbr);
@@ -538,7 +546,7 @@ public class NetView implements DataSerializableFixedID {
     if (!this.crashedMembers.isEmpty()) {
       sb.append("]  crashed: [");
       first = true;
-      for (InternalDistributedMember mbr : this.crashedMembers) {
+      for (GMSMember mbr : this.crashedMembers) {
         if (!first)
           sb.append(", ");
         sb.append(mbr);
@@ -562,9 +570,9 @@ public class NetView implements DataSerializableFixedID {
    * Returns the ID from this view that is equal to the argument. If no such ID exists the argument
    * is returned.
    */
-  public synchronized InternalDistributedMember getCanonicalID(InternalDistributedMember id) {
+  public synchronized GMSMember getCanonicalID(GMSMember id) {
     if (hashedMembers.contains(id)) {
-      for (InternalDistributedMember m : this.members) {
+      for (GMSMember m : this.members) {
         if (id.equals(m)) {
           return m;
         }
@@ -578,8 +586,8 @@ public class NetView implements DataSerializableFixedID {
     if (other == this) {
       return true;
     }
-    if (other instanceof NetView) {
-      return this.members.equals(((NetView) other).getMembers());
+    if (other instanceof GMSMembershipView) {
+      return this.members.equals(((GMSMembershipView) other).getGMSMembers());
     }
     return false;
   }
@@ -591,25 +599,25 @@ public class NetView implements DataSerializableFixedID {
 
   @Override
   public void toData(DataOutput out) throws IOException {
-    DataSerializer.writeObject(creator, out);
+    GMSUtil.writeMemberID(creator, out);
     out.writeInt(viewId);
     writeAsArrayList(members, out);
-    InternalDataSerializer.writeSet(shutdownMembers, out);
-    InternalDataSerializer.writeSet(crashedMembers, out);
+    GMSUtil.writeSetOfMemberIDs(shutdownMembers, out);
+    GMSUtil.writeSetOfMemberIDs(crashedMembers, out);
     DataSerializer.writeIntArray(failureDetectionPorts, out);
     // TODO expensive serialization
     DataSerializer.writeHashMap(publicKeys, out);
   }
 
-  @Override
+
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    creator = DataSerializer.readObject(in);
+    creator = GMSUtil.readMemberID(in);
     viewId = in.readInt();
-    members = DataSerializer.readArrayList(in);
+    members = GMSUtil.readArrayOfIDs(in);
     assert members != null;
     this.hashedMembers = new HashSet<>(members);
-    shutdownMembers = InternalDataSerializer.readHashSet(in);
-    crashedMembers = InternalDataSerializer.readHashSet(in);
+    shutdownMembers = GMSUtil.readHashSetOfMemberIDs(in);
+    crashedMembers = GMSUtil.readHashSetOfMemberIDs(in);
     failureDetectionPorts = DataSerializer.readIntArray(in);
     Map pubkeys = DataSerializer.readHashMap(in);
     if (pubkeys != null) {
@@ -618,7 +626,7 @@ public class NetView implements DataSerializableFixedID {
   }
 
   /** this will deserialize as an ArrayList */
-  private void writeAsArrayList(List list, DataOutput out) throws IOException {
+  private void writeAsArrayList(List<GMSMember> list, DataOutput out) throws IOException {
     int size;
     if (list == null) {
       size = -1;
@@ -628,7 +636,22 @@ public class NetView implements DataSerializableFixedID {
     InternalDataSerializer.writeArrayLength(size, out);
     if (size > 0) {
       for (int i = 0; i < size; i++) {
-        DataSerializer.writeObject(list.get(i), out);
+        GMSUtil.writeMemberID(list.get(i), out);
+      }
+    }
+  }
+
+  private void writeAsSet(Set<GMSMember> set, DataOutput out) throws IOException {
+    int size;
+    if (set == null) {
+      size = -1;
+    } else {
+      size = set.size();
+    }
+    InternalDataSerializer.writeArrayLength(size, out);
+    if (size > 0) {
+      for (GMSMember member : set) {
+        GMSUtil.writeMemberID(member, out);
       }
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java
index 0ccc8dc..d320d84 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java
@@ -14,6 +14,11 @@
  */
 package org.apache.geode.distributed.internal.membership.gms;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
@@ -23,11 +28,19 @@ import java.util.List;
 import java.util.Set;
 import java.util.StringTokenizer;
 
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.DataSerializer;
 import org.apache.geode.GemFireConfigException;
 import org.apache.geode.distributed.internal.membership.gms.membership.HostAddress;
+import org.apache.geode.internal.DSFIDFactory;
+import org.apache.geode.internal.InternalDataSerializer;
+import org.apache.geode.internal.Version;
+import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.net.SocketCreator;
 
 public class GMSUtil {
+  private static final Logger logger = LogService.getLogger();
 
   /**
    * parse locators & check that the resulting address is compatible with the given address
@@ -51,6 +64,35 @@ public class GMSUtil {
     return parseLocators(locatorsString, addr);
   }
 
+  public static GMSMember readMemberID(DataInput in) throws IOException, ClassNotFoundException {
+    Object id = DataSerializer.readObject(in);
+    if (id == null || id instanceof GMSMember) {
+      return (GMSMember) id;
+    }
+    // return ((GMSMemberAdapter)((InternalDistributedMember)id).getNetMember()).getGmsMember();
+    try {
+      Method getNetMember = id.getClass().getMethod("getNetMember");
+      Object netMember = getNetMember.invoke(id);
+      Method getGmsMember = netMember.getClass().getMethod("getGmsMember");
+      return (GMSMember) getGmsMember.invoke(netMember);
+    } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+      throw new IllegalStateException("Unable to deserialize a member ID", e);
+    }
+  }
+
+  public static Set<GMSMember> readHashSetOfMemberIDs(DataInput in)
+      throws IOException, ClassNotFoundException {
+    int size = InternalDataSerializer.readArrayLength(in);
+    if (size == -1) {
+      return null;
+    }
+    Set<GMSMember> result = new HashSet<>();
+    for (int i = 0; i < size; i++) {
+      result.add(readMemberID(in));
+    }
+    return result;
+  }
+
   /**
    * parse locators & check that the resulting address is compatible with the given address
    *
@@ -113,6 +155,28 @@ public class GMSUtil {
     return result;
   }
 
+  /** Parses comma-separated-roles/groups into array of groups (strings). */
+  public static String[] parseGroups(String csvRoles, String csvGroups) {
+    List<String> groups = new ArrayList<String>();
+    parseCsv(groups, csvRoles);
+    parseCsv(groups, csvGroups);
+    return groups.toArray(new String[groups.size()]);
+  }
+
+
+  private static void parseCsv(List<String> groups, String csv) {
+    if (csv == null || csv.length() == 0) {
+      return;
+    }
+    StringTokenizer st = new StringTokenizer(csv, ",");
+    while (st.hasMoreTokens()) {
+      String groupName = st.nextToken().trim();
+      if (!groups.contains(groupName)) { // only add each group once
+        groups.add(groupName);
+      }
+    }
+  }
+
   /**
    * replaces all occurrences of a given string in the properties argument with the given value
    */
@@ -131,4 +195,66 @@ public class GMSUtil {
     return sb.toString();
   }
 
+  public static List<GMSMember> readArrayOfIDs(DataInput in)
+      throws IOException, ClassNotFoundException {
+    int size = InternalDataSerializer.readArrayLength(in);
+    if (size == -1) {
+      return null;
+    }
+    List<GMSMember> result = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      result.add(readMemberID(in));
+    }
+    return result;
+  }
+
+  private static void writeAsInternalDistributedMember(GMSMember suspect, DataOutput out)
+      throws IOException {
+    InternalDataSerializer.writeDSFID(suspect, DSFIDFactory.DISTRIBUTED_MEMBER, out);
+    // DataSerializer.writeObject(new InternalDistributedMember(new GMSMemberAdapter(suspect)),
+    // out);
+  }
+
+  public static void writeMemberID(GMSMember id, DataOutput out) throws IOException {
+    if (id == null) {
+      DataSerializer.writeObject(id, out);
+      return;
+    }
+    if (InternalDataSerializer.getVersionForDataStream(out).ordinal() < Version.GEODE_1_10_0
+        .ordinal()) {
+      writeAsInternalDistributedMember(id, out);
+    } else {
+      DataSerializer.writeObject(id, out);
+    }
+  }
+
+  public static Set<GMSMember> readSetOfMemberIDs(DataInput in)
+      throws IOException, ClassNotFoundException {
+    int size = InternalDataSerializer.readArrayLength(in);
+    if (size == -1) {
+      return null;
+    }
+    Set<GMSMember> result = new HashSet<>(size);
+    for (int i = 0; i < size; i++) {
+      result.add(readMemberID(in));
+    }
+    return result;
+
+
+  }
+
+  public static void writeSetOfMemberIDs(Set<GMSMember> set, DataOutput out) throws IOException {
+    int size;
+    if (set == null) {
+      size = -1;
+    } else {
+      size = set.size();
+    }
+    InternalDataSerializer.writeArrayLength(size, out);
+    if (size > 0) {
+      for (GMSMember member : set) {
+        GMSUtil.writeMemberID(member, out);
+      }
+    }
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/ServiceConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/ServiceConfig.java
index 1e3e3da..653fd90 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/ServiceConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/ServiceConfig.java
@@ -141,6 +141,7 @@ public class ServiceConfig {
 
     // if network partition detection is enabled, we must connect to the locators
     // more frequently in order to make sure we're not isolated from them
+    SocketCreator.resolve_dns = true;
     if (theConfig.getEnableNetworkPartitionDetection()) {
       if (!SocketCreator.FORCE_DNS_USE) {
         SocketCreator.resolve_dns = false;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java
index d90ba5b..447057f 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java
@@ -23,11 +23,6 @@ import org.apache.geode.ForcedDisconnectException;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.distributed.internal.membership.DistributedMembershipListener;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
 import org.apache.geode.distributed.internal.membership.gms.fd.GMSHealthMonitor;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Authenticator;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
@@ -37,7 +32,6 @@ import org.apache.geode.distributed.internal.membership.gms.interfaces.Manager;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Messenger;
 import org.apache.geode.distributed.internal.membership.gms.membership.GMSJoinLeave;
 import org.apache.geode.distributed.internal.membership.gms.messenger.JGroupsMessenger;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.logging.LogService;
 
@@ -95,27 +89,34 @@ public class Services {
     this.auth = null;
   }
 
-  public Services(DistributedMembershipListener listener,
+  public Services(Manager membershipManager,
       RemoteTransportConfig transport, DMStats stats,
       final Authenticator authenticator, DistributionConfig config) {
     this.cancelCriterion = new Stopper();
     this.stats = stats;
     this.config = new ServiceConfig(transport, config);
-    this.manager = new GMSMembershipManager(listener);
+    this.manager = membershipManager;
     this.joinLeave = new GMSJoinLeave();
     this.healthMon = new GMSHealthMonitor();
     this.messenger = new JGroupsMessenger();
     this.auth = authenticator;
   }
 
-  protected void init() {
+  /**
+   * Initialize services - do this before invoking start()
+   */
+  public void init() {
     this.messenger.init(this);
     this.manager.init(this);
     this.joinLeave.init(this);
     this.healthMon.init(this);
   }
 
-  protected void start() {
+  /**
+   * Start services - this will start everything up and join the cluster.
+   * Invoke init() before this method.
+   */
+  public void start() {
     boolean started = false;
     try {
       logger.info("Starting membership services");
@@ -144,12 +145,6 @@ public class Services {
     this.joinLeave.started();
     this.healthMon.started();
     this.manager.started();
-    InternalLocator l = (InternalLocator) org.apache.geode.distributed.Locator.getLocator();
-    if (l != null && l.getLocatorHandler() != null) {
-      if (l.getLocatorHandler().setMembershipManager((MembershipManager) this.manager)) {
-        this.locator = (Locator) l.getLocatorHandler();
-      }
-    }
     logger.debug("All membership services have been started");
     try {
       this.manager.joinDistributedSystem();
@@ -159,7 +154,7 @@ public class Services {
     }
   }
 
-  public void setLocalAddress(InternalDistributedMember address) {
+  public void setLocalAddress(GMSMember address) {
     this.messenger.setLocalAddress(address);
     this.joinLeave.setLocalAddress(address);
     this.healthMon.setLocalAddress(address);
@@ -228,7 +223,7 @@ public class Services {
     return this.auth;
   }
 
-  public void installView(NetView v) {
+  public void installView(GMSMembershipView v) {
     if (this.locator != null) {
       this.locator.installView(v);
     }
@@ -237,8 +232,8 @@ public class Services {
     this.manager.installView(v);
   }
 
-  public void memberSuspected(InternalDistributedMember initiator,
-      InternalDistributedMember suspect, String reason) {
+  public void memberSuspected(GMSMember initiator,
+      GMSMember suspect, String reason) {
     try {
       this.joinLeave.memberSuspected(initiator, suspect, reason);
     } finally {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
index f48097f..304bcaa 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
@@ -14,22 +14,20 @@
  */
 package org.apache.geode.distributed.internal.membership.gms;
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-
 /** represents a suspicion raised about a member */
 public class SuspectMember {
   /** the source of suspicion */
-  public InternalDistributedMember whoSuspected;
+  public GMSMember whoSuspected;
 
   /** suspected member */
-  public InternalDistributedMember suspectedMember;
+  public GMSMember suspectedMember;
 
   /** the reason */
   public String reason;
 
   /** create a new SuspectMember */
-  public SuspectMember(InternalDistributedMember whoSuspected,
-      InternalDistributedMember suspectedMember, String reason) {
+  public SuspectMember(GMSMember whoSuspected,
+      GMSMember suspectedMember, String reason) {
     this.whoSuspected = whoSuspected;
     this.suspectedMember = suspectedMember;
     this.reason = reason;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 81cbddc..d085cd2 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -54,16 +54,14 @@ import org.jgroups.util.UUID;
 import org.apache.geode.CancelException;
 import org.apache.geode.GemFireConfigException;
 import org.apache.geode.SystemConnectException;
-import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
+import org.apache.geode.distributed.internal.membership.gms.messages.AbstractGMSMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.FinalCheckPassedMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.HeartbeatMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.HeartbeatRequestMessage;
@@ -85,11 +83,11 @@ import org.apache.geode.internal.security.SecurableCommunicationChannel;
  * we check whether this member is still alive or not. Based on that we informed probable
  * coordinators to remove that member from view.
  * <p>
- * It has {@link #suspect(InternalDistributedMember, String)} api, which can be used to initiate
+ * It has {@link #suspect(GMSMember, String)} api, which can be used to initiate
  * suspect processing for any member. First is checks whether the member is responding or not. Then
  * it informs probable coordinators to remove that member from view.
  * <p>
- * It has {@link HealthMonitor#checkIfAvailable(InternalDistributedMember, String, boolean)} api to
+ * It has {@link HealthMonitor#checkIfAvailable(GMSMember, String, boolean)} api to
  * see if that member is
  * alive. Then based on removal flag it initiates the suspect processing for that member.
  */
@@ -97,8 +95,8 @@ import org.apache.geode.internal.security.SecurableCommunicationChannel;
 public class GMSHealthMonitor implements HealthMonitor {
 
   private Services services;
-  private volatile NetView currentView;
-  private volatile InternalDistributedMember nextNeighbor;
+  private volatile GMSMembershipView currentView;
+  private volatile GMSMember nextNeighbor;
 
   long memberTimeout;
   private volatile boolean isStopping = false;
@@ -134,24 +132,24 @@ public class GMSHealthMonitor implements HealthMonitor {
   /**
    * this member's ID
    */
-  private InternalDistributedMember localAddress;
+  private GMSMember localAddress;
 
   /**
    * Timestamp at which we last had contact from a member
    */
-  final ConcurrentMap<InternalDistributedMember, TimeStamp> memberTimeStamps =
+  final ConcurrentMap<GMSMember, TimeStamp> memberTimeStamps =
       new ConcurrentHashMap<>();
 
   /**
    * Members currently being suspected and the view they were suspected in
    */
-  private final ConcurrentHashMap<InternalDistributedMember, NetView> suspectedMemberIds =
+  private final ConcurrentHashMap<GMSMember, GMSMembershipView> suspectedMemberIds =
       new ConcurrentHashMap<>();
 
   /**
    * Members undergoing final checks
    */
-  private final List<InternalDistributedMember> membersInFinalCheck =
+  private final List<GMSMember> membersInFinalCheck =
       Collections.synchronizedList(new ArrayList<>(30));
 
   /**
@@ -162,7 +160,7 @@ public class GMSHealthMonitor implements HealthMonitor {
   /**
    * Members suspected in a particular view
    */
-  private final Map<NetView, Set<SuspectRequest>> suspectRequestsInView = new HashMap<>();
+  private final Map<GMSMembershipView, Set<SuspectRequest>> suspectRequestsInView = new HashMap<>();
 
   private ScheduledExecutorService scheduler;
 
@@ -223,7 +221,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
   /***
    * This class sets start interval timestamp to record the activity of all members. That is used by
-   * {@link GMSHealthMonitor#contactedBy(InternalDistributedMember)} to record the activity of
+   * {@link GMSHealthMonitor#contactedBy(GMSMember)} to record the activity of
    * member.
    *
    * It initiates the suspect processing for next neighbour if it doesn't see any activity from that
@@ -250,7 +248,7 @@ public class GMSHealthMonitor implements HealthMonitor {
         return;
       }
 
-      InternalDistributedMember neighbour = nextNeighbor;
+      GMSMember neighbour = nextNeighbor;
 
       long currentTime = System.currentTimeMillis();
       // this is the start of interval to record member activity
@@ -260,7 +258,7 @@ public class GMSHealthMonitor implements HealthMonitor {
       long oldTimeStamp = currentTimeStamp;
       currentTimeStamp = System.currentTimeMillis();
 
-      NetView myView = GMSHealthMonitor.this.currentView;
+      GMSMembershipView myView = GMSHealthMonitor.this.currentView;
       if (myView == null) {
         return;
       }
@@ -270,7 +268,7 @@ public class GMSHealthMonitor implements HealthMonitor {
         logger.info(
             "Failure detector has noticed a JVM pause and is giving all members a heartbeat in view {}",
             currentView);
-        for (InternalDistributedMember member : myView.getMembers()) {
+        for (GMSMember member : myView.getMembers()) {
           contactedBy(member);
         }
         return;
@@ -308,13 +306,13 @@ public class GMSHealthMonitor implements HealthMonitor {
    */
   private class Response {
 
-    private DistributionMessage responseMsg;
+    private AbstractGMSMessage responseMsg;
 
-    public DistributionMessage getResponseMsg() {
+    public AbstractGMSMessage getResponseMsg() {
       return responseMsg;
     }
 
-    public void setResponseMsg(DistributionMessage responseMsg) {
+    public void setResponseMsg(AbstractGMSMessage responseMsg) {
       this.responseMsg = responseMsg;
     }
 
@@ -341,7 +339,7 @@ public class GMSHealthMonitor implements HealthMonitor {
         long uuidMSBs = in.readLong();
         GMSHealthMonitor.this.stats.incFinalCheckRequestsReceived();
         GMSHealthMonitor.this.stats.incTcpFinalCheckRequestsReceived();
-        GMSMember gmbr = (GMSMember) GMSHealthMonitor.this.localAddress.getNetMember();
+        GMSMember gmbr = localAddress;
         UUID myUUID = gmbr.getUUID();
         // during reconnect or rapid restart we will have a zero viewId but there may still
         // be an old ID in the membership view that we do not want to respond to
@@ -405,7 +403,7 @@ public class GMSHealthMonitor implements HealthMonitor {
    * Record the member activity for current time interval.
    */
   @Override
-  public void contactedBy(InternalDistributedMember sender) {
+  public void contactedBy(GMSMember sender) {
     contactedBy(sender, currentTimeStamp);
   }
 
@@ -413,7 +411,7 @@ public class GMSHealthMonitor implements HealthMonitor {
   /**
    * Record member activity at a specified time
    */
-  private void contactedBy(InternalDistributedMember sender, long timeStamp) {
+  private void contactedBy(GMSMember sender, long timeStamp) {
     TimeStamp cTS = new TimeStamp(timeStamp);
     cTS = memberTimeStamps.putIfAbsent(sender, cTS);
     if (cTS != null && cTS.getTime() < timeStamp) {
@@ -427,7 +425,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
 
   private HeartbeatRequestMessage constructHeartbeatRequestMessage(
-      final InternalDistributedMember mbr) {
+      final GMSMember mbr) {
     final int reqId = requestId.getAndIncrement();
     final HeartbeatRequestMessage hrm = new HeartbeatRequestMessage(mbr, reqId);
     hrm.setRecipient(mbr);
@@ -435,8 +433,8 @@ public class GMSHealthMonitor implements HealthMonitor {
     return hrm;
   }
 
-  private void checkMember(final InternalDistributedMember mbr) {
-    final NetView cv = GMSHealthMonitor.this.currentView;
+  private void checkMember(final GMSMember mbr) {
+    final GMSMembershipView cv = GMSHealthMonitor.this.currentView;
 
     // as check may take time
     setNextNeighbor(cv, mbr);
@@ -465,7 +463,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
   }
 
-  private void initiateSuspicion(InternalDistributedMember mbr, String reason) {
+  private void initiateSuspicion(GMSMember mbr, String reason) {
     if (services.getJoinLeave().isMemberLeaving(mbr)) {
       return;
     }
@@ -476,7 +474,7 @@ public class GMSHealthMonitor implements HealthMonitor {
    * This method sends heartbeat request to other member and waits for member-timeout time for
    * response. If it doesn't see response then it returns false.
    */
-  private boolean doCheckMember(InternalDistributedMember member, boolean waitForResponse) {
+  private boolean doCheckMember(GMSMember member, boolean waitForResponse) {
     if (playingDead || beingSick) {
       // a member playingDead should not be sending messages to other
       // members, so we avoid sending heartbeat requests or suspect
@@ -494,7 +492,7 @@ public class GMSHealthMonitor implements HealthMonitor {
       hrm.clearRequestId();
     }
     try {
-      Set<InternalDistributedMember> membersNotReceivedMsg = this.services.getMessenger().send(hrm);
+      Set<GMSMember> membersNotReceivedMsg = this.services.getMessenger().send(hrm);
       this.stats.incHeartbeatRequestsSent();
       if (membersNotReceivedMsg != null && membersNotReceivedMsg.contains(member)) {
         // member is not part of current view.
@@ -543,7 +541,7 @@ public class GMSHealthMonitor implements HealthMonitor {
    * @param suspectMember member that does not respond to HeartbeatRequestMessage
    * @return true if successfully exchanged PING/PONG with TCP connection, otherwise false.
    */
-  boolean doTCPCheckMember(InternalDistributedMember suspectMember, int port,
+  boolean doTCPCheckMember(GMSMember suspectMember, int port,
       boolean retryIfConnectFails) {
     Socket clientSocket = null;
     // make sure we try to check on the member for the contracted memberTimeout period
@@ -594,13 +592,13 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   // Package protected for testing purposes
-  boolean doTCPCheckMember(InternalDistributedMember suspectMember, Socket clientSocket) {
+  boolean doTCPCheckMember(GMSMember suspectMember, Socket clientSocket) {
     try {
       if (clientSocket.isConnected()) {
         clientSocket.setSoTimeout((int) services.getConfig().getMemberTimeout());
         InputStream in = clientSocket.getInputStream();
         DataOutputStream out = new DataOutputStream(clientSocket.getOutputStream());
-        GMSMember gmbr = (GMSMember) suspectMember.getNetMember();
+        GMSMember gmbr = suspectMember;
         writeMemberToStream(gmbr, out);
         this.stats.incFinalCheckRequestsSent();
         this.stats.incTcpFinalCheckRequestsSent();
@@ -646,12 +644,12 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   @Override
-  public void suspect(InternalDistributedMember mbr, String reason) {
+  public void suspect(GMSMember mbr, String reason) {
     initiateSuspicion(mbr, reason);
   }
 
   @Override
-  public boolean checkIfAvailable(InternalDistributedMember mbr, String reason,
+  public boolean checkIfAvailable(GMSMember mbr, String reason,
       boolean initiateRemoval) {
     return inlineCheckIfAvailable(localAddress, currentView, initiateRemoval,
         mbr, reason);
@@ -753,9 +751,9 @@ public class GMSHealthMonitor implements HealthMonitor {
           } catch (InterruptedException e) {
             return;
           }
-          NetView v = currentView;
+          GMSMembershipView v = currentView;
           if (v != null) {
-            List<InternalDistributedMember> mbrs = v.getMembers();
+            List<GMSMember> mbrs = v.getMembers();
             int index = mbrs.indexOf(localAddress);
             if (index < 0 || mbrs.size() < 2) {
               continue;
@@ -767,8 +765,8 @@ public class GMSHealthMonitor implements HealthMonitor {
         }
       }
 
-      private void sendHeartbeats(List<InternalDistributedMember> mbrs, int startIndex) {
-        InternalDistributedMember coordinator = currentView.getCoordinator();
+      private void sendHeartbeats(List<GMSMember> mbrs, int startIndex) {
+        GMSMember coordinator = currentView.getCoordinator();
         if (coordinator != null && !coordinator.equals(localAddress)) {
           HeartbeatMessage message = new HeartbeatMessage(-1);
           message.setRecipient(coordinator);
@@ -790,7 +788,7 @@ public class GMSHealthMonitor implements HealthMonitor {
           if (index < 0) {
             index = mbrs.size() - 1;
           }
-          InternalDistributedMember mbr = mbrs.get(index);
+          GMSMember mbr = mbrs.get(index);
           if (mbr.equals(localAddress)) {
             break;
           }
@@ -818,17 +816,17 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   @Override
-  public synchronized void installView(NetView newView) {
+  public synchronized void installView(GMSMembershipView newView) {
     synchronized (suspectRequestsInView) {
       suspectRequestsInView.clear();
     }
-    for (Iterator<InternalDistributedMember> it = memberTimeStamps.keySet().iterator(); it
+    for (Iterator<GMSMember> it = memberTimeStamps.keySet().iterator(); it
         .hasNext();) {
       if (!newView.contains(it.next())) {
         it.remove();
       }
     }
-    for (Iterator<InternalDistributedMember> it = suspectedMemberIds.keySet().iterator(); it
+    for (Iterator<GMSMember> it = suspectedMemberIds.keySet().iterator(); it
         .hasNext();) {
       if (!newView.contains(it.next())) {
         it.remove();
@@ -842,7 +840,7 @@ public class GMSHealthMonitor implements HealthMonitor {
    * this method is primarily for tests. The current view should be pulled from JoinLeave or the
    * MembershipManager (which includes surprise members)
    */
-  public synchronized NetView getView() {
+  public synchronized GMSMembershipView getView() {
     return currentView;
   }
 
@@ -854,7 +852,7 @@ public class GMSHealthMonitor implements HealthMonitor {
    * It becomes null when we suspect current neighbour, during that time it watches member next to
    * suspect member.
    */
-  protected synchronized void setNextNeighbor(NetView newView, InternalDistributedMember nextTo) {
+  protected synchronized void setNextNeighbor(GMSMembershipView newView, GMSMember nextTo) {
     if (newView == null) {
       return;
     }
@@ -862,11 +860,11 @@ public class GMSHealthMonitor implements HealthMonitor {
       nextTo = localAddress;
     }
 
-    List<InternalDistributedMember> allMembers = newView.getMembers();
+    List<GMSMember> allMembers = newView.getMembers();
 
     if (allMembers.size() > 1 && suspectedMemberIds.size() >= allMembers.size() - 1) {
       boolean nonSuspectFound = false;
-      for (InternalDistributedMember member : allMembers) {
+      for (GMSMember member : allMembers) {
         if (member.equals(localAddress)) {
           continue;
         }
@@ -885,12 +883,12 @@ public class GMSHealthMonitor implements HealthMonitor {
     int index = allMembers.indexOf(nextTo);
     if (index != -1) {
       int nextNeighborIndex = (index + 1) % allMembers.size();
-      InternalDistributedMember newNeighbor = allMembers.get(nextNeighborIndex);
+      GMSMember newNeighbor = allMembers.get(nextNeighborIndex);
       if (suspectedMemberIds.containsKey(newNeighbor)) {
         setNextNeighbor(newView, newNeighbor);
         return;
       }
-      InternalDistributedMember oldNeighbor = nextNeighbor;
+      GMSMember oldNeighbor = nextNeighbor;
       if (oldNeighbor != newNeighbor) {
         logger.debug("Failure detection is now watching " + newNeighbor);
         nextNeighbor = newNeighbor;
@@ -908,7 +906,7 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   /** test method */
-  public InternalDistributedMember getNextNeighbor() {
+  public GMSMember getNextNeighbor() {
     return nextNeighbor;
   }
 
@@ -996,7 +994,7 @@ public class GMSHealthMonitor implements HealthMonitor {
   /**
    * Test method - check to see if a member is under suspicion
    */
-  public boolean isSuspectMember(InternalDistributedMember m) {
+  public boolean isSuspectMember(GMSMember m) {
     return this.suspectedMemberIds.containsKey(m);
   }
 
@@ -1006,8 +1004,8 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   @Override
-  public void memberSuspected(InternalDistributedMember initiator,
-      InternalDistributedMember suspect, String reason) {
+  public void memberSuspected(GMSMember initiator,
+      GMSMember suspect, String reason) {
     synchronized (suspectRequestsInView) {
       suspectedMemberIds.put(suspect, currentView);
       Collection<SuspectRequest> requests = suspectRequestsInView.get(currentView);
@@ -1028,7 +1026,7 @@ public class GMSHealthMonitor implements HealthMonitor {
     }
   }
 
-  private void memberUnsuspected(InternalDistributedMember mbr) {
+  private void memberUnsuspected(GMSMember mbr) {
     synchronized (suspectRequestsInView) {
       if (suspectedMemberIds.remove(mbr) != null) {
         logger.info("No longer suspecting {}", mbr);
@@ -1068,7 +1066,7 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   @Override
-  public void setLocalAddress(InternalDistributedMember idm) {
+  public void setLocalAddress(GMSMember idm) {
     this.localAddress = idm;
   }
 
@@ -1088,12 +1086,12 @@ public class GMSHealthMonitor implements HealthMonitor {
     }
 
     // only respond if the intended recipient is this member
-    InternalDistributedMember me = localAddress;
+    GMSMember me = localAddress;
 
     if (me == null || me.getVmViewId() >= 0 && m.getTarget().equals(me)) {
       HeartbeatMessage hm = new HeartbeatMessage(m.getRequestId());
       hm.setRecipient(m.getSender());
-      Set<InternalDistributedMember> membersNotReceivedMsg = services.getMessenger().send(hm);
+      Set<GMSMember> membersNotReceivedMsg = services.getMessenger().send(hm);
       this.stats.incHeartbeatsSent();
       if (membersNotReceivedMsg != null && membersNotReceivedMsg.contains(m.getSender())) {
         logger.debug("Unable to send heartbeat to member: {}", m.getSender());
@@ -1147,7 +1145,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
     this.stats.incSuspectsReceived();
 
-    NetView cv = currentView;
+    GMSMembershipView cv = currentView;
 
     if (cv == null) {
       return;
@@ -1155,7 +1153,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
     List<SuspectRequest> suspectRequests = incomingRequest.getMembers();
 
-    InternalDistributedMember sender = incomingRequest.getSender();
+    GMSMember sender = incomingRequest.getSender();
     int viewId = sender.getVmViewId();
     if (cv.getViewId() >= viewId && !cv.contains(incomingRequest.getSender())) {
       logger.info("Membership ignoring suspect request for " + incomingRequest + " from non-member "
@@ -1193,7 +1191,7 @@ public class GMSHealthMonitor implements HealthMonitor {
       // Another process has raised suspicion - check to see if
       // this process should become the membership coordinator if
       // all current suspects are gone
-      NetView check = new NetView(cv, cv.getViewId() + 1);
+      GMSMembershipView check = new GMSMembershipView(cv, cv.getViewId() + 1);
       ArrayList<SuspectRequest> membersToCheck = new ArrayList<>();
       synchronized (suspectRequestsInView) {
         recordSuspectRequests(suspectRequests, cv);
@@ -1205,7 +1203,7 @@ public class GMSHealthMonitor implements HealthMonitor {
         }
       }
       List membersLeaving = new ArrayList();
-      for (InternalDistributedMember member : cv.getMembers()) {
+      for (GMSMember member : cv.getMembers()) {
         if (services.getJoinLeave().isMemberLeaving(member)) {
           membersLeaving.add(member);
         }
@@ -1219,7 +1217,7 @@ public class GMSHealthMonitor implements HealthMonitor {
           check,
           check.getCoordinator(), localAddress);
 
-      InternalDistributedMember coordinator = check.getCoordinator();
+      GMSMember coordinator = check.getCoordinator();
       if (coordinator != null && coordinator.equals(localAddress)) {
         // new coordinator
         logSuspectRequests(incomingRequest, sender);
@@ -1239,7 +1237,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
 
   private void logSuspectRequests(SuspectMembersMessage incomingRequest,
-      InternalDistributedMember sender) {
+      GMSMember sender) {
     for (SuspectRequest req : incomingRequest.getMembers()) {
       String who = sender.equals(localAddress) ? "myself" : sender.toString();
       logger.info("received suspect message from {} for {}: {}", who, req.getSuspectMember(),
@@ -1251,7 +1249,7 @@ public class GMSHealthMonitor implements HealthMonitor {
    * This method make sure that records suspectRequest. We need to make sure this on preferred
    * coordinators, as elder coordinator might be in suspected list next.
    */
-  private void recordSuspectRequests(List<SuspectRequest> suspectRequests, NetView cv) {
+  private void recordSuspectRequests(List<SuspectRequest> suspectRequests, GMSMembershipView cv) {
     // record suspect requests
     Set<SuspectRequest> suspectedMembers;
     synchronized (suspectRequestsInView) {
@@ -1269,11 +1267,11 @@ public class GMSHealthMonitor implements HealthMonitor {
    * available for the member (in the view) then we attempt to connect to its socket and ask if it's
    * the expected member. Otherwise we send a heartbeat request and wait for a reply.
    */
-  private void checkIfAvailable(final InternalDistributedMember initiator,
-      List<SuspectRequest> sMembers, final NetView cv) {
+  private void checkIfAvailable(final GMSMember initiator,
+      List<SuspectRequest> sMembers, final GMSMembershipView cv) {
 
     for (final SuspectRequest sr : sMembers) {
-      final InternalDistributedMember mbr = sr.getSuspectMember();
+      final GMSMember mbr = sr.getSuspectMember();
 
       if (!cv.contains(mbr) || membersInFinalCheck.contains(mbr)) {
         continue;
@@ -1298,8 +1296,8 @@ public class GMSHealthMonitor implements HealthMonitor {
     }
   }
 
-  protected boolean inlineCheckIfAvailable(final InternalDistributedMember initiator,
-      final NetView cv, boolean forceRemovalIfCheckFails, final InternalDistributedMember mbr,
+  protected boolean inlineCheckIfAvailable(final GMSMember initiator,
+      final GMSMembershipView cv, boolean forceRemovalIfCheckFails, final GMSMember mbr,
       final String reason) {
 
     if (services.getJoinLeave().isMemberLeaving(mbr)) {
@@ -1389,7 +1387,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
       if (!failed) {
         if (!isStopping && !initiator.equals(localAddress)
-            && initiator.getVersionObject().compareTo(Version.GEODE_1_3_0) >= 0) {
+            && initiator.getVersionOrdinal() >= Version.GEODE_1_3_0.ordinal()) {
           // let the sender know that it's okay to monitor this member again
           FinalCheckPassedMessage message = new FinalCheckPassedMessage(initiator, mbr);
           services.getMessenger().send(message);
@@ -1408,7 +1406,7 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   @Override
-  public void memberShutdown(DistributedMember mbr, String reason) {}
+  public void memberShutdown(GMSMember mbr, String reason) {}
 
   @Override
   public int getFailureDetectionPort() {
@@ -1416,16 +1414,16 @@ public class GMSHealthMonitor implements HealthMonitor {
   }
 
   @Override
-  public Collection<InternalDistributedMember> getMembersFailingAvailabilityCheck() {
+  public Collection<GMSMember> getMembersFailingAvailabilityCheck() {
     return Collections.unmodifiableCollection(this.suspectedMemberIds.keySet());
   }
 
   private void sendSuspectRequest(final List<SuspectRequest> requests) {
     logger.debug("Sending suspect request for members {}", requests);
-    List<InternalDistributedMember> recipients;
+    List<GMSMember> recipients;
     if (currentView.size() > ServiceConfig.SMALL_CLUSTER_SIZE) {
-      HashSet<InternalDistributedMember> filter = new HashSet<>();
-      for (Enumeration<InternalDistributedMember> e = suspectedMemberIds.keys(); e
+      HashSet<GMSMember> filter = new HashSet<>();
+      for (Enumeration<GMSMember> e = suspectedMemberIds.keys(); e
           .hasMoreElements();) {
         filter.add(e.nextElement());
       }
@@ -1440,7 +1438,7 @@ public class GMSHealthMonitor implements HealthMonitor {
 
     logger.trace("Sending suspect messages to {}", recipients);
     SuspectMembersMessage smm = new SuspectMembersMessage(recipients, requests);
-    Set<InternalDistributedMember> failedRecipients;
+    Set<GMSMember> failedRecipients;
     try {
       failedRecipients = services.getMessenger().send(smm);
       this.stats.incSuspectsSent();
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Authenticator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Authenticator.java
index bcb114e..b25b6b5 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Authenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Authenticator.java
@@ -16,7 +16,7 @@ package org.apache.geode.distributed.internal.membership.gms.interfaces;
 
 import java.util.Properties;
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
 
 public interface Authenticator {
 
@@ -28,7 +28,7 @@ public interface Authenticator {
    * @return null if authentication succeed (including no authenticator case), otherwise, return
    *         failure message
    */
-  String authenticate(InternalDistributedMember member, Properties credentials);
+  String authenticate(GMSMember member, Properties credentials);
 
   /**
    * Get credential object for the given GemFire distributed member.
@@ -36,5 +36,5 @@ public interface Authenticator {
    * @param member the target distributed member
    * @return the credentials
    */
-  Properties getCredentials(InternalDistributedMember member);
+  Properties getCredentials(GMSMember member);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java
new file mode 100644
index 0000000..074bf0e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java
@@ -0,0 +1,57 @@
+/*
+ * 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.distributed.internal.membership.gms.interfaces;
+
+import java.util.List;
+
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+
+public interface GMSMessage {
+
+  void setRecipient(GMSMember member);
+
+  void setRecipients(List<GMSMember> recipients);
+
+  /** is this a high priority message that should be sent out-of-band? */
+  boolean isHighPriority();
+
+
+  /** register any reply processor prior to transmission, if necessary */
+  void registerProcessor();
+
+  List<GMSMember> getRecipients();
+
+  /** from DataSerializableFixedID */
+  int getDSFID();
+
+  boolean forAll();
+
+  default boolean getMulticast() {
+    return false;
+  }
+
+  default void setMulticast(boolean useMulticast) {
+    // no-op by default
+  }
+
+  /** establishes the sender of a message on the receiving side of a communications channel */
+  void setSender(GMSMember sender);
+
+  GMSMember getSender();
+
+  void resetTimestamp();
+
+  void setBytesRead(int amount);
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java
index 9ea751c..c134088 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java
@@ -16,8 +16,7 @@ package org.apache.geode.distributed.internal.membership.gms.interfaces;
 
 import java.util.Collection;
 
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
 
 public interface HealthMonitor extends Service {
 
@@ -25,13 +24,13 @@ public interface HealthMonitor extends Service {
    * Note that this member has been contacted by the given member
    *
    */
-  void contactedBy(InternalDistributedMember sender);
+  void contactedBy(GMSMember sender);
 
   /**
    * initiate, asynchronously, suspicion that the member is no longer available
    *
    */
-  void suspect(InternalDistributedMember mbr, String reason);
+  void suspect(GMSMember mbr, String reason);
 
   /**
    * Check on the health of the given member, initiating suspicion if it fails. Return true if the
@@ -40,13 +39,13 @@ public interface HealthMonitor extends Service {
    * @param reason the reason this check is being performed
    * @param initiateRemoval if the member should be removed if it is not available
    */
-  boolean checkIfAvailable(InternalDistributedMember mbr, String reason, boolean initiateRemoval);
+  boolean checkIfAvailable(GMSMember mbr, String reason, boolean initiateRemoval);
 
   /**
    * Invoked by the Manager, this notifies the HealthMonitor that a ShutdownMessage has been
    * received from the given member
    */
-  void memberShutdown(DistributedMember mbr, String reason);
+  void memberShutdown(GMSMember mbr, String reason);
 
   /**
    * Returns the failure detection port for this member, or -1 if there is no such port
@@ -56,6 +55,6 @@ public interface HealthMonitor extends Service {
   /**
    * Returns the set of members declared dead by the health monitor
    */
-  Collection<InternalDistributedMember> getMembersFailingAvailabilityCheck();
+  Collection<GMSMember> getMembersFailingAvailabilityCheck();
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java
index 250b000..6954515 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java
@@ -14,10 +14,8 @@
  */
 package org.apache.geode.distributed.internal.membership.gms.interfaces;
 
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 
 public interface JoinLeave extends Service {
 
@@ -35,40 +33,40 @@ public interface JoinLeave extends Service {
   /**
    * force another member out of the system
    */
-  void remove(InternalDistributedMember m, String reason);
+  void remove(GMSMember m, String reason);
 
   /**
    * Invoked by the Manager, this notifies the HealthMonitor that a ShutdownMessage has been
    * received from the given member
    */
-  void memberShutdown(DistributedMember mbr, String reason);
+  void memberShutdown(GMSMember mbr, String reason);
 
   /**
    * returns the local address
    */
-  InternalDistributedMember getMemberID();
+  GMSMember getMemberID();
 
   /**
-   * Get "InternalDistributedMember" from current view or prepared view.
+   * Get canonical "GMSMember" from current view or prepared view.
    */
-  InternalDistributedMember getMemberID(NetMember m);
+  GMSMember getMemberID(GMSMember m);
 
   /**
    * returns the current membership view
    */
-  NetView getView();
+  GMSMembershipView getView();
 
 
   /**
    * returns the last known view prior to close - for reconnecting
    */
-  NetView getPreviousView();
+  GMSMembershipView getPreviousView();
 
   /**
    * check to see if a member is already in the process of leaving or being removed (in the next
    * view)
    */
-  boolean isMemberLeaving(DistributedMember mbr);
+  boolean isMemberLeaving(GMSMember mbr);
 
   /**
    * test hook
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Locator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Locator.java
index a365143..7adc123 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Locator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Locator.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.distributed.internal.membership.gms.interfaces;
 
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 
 /**
  * The Locator interface allows member services to interact with the Locator TcpHandler component of
@@ -24,7 +24,7 @@ public interface Locator {
   /**
    * called when a new view is installed by Membership
    */
-  void installView(NetView v);
+  void installView(GMSMembershipView v);
 
   /**
    * informs the locator that this process is becoming the membership coordinator
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java
index a76e8e2..46510d6 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java
@@ -14,20 +14,18 @@
  */
 package org.apache.geode.distributed.internal.membership.gms.interfaces;
 
-import java.io.NotSerializableException;
 import java.util.Collection;
-import java.util.Set;
 
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DistributionMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.internal.DataSerializableFixedID;
 
 /**
  * Manager presents the GMS services to the outside world and handles startup/shutdown race
  * conditions. It is also the default MessageHandler
  */
-public interface Manager extends Service, MessageHandler<DistributionMessage> {
+public interface Manager extends Service, MessageHandler<GMSMessage> {
 
   /**
    * After all services have been started this is used to join the distributed system
@@ -35,13 +33,6 @@ public interface Manager extends Service, MessageHandler<DistributionMessage> {
   void joinDistributedSystem();
 
   /**
-   * Sends a message using a selected distribution channel (e.g. Messenger or DirectChannel)
-   *
-   * @return a set of recipients that did not receive the message
-   */
-  Set<InternalDistributedMember> send(DistributionMessage m) throws NotSerializableException;
-
-  /**
    * initiates a Forced Disconnect, shutting down the distributed system and closing the cache
    *
    */
@@ -50,36 +41,7 @@ public interface Manager extends Service, MessageHandler<DistributionMessage> {
   /**
    * notifies the manager that membership quorum has been lost
    */
-  void quorumLost(Collection<InternalDistributedMember> failures, NetView view);
-
-  /**
-   * Notifies the manager that a member has contacted us who is not in the current membership view
-   *
-   */
-  void addSurpriseMemberForTesting(DistributedMember mbr, long birthTime);
-
-  /**
-   * Tests to see if the given member has been put into "shunned" state, meaning that it has left
-   * the distributed system and we should no longer process requests from it. Shunned status
-   * eventually times out.
-   *
-   * @return true if the member is shunned
-   */
-  boolean isShunned(DistributedMember mbr);
-
-  /**
-   * returns the lead member from the current membership view. This is typically the oldest member
-   * that is not an Admin or Locator member.
-   *
-   * @return the ID of the lead member
-   */
-  DistributedMember getLeadMember();
-
-  /**
-   * returns the coordinator of the current membership view. This is who created and distributed the
-   * view. See NetView.
-   */
-  DistributedMember getCoordinator();
+  void quorumLost(Collection<GMSMember> failures, GMSMembershipView view);
 
   /**
    * sometimes we cannot perform multicast messaging, such as during a rolling upgrade.
@@ -89,11 +51,6 @@ public interface Manager extends Service, MessageHandler<DistributionMessage> {
   boolean isMulticastAllowed();
 
   /**
-   * Returns the reason for a shutdown.
-   */
-  Throwable getShutdownCause();
-
-  /**
    * Returns true if a shutdown is in progress or has been completed . When it returns true,
    * shutdown message is already sent.
    */
@@ -111,9 +68,23 @@ public interface Manager extends Service, MessageHandler<DistributionMessage> {
   boolean isReconnectingDS();
 
   /**
-   * If this.isReconnectingDS() then this method will inform whether the reconnect
-   * has completed
+   * When Messenger receives a message from another node it may be in a form that
+   * Messenger can't deal with, depending on what payload was serialized. It may
+   * be a GMSMessage already or it may be a message wrapped in an adapter class
+   * that serializes a non-GMSMessage payload. (See GMSMessageAdapter, which
+   * wraps Geode DistributionMessages)
+   */
+  GMSMessage wrapMessage(Object receivedMessage);
+
+  /**
+   * When Messenger is going to transmit a message it gets the actual payload to serialize
+   * from this method
+   */
+  DataSerializableFixedID unwrapMessage(GMSMessage messageToSend);
+
+  /**
+   * Return the Services object owning this Manager service
    */
-  boolean isReconnectCompleted();
+  Services getServices();
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java
index 3e986f3..0a45aa1 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java
@@ -17,11 +17,9 @@ package org.apache.geode.distributed.internal.membership.gms.interfaces;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DistributionMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
-import org.apache.geode.distributed.internal.membership.QuorumChecker;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.messenger.GMSQuorumChecker;
 
 public interface Messenger extends Service {
   /**
@@ -33,34 +31,34 @@ public interface Messenger extends Service {
    * sends an asynchronous message when the membership view may not have been established. Returns
    * destinations that did not receive the message due to no longer being in the view
    */
-  Set<InternalDistributedMember> send(DistributionMessage m, NetView alternateView);
+  Set<GMSMember> send(GMSMessage m, GMSMembershipView alternateView);
 
   /**
    * sends an asynchronous message. Returns destinations that did not receive the message due to no
    * longer being in the view
    */
-  Set<InternalDistributedMember> send(DistributionMessage m);
+  Set<GMSMember> send(GMSMessage m);
 
   /**
    * sends an asynchronous message. Returns destinations that did not receive the message due to no
    * longer being in the view. Does not guarantee delivery of the message (no retransmissions)
    */
-  Set<InternalDistributedMember> sendUnreliably(DistributionMessage m);
+  Set<GMSMember> sendUnreliably(GMSMessage m);
 
   /**
    * returns the endpoint ID for this member
    */
-  InternalDistributedMember getMemberID();
+  GMSMember getMemberID();
 
   /**
    * check to see if a member ID has already been used
    */
-  boolean isOldMembershipIdentifier(DistributedMember id);
+  boolean isOldMembershipIdentifier(GMSMember id);
 
   /**
    * retrieves the quorum checker that is used during auto-reconnect attempts
    */
-  QuorumChecker getQuorumChecker();
+  GMSQuorumChecker getQuorumChecker();
 
   /**
    * test whether multicast is not only turned on but is working
@@ -78,7 +76,7 @@ public interface Messenger extends Service {
    * @param state messaging state is stored in this map
    * @param includeMulticast whether to record multicast state
    */
-  void getMessageState(InternalDistributedMember member, Map<String, Long> state,
+  void getMessageState(GMSMember member, Map<String, Long> state,
       boolean includeMulticast);
 
   /**
@@ -88,7 +86,7 @@ public interface Messenger extends Service {
    * @param member the member flushing operations to this member
    * @param state the state of that member's outgoing messaging to this member
    */
-  void waitForMessageState(InternalDistributedMember member, Map<String, Long> state)
+  void waitForMessageState(GMSMember member, Map<String, Long> state)
       throws InterruptedException;
 
   /**
@@ -96,14 +94,14 @@ public interface Messenger extends Service {
    *
    * @return byte[] public key for member
    */
-  byte[] getPublicKey(InternalDistributedMember mbr);
+  byte[] getPublicKey(GMSMember mbr);
 
   /**
    * Set public key of member.
    *
    */
 
-  void setPublicKey(byte[] publickey, InternalDistributedMember mbr);
+  void setPublicKey(byte[] publickey, GMSMember mbr);
 
   /**
    * Set cluster key in local member.Memebr calls when it gets cluster key in join response
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java
index 403518d..956d4f7 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java
@@ -14,8 +14,8 @@
  */
 package org.apache.geode.distributed.internal.membership.gms.interfaces;
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 
 /**
@@ -49,7 +49,7 @@ public interface Service {
   /**
    * called when a new view is installed by Membership
    */
-  void installView(NetView v);
+  void installView(GMSMembershipView v);
 
   /**
    * test method for simulating a sick/dead member
@@ -74,9 +74,9 @@ public interface Service {
   /**
    * a member is suspected of having crashed
    */
-  void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect,
+  void memberSuspected(GMSMember initiator, GMSMember suspect,
       String reason);
 
 
-  default void setLocalAddress(InternalDistributedMember address) {}
+  default void setLocalAddress(GMSMember address) {}
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
index 0ede03e..740d600 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
@@ -20,30 +20,29 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
-import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.messages.AbstractGMSMessage;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
 
-public class FindCoordinatorRequest extends HighPriorityDistributionMessage
+public class FindCoordinatorRequest extends AbstractGMSMessage
     implements PeerLocatorRequest {
 
-  private InternalDistributedMember memberID;
-  private Collection<InternalDistributedMember> rejectedCoordinators;
+  private GMSMember memberID;
+  private Collection<GMSMember> rejectedCoordinators;
   private int lastViewId;
   private byte[] myPublicKey;
   private int requestId;
   private String dhalgo;
 
-  public FindCoordinatorRequest(InternalDistributedMember myId) {
+  public FindCoordinatorRequest(GMSMember myId) {
     this.memberID = myId;
     this.dhalgo = "";
   }
 
-  public FindCoordinatorRequest(InternalDistributedMember myId,
-      Collection<InternalDistributedMember> rejectedCoordinators, int lastViewId, byte[] pk,
+  public FindCoordinatorRequest(GMSMember myId,
+      Collection<GMSMember> rejectedCoordinators, int lastViewId, byte[] pk,
       int requestId, String dhalgo) {
     this.memberID = myId;
     this.rejectedCoordinators = rejectedCoordinators;
@@ -57,7 +56,7 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
     // no-arg constructor for serialization
   }
 
-  public InternalDistributedMember getMemberID() {
+  public GMSMember getMemberID() {
     return memberID;
   }
 
@@ -69,7 +68,7 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
     return dhalgo;
   }
 
-  public Collection<InternalDistributedMember> getRejectedCoordinators() {
+  public Collection<GMSMember> getRejectedCoordinators() {
     return rejectedCoordinators;
   }
 
@@ -101,13 +100,15 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
     return requestId;
   }
 
+  // TODO serialization not backward compatible with 1.9 - may need InternalDistributedMember, not
+  // GMSMember
   @Override
   public void toData(DataOutput out) throws IOException {
-    DataSerializer.writeObject(this.memberID, out);
+    GMSUtil.writeMemberID(memberID, out);
     if (this.rejectedCoordinators != null) {
       out.writeInt(this.rejectedCoordinators.size());
-      for (InternalDistributedMember mbr : this.rejectedCoordinators) {
-        DataSerializer.writeObject(mbr, out);
+      for (GMSMember mbr : this.rejectedCoordinators) {
+        GMSUtil.writeMemberID(mbr, out);
       }
     } else {
       out.writeInt(0);
@@ -120,11 +121,11 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
 
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    this.memberID = DataSerializer.readObject(in);
+    this.memberID = GMSUtil.readMemberID(in);
     int size = in.readInt();
-    this.rejectedCoordinators = new ArrayList<InternalDistributedMember>(size);
+    this.rejectedCoordinators = new ArrayList<GMSMember>(size);
     for (int i = 0; i < size; i++) {
-      this.rejectedCoordinators.add((InternalDistributedMember) DataSerializer.readObject(in));
+      this.rejectedCoordinators.add(GMSUtil.readMemberID(in));
     }
     this.lastViewId = in.readInt();
     this.requestId = in.readInt();
@@ -133,11 +134,6 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
   }
 
   @Override
-  protected void process(ClusterDistributionManager dm) {
-    throw new IllegalStateException("this message should not be executed");
-  }
-
-  @Override
   public int hashCode() {
     final int prime = 31;
     int result = 1;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java
index 8847036..1b05a68 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java
@@ -23,22 +23,22 @@ import java.util.Objects;
 import java.util.Set;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.messages.AbstractGMSMessage;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
 
-public class FindCoordinatorResponse extends HighPriorityDistributionMessage
+public class FindCoordinatorResponse extends AbstractGMSMessage
     implements DataSerializableFixedID {
 
-  private InternalDistributedMember coordinator;
-  private InternalDistributedMember senderId;
+  private GMSMember coordinator;
+  private GMSMember senderId;
   private boolean fromView;
-  private NetView view;
-  private Set<InternalDistributedMember> registrants;
+  private GMSMembershipView view;
+  private Set<GMSMember> registrants;
   private boolean networkPartitionDetectionEnabled;
   private boolean usePreferredCoordinators;
   private boolean isShortForm;
@@ -47,9 +47,9 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
 
   private int requestId;
 
-  public FindCoordinatorResponse(InternalDistributedMember coordinator,
-      InternalDistributedMember senderId, boolean fromView, NetView view,
-      HashSet<InternalDistributedMember> registrants, boolean networkPartitionDectionEnabled,
+  public FindCoordinatorResponse(GMSMember coordinator,
+      GMSMember senderId, boolean fromView, GMSMembershipView view,
+      HashSet<GMSMember> registrants, boolean networkPartitionDectionEnabled,
       boolean usePreferredCoordinators, byte[] pk) {
     this.coordinator = coordinator;
     this.senderId = senderId;
@@ -62,8 +62,8 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
     this.coordinatorPublicKey = pk;
   }
 
-  public FindCoordinatorResponse(InternalDistributedMember coordinator,
-      InternalDistributedMember senderId, byte[] pk, int requestId) {
+  public FindCoordinatorResponse(GMSMember coordinator,
+      GMSMember senderId, byte[] pk, int requestId) {
     this.coordinator = coordinator;
     this.senderId = senderId;
     this.isShortForm = true;
@@ -99,7 +99,7 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
     return usePreferredCoordinators;
   }
 
-  public InternalDistributedMember getCoordinator() {
+  public GMSMember getCoordinator() {
     return coordinator;
   }
 
@@ -107,7 +107,7 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
    * When the response comes from a locator via TcpClient this will return the locators member ID.
    * If the locator hasn't yet joined this may be null.
    */
-  public InternalDistributedMember getSenderId() {
+  public GMSMember getSenderId() {
     return senderId;
   }
 
@@ -115,11 +115,11 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
     return fromView;
   }
 
-  public NetView getView() {
+  public GMSMembershipView getView() {
     return view;
   }
 
-  public Set<InternalDistributedMember> getRegistrants() {
+  public Set<GMSMember> getRegistrants() {
     return registrants;
   }
 
@@ -132,7 +132,8 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
           + ", viewId=" + (view == null ? "null" : view.getViewId()) + ", registrants="
           + (registrants == null ? "none" : registrants) + ", senderId=" + senderId
           + ", network partition detection enabled=" + this.networkPartitionDetectionEnabled
-          + ", locators preferred as coordinators=" + this.usePreferredCoordinators + ")";
+          + ", locators preferred as coordinators=" + this.usePreferredCoordinators
+          + ", view=" + view + ")";
     }
   }
 
@@ -148,10 +149,12 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
     return FIND_COORDINATOR_RESP;
   }
 
+  // TODO serialization not backward compatible with 1.9 - may need InternalDistributedMember, not
+  // GMSMember
   @Override
   public void toData(DataOutput out) throws IOException {
-    DataSerializer.writeObject(coordinator, out);
-    DataSerializer.writeObject(senderId, out);
+    GMSUtil.writeMemberID(coordinator, out);
+    GMSUtil.writeMemberID(senderId, out);
     InternalDataSerializer.writeByteArray(coordinatorPublicKey, out);
     InternalDataSerializer.writeString(rejectionMessage, out);
     out.writeBoolean(isShortForm);
@@ -159,13 +162,13 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
     out.writeBoolean(networkPartitionDetectionEnabled);
     out.writeBoolean(usePreferredCoordinators);
     DataSerializer.writeObject(view, out);
-    InternalDataSerializer.writeSet(registrants, out);
+    GMSUtil.writeSetOfMemberIDs(registrants, out);
   }
 
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    coordinator = DataSerializer.readObject(in);
-    senderId = DataSerializer.readObject(in);
+    coordinator = GMSUtil.readMemberID(in);
+    senderId = GMSUtil.readMemberID(in);
     coordinatorPublicKey = InternalDataSerializer.readByteArray(in);
     rejectionMessage = InternalDataSerializer.readString(in);
     isShortForm = in.readBoolean();
@@ -174,16 +177,11 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
       networkPartitionDetectionEnabled = in.readBoolean();
       usePreferredCoordinators = in.readBoolean();
       view = DataSerializer.readObject(in);
-      registrants = InternalDataSerializer.readHashSet(in);
+      registrants = GMSUtil.readHashSetOfMemberIDs(in);
     }
   }
 
   @Override
-  protected void process(ClusterDistributionManager dm) {
-    throw new IllegalStateException("this message should not be executed");
-  }
-
-  @Override
   public int hashCode() {
     return Objects.hash(senderId, view, registrants, requestId);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
index 286f791..131482d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
@@ -39,29 +39,19 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.annotations.VisibleForTesting;
-import org.apache.geode.cache.GemFireCache;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.LocatorStats;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember.InternalDistributedMemberWrapper;
-import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
-import org.apache.geode.distributed.internal.membership.gms.NetLocator;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.Locator;
 import org.apache.geode.distributed.internal.membership.gms.membership.HostAddress;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import org.apache.geode.distributed.internal.tcpserver.TcpClient;
-import org.apache.geode.distributed.internal.tcpserver.TcpServer;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.VersionedObjectInput;
 import org.apache.geode.internal.logging.LogService;
 
-public class GMSLocator implements Locator, NetLocator {
+public class GMSLocator implements Locator {
 
   static final int LOCATOR_FILE_STAMP = 0x7b8cf741;
 
@@ -73,22 +63,22 @@ public class GMSLocator implements Locator, NetLocator {
   private final String locatorString;
   private final List<HostAddress> locators;
   private final LocatorStats locatorStats;
-  private final Set<InternalDistributedMember> registrants = new HashSet<>();
-  private final Map<InternalDistributedMemberWrapper, byte[]> publicKeys =
+  private final Set<GMSMember> registrants = new HashSet<>();
+  private final Map<GMSMember.GMSMemberWrapper, byte[]> publicKeys =
       new ConcurrentHashMap<>();
   private final Path workingDirectory;
 
   private volatile boolean isCoordinator;
 
   private Services services;
-  private InternalDistributedMember localAddress;
+  private GMSMember localAddress;
 
   /**
    * The current membership view, or one recovered from disk. This is a copy-on-write variable.
    */
-  private NetView view;
+  private GMSMembershipView view;
 
-  private NetView recoveredView;
+  private GMSMembershipView recoveredView;
 
   private File viewFile;
 
@@ -117,16 +107,15 @@ public class GMSLocator implements Locator, NetLocator {
     this.workingDirectory = workingDirectory;
   }
 
-  @Override
-  public synchronized boolean setMembershipManager(MembershipManager mgr) {
+  public synchronized boolean setServices(Services pservices) {
     if (services == null || services.isStopped()) {
-      services = ((GMSMembershipManager) mgr).getServices();
+      services = pservices;
       localAddress = services.getMessenger().getMemberID();
       Objects.requireNonNull(localAddress, "member address should have been established");
       logger.info("Peer locator is connecting to local membership services with ID {}",
           localAddress);
       services.setLocator(this);
-      NetView newView = services.getJoinLeave().getView();
+      GMSMembershipView newView = services.getJoinLeave().getView();
       if (newView != null) {
         view = newView;
         recoveredView = null;
@@ -165,10 +154,10 @@ public class GMSLocator implements Locator, NetLocator {
     return viewFile;
   }
 
-  @Override
-  public void init(TcpServer server) throws InternalGemFireException {
+  public void init(String persistentFileIdentifier) throws InternalGemFireException {
     if (viewFile == null) {
-      viewFile = workingDirectory.resolve("locator" + server.getPort() + "view.dat").toFile();
+      viewFile =
+          workingDirectory.resolve("locator" + persistentFileIdentifier + "view.dat").toFile();
     }
     logger.info(
         "GemFire peer location service starting.  Other locators: {}  Locators preferred as coordinators: {}  Network partition detection enabled: {}  View persistence file: {}",
@@ -177,7 +166,7 @@ public class GMSLocator implements Locator, NetLocator {
   }
 
   @Override
-  public void installView(NetView view) {
+  public void installView(GMSMembershipView view) {
     synchronized (registrants) {
       registrants.clear();
     }
@@ -196,7 +185,6 @@ public class GMSLocator implements Locator, NetLocator {
     this.isCoordinator = isCoordinator;
   }
 
-  @Override
   public Object processRequest(Object request) {
     if (logger.isDebugEnabled()) {
       logger.debug("Peer locator processing {}", request);
@@ -230,7 +218,7 @@ public class GMSLocator implements Locator, NetLocator {
 
     if (services == null) {
       if (findRequest.getMyPublicKey() != null) {
-        publicKeys.put(new InternalDistributedMemberWrapper(findRequest.getMemberID()),
+        publicKeys.put(new GMSMember.GMSMemberWrapper(findRequest.getMemberID()),
             findRequest.getMyPublicKey());
       }
       logger.debug(
@@ -254,7 +242,7 @@ public class GMSLocator implements Locator, NetLocator {
       }
     }
 
-    NetView responseView = view;
+    GMSMembershipView responseView = view;
     if (responseView == null) {
       responseView = recoveredView;
     }
@@ -266,15 +254,15 @@ public class GMSLocator implements Locator, NetLocator {
       }
     }
 
-    InternalDistributedMember coordinator = null;
+    GMSMember coordinator = null;
     boolean fromView = false;
     if (responseView != null) {
       // if the ID of the requester matches an entry in the membership view then remove
       // that entry - it's obviously an old member since the ID has been reused
-      InternalDistributedMember requestingMemberID = findRequest.getMemberID();
-      for (InternalDistributedMember id : responseView.getMembers()) {
+      GMSMember requestingMemberID = findRequest.getMemberID();
+      for (GMSMember id : responseView.getMembers()) {
         if (requestingMemberID.compareTo(id, false) == 0) {
-          NetView newView = new NetView(responseView, responseView.getViewId());
+          GMSMembershipView newView = new GMSMembershipView(responseView, responseView.getViewId());
           newView.remove(id);
           responseView = newView;
           break;
@@ -293,17 +281,17 @@ public class GMSLocator implements Locator, NetLocator {
 
     if (coordinator == null) {
       // find the "oldest" registrant
-      Collection<InternalDistributedMember> rejections = findRequest.getRejectedCoordinators();
+      Collection<GMSMember> rejections = findRequest.getRejectedCoordinators();
       if (rejections == null) {
         rejections = Collections.emptyList();
       }
 
       synchronized (registrants) {
         coordinator = services.getJoinLeave().getMemberID();
-        for (InternalDistributedMember mbr : registrants) {
+        for (GMSMember mbr : registrants) {
           if (mbr != coordinator && (coordinator == null || mbr.compareTo(coordinator) < 0)) {
-            if (!rejections.contains(mbr) && (mbr.getNetMember().preferredForCoordinator()
-                || !mbr.getNetMember().isNetworkPartitionDetectionEnabled())) {
+            if (!rejections.contains(mbr) && (mbr.preferredForCoordinator()
+                || !mbr.isNetworkPartitionDetectionEnabled())) {
               coordinator = mbr;
             }
           }
@@ -336,7 +324,7 @@ public class GMSLocator implements Locator, NetLocator {
     }
   }
 
-  private void saveView(NetView view) {
+  private void saveView(GMSMembershipView view) {
     if (viewFile == null) {
       return;
     }
@@ -356,28 +344,25 @@ public class GMSLocator implements Locator, NetLocator {
     }
   }
 
-  @Override
   public void endRequest(Object request, long startTime) {
     locatorStats.endLocatorRequest(startTime);
   }
 
-  @Override
   public void endResponse(Object request, long startTime) {
     locatorStats.endLocatorResponse(startTime);
   }
 
-  public byte[] getPublicKey(InternalDistributedMember member) {
-    return publicKeys.get(new InternalDistributedMemberWrapper(member));
+  public byte[] getPublicKey(GMSMember member) {
+    return publicKeys.get(new GMSMember.GMSMemberWrapper(member));
   }
 
-  @Override
   public void shutDown() {
     // nothing to do for GMSLocator
     publicKeys.clear();
   }
 
   @VisibleForTesting
-  public List<InternalDistributedMember> getMembers() {
+  public List<GMSMember> getMembers() {
     if (view != null) {
       return new ArrayList<>(view.getMembers());
     }
@@ -386,12 +371,6 @@ public class GMSLocator implements Locator, NetLocator {
     }
   }
 
-  @Override
-  public void restarting(DistributedSystem ds, GemFireCache cache,
-      InternalConfigurationPersistenceService sharedConfig) {
-    setMembershipManager(((InternalDistributedSystem) ds).getDM().getMembershipManager());
-  }
-
   private void recover() throws InternalGemFireException {
     if (!recoverFromOtherLocators()) {
       recoverFromFile(viewFile);
@@ -444,17 +423,20 @@ public class GMSLocator implements Locator, NetLocator {
       if (version != Version.CURRENT_ORDINAL) {
         Version geodeVersion = Version.fromOrdinalNoThrow((short) version, false);
         logger.info("Peer locator found that persistent view was written with {}", geodeVersion);
+        if (version > Version.CURRENT_ORDINAL) {
+          return false;
+        }
         input = new VersionedObjectInput(input, geodeVersion);
       }
 
       recoveredView = DataSerializer.readObject(input);
       // this is not a valid view so it shouldn't have a usable Id
       recoveredView.setViewId(-1);
-      List<InternalDistributedMember> members = new ArrayList<>(recoveredView.getMembers());
+      List<GMSMember> members = new ArrayList<>(recoveredView.getMembers());
       // Remove locators from the view. Since we couldn't recover from an existing
       // locator we know that all of the locators in the view are defunct
-      for (InternalDistributedMember member : members) {
-        if (member.getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE) {
+      for (GMSMember member : members) {
+        if (member.getVmKind() == GMSMember.LOCATOR_DM_TYPE) {
           recoveredView.remove(member);
         }
       }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GetViewResponse.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GetViewResponse.java
index 1766226..aeb9984 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GetViewResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GetViewResponse.java
@@ -19,15 +19,15 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.Version;
 
 public class GetViewResponse implements DataSerializableFixedID {
 
-  private NetView view;
+  private GMSMembershipView view;
 
-  public GetViewResponse(NetView view) {
+  public GetViewResponse(GMSMembershipView view) {
     this.view = view;
   }
 
@@ -35,7 +35,7 @@ public class GetViewResponse implements DataSerializableFixedID {
     // no-arg constructor for serialization
   }
 
-  public NetView getView() {
+  public GMSMembershipView getView() {
     return view;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 86fa317..3ce099e 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -45,22 +45,19 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.GemFireConfigException;
 import org.apache.geode.SystemConnectException;
-import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionMessage;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetMember;
-import org.apache.geode.distributed.internal.membership.NetView;
 import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
 import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
 import org.apache.geode.distributed.internal.membership.gms.ServiceConfig;
 import org.apache.geode.distributed.internal.membership.gms.Services;
 import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
 import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorRequest;
 import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
+import org.apache.geode.distributed.internal.membership.gms.messages.AbstractGMSMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.HasMemberID;
 import org.apache.geode.distributed.internal.membership.gms.messages.InstallViewMessage;
 import org.apache.geode.distributed.internal.membership.gms.messages.JoinRequestMessage;
@@ -131,7 +128,7 @@ public class GMSJoinLeave implements JoinLeave {
   /**
    * my address
    */
-  private InternalDistributedMember localAddress;
+  private GMSMember localAddress;
 
   private Services services;
 
@@ -153,39 +150,39 @@ public class GMSJoinLeave implements JoinLeave {
   /**
    * the currently installed view. Guarded by viewInstallationLock
    */
-  private volatile NetView currentView;
+  private volatile GMSMembershipView currentView;
 
   /**
    * the previous view
    **/
-  private volatile NetView previousView;
+  private volatile GMSMembershipView previousView;
 
   /**
    * members who we have been declared dead in the current view
    */
-  private final Set<InternalDistributedMember> removedMembers = new HashSet<>();
+  private final Set<GMSMember> removedMembers = new HashSet<>();
 
   /**
    * members who we've received a leave message from
    **/
-  private final Set<InternalDistributedMember> leftMembers = new HashSet<>();
+  private final Set<GMSMember> leftMembers = new HashSet<>();
 
   /**
    * a new view being installed
    */
-  private volatile NetView preparedView;
+  private volatile GMSMembershipView preparedView;
 
   /**
    * the last view that conflicted with view preparation
    */
-  private NetView lastConflictingView;
+  private GMSMembershipView lastConflictingView;
 
   private List<HostAddress> locators;
 
   /**
    * a list of join/leave/crashes
    */
-  private final List<DistributionMessage> viewRequests = new LinkedList<DistributionMessage>();
+  private final List<AbstractGMSMessage> viewRequests = new LinkedList<>();
 
   /**
    * the established request collection jitter. This can be overridden for testing with
@@ -246,17 +243,17 @@ public class GMSJoinLeave implements JoinLeave {
   /**
    * the view where quorum was most recently lost
    */
-  NetView quorumLostView;
+  GMSMembershipView quorumLostView;
 
   static class SearchState {
     public int joinedMembersContacted;
-    Set<InternalDistributedMember> alreadyTried = new HashSet<>();
-    Set<InternalDistributedMember> registrants = new HashSet<>();
-    InternalDistributedMember possibleCoordinator;
+    Set<GMSMember> alreadyTried = new HashSet<>();
+    Set<GMSMember> registrants = new HashSet<>();
+    GMSMember possibleCoordinator;
     int viewId = -100;
     int locatorsContacted = 0;
     boolean hasContactedAJoinedLocator;
-    NetView view;
+    GMSMembershipView view;
     int lastFindCoordinatorInViewId = -1000;
     final Set<FindCoordinatorResponse> responses = new HashSet<>();
     public int responsesExpected;
@@ -325,7 +322,7 @@ public class GMSJoinLeave implements JoinLeave {
             state);
         if (found) {
           logger.info("found possible coordinator {}", state.possibleCoordinator);
-          if (localAddress.getNetMember().preferredForCoordinator()
+          if (localAddress.preferredForCoordinator()
               && state.possibleCoordinator.equals(this.localAddress)) {
             // if we haven't contacted a member of a cluster maybe this node should
             // become the coordinator.
@@ -420,7 +417,7 @@ public class GMSJoinLeave implements JoinLeave {
     SearchState state = searchState;
 
     // send a join request to the coordinator and wait for a response
-    InternalDistributedMember coord = state.possibleCoordinator;
+    GMSMember coord = state.possibleCoordinator;
     if (state.alreadyTried.contains(coord)) {
       logger.info("Probable coordinator is still {} - waiting for a join-response", coord);
     } else {
@@ -483,8 +480,8 @@ public class GMSJoinLeave implements JoinLeave {
           // reset joinResponse[0]
           joinResponse[0] = null;
           // we got view here that means either we have to wait for
-          NetView v = response.getCurrentView();
-          InternalDistributedMember coord = v.getCoordinator();
+          GMSMembershipView v = response.getCurrentView();
+          GMSMember coord = v.getCoordinator();
           if (searchState.alreadyTried.contains(coord)) {
             searchState.view = response.getCurrentView();
             // we already sent join request to it..so lets wait some more time here
@@ -509,7 +506,7 @@ public class GMSJoinLeave implements JoinLeave {
   }
 
   @Override
-  public boolean isMemberLeaving(DistributedMember mbr) {
+  public boolean isMemberLeaving(GMSMember mbr) {
     if (getPendingRequestIDs(LEAVE_REQUEST_MESSAGE).contains(mbr)
         || getPendingRequestIDs(REMOVE_MEMBER_REQUEST).contains(mbr)
         || !currentView.contains(mbr)) {
@@ -543,7 +540,7 @@ public class GMSJoinLeave implements JoinLeave {
     logger.info("Received a join request from {}", incomingRequest.getMemberID());
 
     if (!ALLOW_OLD_VERSION_FOR_TESTING
-        && incomingRequest.getMemberID().getVersionObject().compareTo(Version.CURRENT) < 0) {
+        && incomingRequest.getMemberID().getVersionOrdinal() < Version.CURRENT.ordinal()) {
       logger.warn("detected an attempt to start a peer using an older version of the product {}",
           incomingRequest.getMemberID());
       JoinResponseMessage m =
@@ -586,13 +583,13 @@ public class GMSJoinLeave implements JoinLeave {
     logger.info("received leave request from {} for {}", incomingRequest.getSender(),
         incomingRequest.getMemberID());
 
-    NetView v = currentView;
+    GMSMembershipView v = currentView;
     if (v == null) {
       recordViewRequest(incomingRequest);
       return;
     }
 
-    InternalDistributedMember mbr = incomingRequest.getMemberID();
+    GMSMember mbr = incomingRequest.getMemberID();
 
     logger.info(() -> "JoinLeave.processMessage(LeaveRequestMessage) invoked.  isCoordinator="
         + isCoordinator
@@ -613,7 +610,7 @@ public class GMSJoinLeave implements JoinLeave {
 
     if (!isCoordinator && !isStopping && !services.getCancelCriterion().isCancelInProgress()) {
       logger.info("Checking to see if I should become coordinator");
-      NetView check = new NetView(v, v.getViewId() + 1);
+      GMSMembershipView check = new GMSMembershipView(v, v.getViewId() + 1);
       check.remove(mbr);
       synchronized (removedMembers) {
         check.removeAll(removedMembers);
@@ -623,12 +620,12 @@ public class GMSJoinLeave implements JoinLeave {
         leftMembers.add(mbr);
         check.removeAll(leftMembers);
       }
-      Collection<InternalDistributedMember> suspectMembers =
+      Collection<GMSMember> suspectMembers =
           services.getHealthMonitor().getMembersFailingAvailabilityCheck();
       check.removeAll(suspectMembers);
       logger.info("View with removed and left members removed is {}", check);
       if (check.getCoordinator().equals(localAddress)) {
-        for (InternalDistributedMember suspect : suspectMembers) {
+        for (GMSMember suspect : suspectMembers) {
           recordViewRequest(
               new RemoveMemberMessage(localAddress, suspect, "Failed availability check"));
         }
@@ -656,11 +653,11 @@ public class GMSJoinLeave implements JoinLeave {
       return;
     }
 
-    NetView v = currentView;
+    GMSMembershipView v = currentView;
     boolean fromMe =
         incomingRequest.getSender() == null || incomingRequest.getSender().equals(localAddress);
 
-    InternalDistributedMember mbr = incomingRequest.getMemberID();
+    GMSMember mbr = incomingRequest.getMemberID();
 
     if (v != null && !v.contains(incomingRequest.getSender())) {
       logger.info("Membership ignoring removal request for " + mbr + " from non-member "
@@ -691,7 +688,7 @@ public class GMSJoinLeave implements JoinLeave {
 
     if (!isCoordinator && !isStopping && !services.getCancelCriterion().isCancelInProgress()) {
       logger.debug("Checking to see if I should become coordinator");
-      NetView check = new NetView(v, v.getViewId() + 1);
+      GMSMembershipView check = new GMSMembershipView(v, v.getViewId() + 1);
       synchronized (removedMembers) {
         removedMembers.add(mbr);
         check.addCrashedMembers(removedMembers);
@@ -727,7 +724,7 @@ public class GMSJoinLeave implements JoinLeave {
     }
   }
 
-  private void recordViewRequest(DistributionMessage request) {
+  private void recordViewRequest(AbstractGMSMessage request) {
     try {
       synchronized (viewRequests) {
         logger.debug("Recording the request to be processed in the next membership view");
@@ -741,9 +738,9 @@ public class GMSJoinLeave implements JoinLeave {
   }
 
   // for testing purposes, returns a copy of the view requests for verification
-  List<DistributionMessage> getViewRequests() {
+  List<AbstractGMSMessage> getViewRequests() {
     synchronized (viewRequests) {
-      return new LinkedList<DistributionMessage>(viewRequests);
+      return new LinkedList<>(viewRequests);
     }
   }
 
@@ -775,7 +772,7 @@ public class GMSJoinLeave implements JoinLeave {
    *
    * @param oldCoordinator may be null
    */
-  private void becomeCoordinator(InternalDistributedMember oldCoordinator) {
+  private void becomeCoordinator(GMSMember oldCoordinator) {
 
     assert Thread.holdsLock(viewInstallationLock);
 
@@ -792,7 +789,7 @@ public class GMSJoinLeave implements JoinLeave {
     }
     if (currentView == null) {
       // create the initial membership view
-      NetView newView = new NetView(this.localAddress);
+      GMSMembershipView newView = new GMSMembershipView(this.localAddress);
       newView.setFailureDetectionPort(localAddress,
           services.getHealthMonitor().getFailureDetectionPort());
       this.localAddress.setVmViewId(0);
@@ -802,18 +799,19 @@ public class GMSJoinLeave implements JoinLeave {
       startViewBroadcaster();
     } else {
       // create and send out a new view
-      NetView newView = copyCurrentViewAndAddMyAddress(oldCoordinator);
+      GMSMembershipView newView = copyCurrentViewAndAddMyAddress(oldCoordinator);
       createAndStartViewCreator(newView);
       startViewBroadcaster();
     }
   }
 
-  private void createAndStartViewCreator(NetView newView) {
+  private void createAndStartViewCreator(GMSMembershipView newView) {
     if (viewCreator == null || viewCreator.isShutdown()) {
       services.getMessenger().initClusterKey();
       viewCreator = new ViewCreator("Geode Membership View Creator");
       if (newView != null) {
-        viewCreator.setInitialView(newView, newView.getNewMembers(), newView.getShutdownMembers(),
+        viewCreator.setInitialView(newView, newView.getNewMembers(),
+            newView.getShutdownMembers(),
             newView.getCrashedMembers());
       }
       logger.info("ViewCreator starting on:" + localAddress);
@@ -821,18 +819,18 @@ public class GMSJoinLeave implements JoinLeave {
     }
   }
 
-  private NetView copyCurrentViewAndAddMyAddress(InternalDistributedMember oldCoordinator) {
+  private GMSMembershipView copyCurrentViewAndAddMyAddress(GMSMember oldCoordinator) {
     boolean testing = unitTesting.contains("noRandomViewChange");
-    NetView newView;
-    Set<InternalDistributedMember> leaving = new HashSet<>();
-    Set<InternalDistributedMember> removals;
+    GMSMembershipView newView;
+    Set<GMSMember> leaving = new HashSet<>();
+    Set<GMSMember> removals;
     synchronized (viewInstallationLock) {
-      int rand = testing ? 0 : NetView.RANDOM.nextInt(10);
+      int rand = testing ? 0 : GMSMembershipView.RANDOM.nextInt(10);
       int viewNumber = currentView.getViewId() + 5 + rand;
       if (this.localAddress.getVmViewId() < 0) {
         this.localAddress.setVmViewId(viewNumber);
       }
-      List<InternalDistributedMember> mbrs = new ArrayList<>(currentView.getMembers());
+      List<GMSMember> mbrs = new ArrayList<>(currentView.getMembers());
       if (!mbrs.contains(localAddress)) {
         mbrs.add(localAddress);
       }
@@ -847,7 +845,7 @@ public class GMSJoinLeave implements JoinLeave {
       }
       mbrs.removeAll(removals);
       mbrs.removeAll(leaving);
-      newView = new NetView(this.localAddress, viewNumber, mbrs, leaving, removals);
+      newView = new GMSMembershipView(this.localAddress, viewNumber, mbrs, leaving, removals);
       newView.setFailureDetectionPorts(currentView);
       newView.setPublicKeys(currentView);
       newView.setFailureDetectionPort(this.localAddress,
@@ -856,10 +854,10 @@ public class GMSJoinLeave implements JoinLeave {
     return newView;
   }
 
-  private void sendRemoveMessages(List<InternalDistributedMember> removals, List<String> reasons,
-      Set<InternalDistributedMember> oldIds) {
+  private void sendRemoveMessages(List<GMSMember> removals, List<String> reasons,
+      Set<GMSMember> oldIds) {
     Iterator<String> reason = reasons.iterator();
-    for (InternalDistributedMember mbr : removals) {
+    for (GMSMember mbr : removals) {
       // if olds not contains mbr then send remove request
       if (!oldIds.contains(mbr)) {
         RemoveMemberMessage response = new RemoveMemberMessage(mbr, mbr, reason.next());
@@ -875,7 +873,7 @@ public class GMSJoinLeave implements JoinLeave {
         || services.getManager().shutdownInProgress() || services.getManager().isShutdownStarted();
   }
 
-  boolean prepareView(NetView view, List<InternalDistributedMember> newMembers)
+  boolean prepareView(GMSMembershipView view, List<GMSMember> newMembers)
       throws InterruptedException {
     // GEODE-2193 - don't send a view with new members if we're shutting down
     if (isShuttingDown()) {
@@ -884,7 +882,7 @@ public class GMSJoinLeave implements JoinLeave {
     return sendView(view, true, this.prepareProcessor);
   }
 
-  void sendView(NetView view, List<InternalDistributedMember> newMembers)
+  void sendView(GMSMembershipView view, List<GMSMember> newMembers)
       throws InterruptedException {
     if (isShuttingDown()) {
       throw new InterruptedException("shutting down");
@@ -892,21 +890,22 @@ public class GMSJoinLeave implements JoinLeave {
     sendView(view, false, this.viewProcessor);
   }
 
-  private boolean sendView(NetView view, boolean preparing, ViewReplyProcessor viewReplyProcessor)
+  private boolean sendView(GMSMembershipView view, boolean preparing,
+      ViewReplyProcessor viewReplyProcessor)
       throws InterruptedException {
 
     int id = view.getViewId();
     InstallViewMessage msg = new InstallViewMessage(view,
         services.getAuthenticator().getCredentials(this.localAddress), preparing);
-    Set<InternalDistributedMember> recips = new HashSet<>(view.getMembers());
+    List<GMSMember> recips = new ArrayList<>(view.getMembers());
 
     // a recent member was seen not to receive a new view - I think this is why
     // recips.removeAll(newMembers); // new members get the view in a JoinResponseMessage
     recips.remove(this.localAddress); // no need to send it to ourselves
 
-    Set<InternalDistributedMember> responders = recips;
+    List<GMSMember> responders = recips;
     if (!view.getCrashedMembers().isEmpty()) {
-      recips = new HashSet<>(recips);
+      recips = new ArrayList<>(recips);
       recips.addAll(view.getCrashedMembers());
     }
 
@@ -922,7 +921,7 @@ public class GMSJoinLeave implements JoinLeave {
         thread.markViewCreatorForShutdown(view.getCoordinator());
         this.isCoordinator = false;
       }
-      installView(new NetView(view, view.getViewId()));
+      installView(new GMSMembershipView(view, view.getViewId()));
     }
 
     if (recips.isEmpty()) {
@@ -936,10 +935,10 @@ public class GMSJoinLeave implements JoinLeave {
 
... 3788 lines suppressed ...