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 2015/08/14 01:39:21 UTC

incubator-geode git commit: GEODE-77 fixing the inability to create multiple MembershipManagers ina single JVM

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-77 0a70d5140 -> ab68f4e68


GEODE-77 fixing the inability to create multiple MembershipManagers ina single JVM

The static MemberAttributes.DEFAULT was being used to pull together all
of the bits of a new members identity (InternalDistributedMember) but
this prevented multiple MembershipManagers from being instantiated w/o
ones identity bleeding into anothers.  Now the MembershipManager is the
assembling a single InternalDistributedMember instance, putting info
into it as it becomes available.  A new unit test ensures that this is
functioning correctly.


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

Branch: refs/heads/feature/GEODE-77
Commit: ab68f4e683ac784b2602adb3d57323d0ace75db0
Parents: 0a70d51
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Aug 13 16:26:30 2015 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Aug 13 16:36:44 2015 -0700

----------------------------------------------------------------------
 .../internal/AdminDistributedSystemImpl.java    |   2 +-
 .../client/internal/AuthenticateUserOp.java     |   2 +
 .../internal/DistributionManager.java           |  71 +--
 .../internal/LonerDistributionManager.java      |  14 +-
 .../membership/InternalDistributedMember.java   |  47 +-
 .../internal/membership/MemberAttributes.java   |   7 +-
 .../internal/membership/gms/GMSMember.java      |   6 +-
 .../membership/gms/GMSMemberFactory.java        |  38 +-
 .../internal/membership/gms/GMSUtil.java        |  14 -
 .../internal/membership/gms/Services.java       |   1 +
 .../gms/messenger/JGroupsMessenger.java         |  46 +-
 .../gms/messenger/JGroupsQuorumChecker.java     |   5 +
 .../gms/mgr/GMSMembershipManager.java           |  39 +-
 .../admin/remote/RemoteTransportConfig.java     |  28 +-
 .../internal/i18n/ParentLocalizedStrings.java   |   2 +-
 .../cache/ClientServerTimeSyncDUnitTest.java    |   1 +
 .../DistributedAckRegionCCEDUnitTest.java       |   4 +-
 .../distributed/DistributedSystemDUnitTest.java |   3 +-
 .../ConsoleDistributionManagerDUnitTest.java    |   2 +-
 .../internal/LocalDistributionManagerTest.java  | 475 -------------------
 .../membership/MembershipJUnitTest.java         | 160 ++++++-
 .../PartitionedRegionLoadModelJUnitTest.java    |   6 -
 .../DurableClientReconnectDUnitTest.java        |   2 +-
 23 files changed, 281 insertions(+), 694 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
index 049b121..d0a879a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
@@ -1735,7 +1735,7 @@ implements com.gemstone.gemfire.admin.AdminDistributedSystem,
         isMcastEnabled(), getDisableTcp(),
         getDisableAutoReconnect(),
         getBindAddress(), buildSSLConfig(), parseLocators(), 
-        getMembershipPortRange(), getTcpPort());
+        getMembershipPortRange(), getTcpPort(), DistributionManager.ADMIN_ONLY_DM_TYPE);
     return new GfManagerAgentConfig(
         getSystemName(), conf, logWriter, this.alertLevel.getSeverity(), this, this);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AuthenticateUserOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AuthenticateUserOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AuthenticateUserOp.java
index e3c7fa3..dc64e6c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AuthenticateUserOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AuthenticateUserOp.java
@@ -99,6 +99,7 @@ public class AuthenticateUserOp {
     public AuthenticateUserOpImpl(Connection con, ExecutablePool pool) {
       super(MessageType.USER_CREDENTIAL_MESSAGE, 1);
       byte[] credentialBytes = null;
+      // TODO this is not a valid way to create a member ID
       DistributedMember server = new InternalDistributedMember(con.getSocket()
           .getInetAddress(), con.getSocket().getPort(), false);
       DistributedSystem sys = InternalDistributedSystem.getConnectedInstance();
@@ -144,6 +145,7 @@ public class AuthenticateUserOp {
       hdos.writeLong(cnx.getConnectionID());
       if (this.securityProperties != null) {
         byte[] credentialBytes = null;
+        // TODO this is not a valid way to create a member ID
         DistributedMember server = new InternalDistributedMember(cnx
             .getSocket().getInetAddress(), cnx.getSocket().getPort(), false);
         DistributedSystem sys = InternalDistributedSystem

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index c65078d..33fd9db 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -53,17 +53,15 @@ import com.gemstone.gemfire.ToDataException;
 import com.gemstone.gemfire.admin.GemFireHealthConfig;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.DurableClientAttributes;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.Role;
 import com.gemstone.gemfire.distributed.internal.locks.ElderState;
 import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
 import com.gemstone.gemfire.distributed.internal.membership.MemberFactory;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
-import com.gemstone.gemfire.i18n.LogWriterI18n;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.NanoTimer;
 import com.gemstone.gemfire.internal.OSProcess;
@@ -75,7 +73,6 @@ import com.gemstone.gemfire.internal.admin.remote.RemoteGfManagerAgent;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.AlertAppender;
@@ -87,7 +84,6 @@ import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.tcp.ReenteredConnectException;
 import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
-import com.gemstone.gemfire.i18n.StringId;
 
 /**
  * The <code>DistributionManager</code> uses a {@link
@@ -345,8 +341,6 @@ public class DistributionManager
   protected boolean exceptionInThreads;
 
   static ThreadLocal isStartupThread = new ThreadLocal();
-  private static InheritableThreadLocal distributionManagerType =
-    new InheritableThreadLocal();
     
   protected volatile boolean shutdownMsgSent = false;
 
@@ -522,45 +516,6 @@ public class DistributionManager
   //////////////////////  Static Methods  //////////////////////
 
   /**
-   * Sets the distribution manager's type (using an InheritableThreadLocal).
-   *
-   * @since 3.5
-   */
-  protected static void setDistributionManagerType(int vmType) {
-    switch (vmType) {
-    case NORMAL_DM_TYPE:
-    case LONER_DM_TYPE:
-    case ADMIN_ONLY_DM_TYPE:
-    case LOCATOR_DM_TYPE:
-       distributionManagerType.set(Integer.valueOf(vmType));
-        break;
-    default:
-      throw new IllegalArgumentException(LocalizedStrings.DistributionManager_UNKNOWN_DISTRIBUTIONMANAGERTYPE_0.toLocalizedString(Integer.valueOf(vmType)));
-    }
-  }
-
-  /**
-   * Returns the DistributionManager type which should match {@link
-   * #NORMAL_DM_TYPE}, {@link #ADMIN_ONLY_DM_TYPE}, {@link #LOCATOR_DM_TYPE}
-   * or {@link #LONER_DM_TYPE}.
-   *
-   * <p>
-   * If the value is null, an Assertion error will occur.
-   * <p>
-   * This method is called from {@link 
-   * InternalDistributedMember} and {@link 
-   * com.gemstone.org.jgroups.protocols.TCPGOSSIP}, and the value is stored
-   * in an InheritableThreadLocal.
-   *
-   * @since 3.5
-   */
-  public static int getDistributionManagerType() {
-    Integer vmType = (Integer) distributionManagerType.get();
-    if (vmType == null) return 0;
-    return vmType.intValue();
-  }
-
-  /**
    * Given two DistributionManager ids, check to see if they are
    * from the same host address.
    * @param id1 a DistributionManager id
@@ -589,18 +544,20 @@ public class DistributionManager
     
     try {
 
+      int vmKind;
+      
       if (Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
         // if this DM is starting for a locator, set it to be a locator DM
-        setDistributionManagerType(LOCATOR_DM_TYPE);
+        vmKind = LOCATOR_DM_TYPE;
 
       } else if (isDedicatedAdminVM) {
-        setDistributionManagerType(ADMIN_ONLY_DM_TYPE);
+        vmKind = ADMIN_ONLY_DM_TYPE;
 
       } else {
-        setDistributionManagerType(NORMAL_DM_TYPE);
+        vmKind = NORMAL_DM_TYPE;
       }
     
-      RemoteTransportConfig transport = new RemoteTransportConfig(system.getConfig());
+      RemoteTransportConfig transport = new RemoteTransportConfig(system.getConfig(), vmKind);
       transport.setIsReconnectingDS(system.isReconnectingDS());
       transport.setOldDSMembershipInfo(system.oldDSMembershipInfo());
       
@@ -834,7 +791,7 @@ public class DistributionManager
   private DistributionManager(RemoteTransportConfig transport,
                               InternalDistributedSystem system) {
 
-    this.dmType = getDistributionManagerType();
+    this.dmType = transport.getVmKind();
     this.system = system;
     this.elderLock = new StoppableReentrantLock(stopper);
     this.transport = transport;
@@ -1186,18 +1143,6 @@ public class DistributionManager
 
     // connect to JGroups
     start = System.currentTimeMillis();
-    DistributionConfig config = system.getConfig();
-    DurableClientAttributes dac = null;
-    if (config.getDurableClientId() != null) {
-      dac = new DurableClientAttributes(config.getDurableClientId(), config
-          .getDurableClientTimeout());
-    }
-    MemberAttributes.setDefaults(-1, 
-        OSProcess.getId(), 
-        getDistributionManagerType(), -1, 
-        config.getName(),
-        MemberAttributes.parseGroups(config.getRoles(), config.getGroups()),
-        dac);
     
     MyListener l = new MyListener(this);
     membershipManager = MemberFactory.newMembershipManager(l, system.getConfig(), transport, stats);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
index 5194d9e..8b70104 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
@@ -678,12 +678,8 @@ public class LonerDistributionManager implements DM {
         dac = new DurableClientAttributes(config.getDurableClientId(), config
             .getDurableClientTimeout());
       }
-      MemberAttributes.setDefaults(lonerPort,
-              com.gemstone.gemfire.internal.OSProcess.getId(),
-              DistributionManager.LONER_DM_TYPE, -1,
-              name,
-              MemberAttributes.parseGroups(config.getRoles(), config.getGroups()), dac);
-      result = new InternalDistributedMember(host, lonerPort, name, uniqueString);
+      result = new InternalDistributedMember(host, lonerPort, name, uniqueString, DistributionManager.LONER_DM_TYPE,
+          MemberAttributes.parseGroups(config.getRoles(), config.getGroups()), dac);
 
     } catch (UnknownHostException ex) {
       throw new InternalGemFireError(LocalizedStrings.LonerDistributionManager_CANNOT_RESOLVE_LOCAL_HOST_NAME_TO_AN_IP_ADDRESS.toLocalizedString());
@@ -702,12 +698,6 @@ public class LonerDistributionManager implements DM {
     this.logger.config(LocalizedStrings.LonerDistributionmanager_CHANGING_PORT_FROM_TO,
         new Object[]{this.lonerPort, newPort});
     this.lonerPort = newPort;
-    MemberAttributes.setDefaults(lonerPort,
-        MemberAttributes.DEFAULT.getVmPid(),
-        DistributionManager.LONER_DM_TYPE,
-        -1,
-        MemberAttributes.DEFAULT.getName(),
-        MemberAttributes.DEFAULT.getGroups(), MemberAttributes.DEFAULT.getDurableClientAttributes());
     this.getId().setPort(this.lonerPort);
   }
   public boolean isCurrentMember(InternalDistributedMember p_id) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
index 1fefcb9..5b957c3 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.versions.VersionSource;
@@ -41,17 +42,6 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 /**
  * 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.
- *
- * TODO fix this.
  */
 public final class InternalDistributedMember
  implements DistributedMember,
@@ -81,7 +71,7 @@ public final class InternalDistributedMember
    * This is a representation of the type of VM. The underlying NetMember must
    * be able to serialize and deliver this value.
    */
-  private int vmKind = -1;
+  private int vmKind = DistributionManager.NORMAL_DM_TYPE;
   
   /**
    * This is the view identifier where this ID was born, or zero if this is
@@ -151,14 +141,7 @@ public final class InternalDistributedMember
         Version.GFE_71, Version.GFE_90 };
 
   private void defaultToCurrentHost() {
-    int defaultDcPort = MemberAttributes.DEFAULT.getPort();
-    this.dcPort = defaultDcPort;;
-    this.vmKind = MemberAttributes.DEFAULT.getVmKind();
-    this.vmPid = MemberAttributes.DEFAULT.getVmPid();
-    this.name = MemberAttributes.DEFAULT.getName();
-    this.groups = MemberAttributes.DEFAULT.getGroups();
-    this.vmViewId = MemberAttributes.DEFAULT.getVmViewId();
-    this.durableClientAttributes = MemberAttributes.DEFAULT.getDurableClientAttributes();
+    this.vmPid = OSProcess.getId();
     try {
       if (SocketCreator.resolve_dns) {
         this.hostName = SocketCreator.getHostName(SocketCreator.getLocalHost());
@@ -318,7 +301,7 @@ public final class InternalDistributedMember
    * WITHOUT TALKING TO ME FIRST.  IT DOES NOT PROPERLY INITIALIZE THE ID.
    * </b>
    *
-   * @param i
+   * @param host
    *          the hostname, must be for the current host
    * @param p
    *          the membership listening port
@@ -326,13 +309,29 @@ public final class InternalDistributedMember
    *          gemfire properties connection name
    * @param u
    *          unique string used make the member more unique
+   * @param vmKind the dmType
+   * @param groups the server groups / roles
+   * @param attr durable client attributes, if any
+   * 
    * @throws UnknownHostException if the given hostname cannot be resolved
    */
-  public InternalDistributedMember(String i, int p, String n, String u) throws UnknownHostException {
-    netMbr = MemberFactory.newNetMember(i, p);
+  public InternalDistributedMember(String host, int p, String n, String u,
+      int vmKind, String[] groups, DurableClientAttributes attr) throws UnknownHostException {
+    MemberAttributes mattr = new MemberAttributes(p,
+        com.gemstone.gemfire.internal.OSProcess.getId(),
+        vmKind, -1,
+        n,
+        groups, attr);
+    InetAddress addr = SocketCreator.toInetAddress(host);
+    netMbr = MemberFactory.newNetMember(addr, p, false, true, Version.CURRENT_ORDINAL, mattr);
     defaultToCurrentHost();
     this.name = n;
     this.uniqueTag = u;
+    this.vmKind = vmKind;
+    this.dcPort = p;
+    this.durableClientAttributes = attr;
+    this.hostName = host;
+    this.vmPid = OSProcess.getId();
   }
 
   /**
@@ -451,7 +450,7 @@ public final class InternalDistributedMember
       if (tmpRolesSet == null) {
         final String[] tmpRoles = this.groups;
         // convert array of string role names to array of Roles...
-        if (tmpRoles.length == 0) {
+        if (tmpRoles == null  ||  tmpRoles.length == 0) {
           tmpRolesSet = Collections.emptySet();
         }
         else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
index 54fd306..3ce4275 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MemberAttributes.java
@@ -29,11 +29,6 @@ public class MemberAttributes implements DataSerializable {
   private static final long serialVersionUID = -3257772958884802693L;
   
   public static final MemberAttributes INVALID = new MemberAttributes(-1, -1, -1, -1, null, null, null);
-  public static MemberAttributes DEFAULT = INVALID;
-  
-  public static void setDefaults(int dcPort, int vmPid, int vmKind, int vmViewId, String name, String[] groups, DurableClientAttributes durableClientAttributes) {
-    DEFAULT = new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups, durableClientAttributes);
-  }
   
   private int dcPort;
   private int vmPid;
@@ -55,12 +50,12 @@ public class MemberAttributes implements DataSerializable {
     if (l_groups == null) {
       l_groups = new String[0];
     }
+    this.groups = l_groups;
     if (p_name == null) {
       this.name = "";
     } else {
       this.name = p_name;
     }
-    this.groups = l_groups;
     this.durableClientAttributes = durableClientAttributes;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
index da9ab99..619481e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
@@ -17,6 +17,7 @@ import org.jgroups.util.UUID;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.distributed.DurableClientAttributes;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
 import com.gemstone.gemfire.distributed.internal.membership.NetMember;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
@@ -112,7 +113,6 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
    * @param p the membership listening port
    */
   public GMSMember(String i, int p) {
-    setAttributes(MemberAttributes.DEFAULT);
     udpPort=p;
     try {
       inetAddr=InetAddress.getByName(i);
@@ -133,10 +133,10 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
    * @param msbs - most significant bytes of UUID
    * @param lsbs - least significant bytes of UUID
    */
-  public GMSMember(InetAddress i, int p, boolean splitBrainEnabled, boolean preferredForCoordinator,
+  public GMSMember(MemberAttributes attr, InetAddress i, int p, boolean splitBrainEnabled, boolean preferredForCoordinator,
       short version,
       long msbs, long lsbs) {
-    setAttributes(MemberAttributes.DEFAULT);
+    setAttributes(attr);
     this.inetAddr = i;
     this.udpPort=p;
     this.splitBrainEnabled = splitBrainEnabled;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
index 52e8054..34e1123 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberFactory.java
@@ -9,23 +9,19 @@ package com.gemstone.gemfire.distributed.internal.membership.gms;
 
 import java.io.File;
 import java.net.InetAddress;
+import java.net.UnknownHostException;
 
-import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.GemFireConfigException;
 import com.gemstone.gemfire.SystemConnectException;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionException;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
 import com.gemstone.gemfire.distributed.internal.membership.MemberServices;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.NetMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.locator.GMSLocator;
-import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
-import com.gemstone.gemfire.distributed.internal.tcpserver.TcpHandler;
-import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -51,26 +47,10 @@ public class GMSMemberFactory implements MemberServices {
    */
   public NetMember newNetMember(InetAddress i, int p, boolean splitBrainEnabled,
       boolean canBeCoordinator, MemberAttributes attr, short version) {
-    GMSMember result = new GMSMember(i, p, splitBrainEnabled, canBeCoordinator, version, 0, 0);
-    result.setAttributes(attr);
+    GMSMember result = new GMSMember(attr, i, p, splitBrainEnabled, canBeCoordinator, version, 0, 0);
     return result;
   }
 
-  private MemberAttributes getDefaultAttributes() {
-    // TODO can we get rid of this??
-    if (MemberAttributes.DEFAULT.getVmPid() == -1 ||
-        MemberAttributes.DEFAULT.getVmKind() == -1) {
-      MemberAttributes.setDefaults(
-          -1, 
-          OSProcess.getId(), 
-          -1,
-          DistributionManager.getDistributionManagerType(), 
-          null,
-          null, null);
-    }
-    return MemberAttributes.DEFAULT;
-  }
-  
   /**
    * Return a new NetMember representing current host.  This assumes that
    * the member does not have network partition detection enabled and can
@@ -80,9 +60,7 @@ public class GMSMemberFactory implements MemberServices {
    * @return the new NetMember
    */
   public NetMember newNetMember(InetAddress i, int p) {
-    GMSMember result = new GMSMember(i, p, false, true, Version.CURRENT_ORDINAL, 0, 0);
-    result.setAttributes(getDefaultAttributes());
-    return result;
+    return new GMSMember(MemberAttributes.INVALID, i, p, false, true, Version.CURRENT_ORDINAL, 0, 0);
   }
 
   /**
@@ -93,9 +71,13 @@ public class GMSMemberFactory implements MemberServices {
    * @return the new member
    */
   public NetMember newNetMember(String s, int p) {
-    GMSMember result = new GMSMember(s, p);
-    result.setAttributes(getDefaultAttributes());
-    return result;
+    InetAddress inetAddr = null;
+    try {
+      inetAddr=InetAddress.getByName(s);
+    } catch (UnknownHostException e) {
+      throw new RuntimeException("Unable to create an identifier for testing for " + s, e);
+    }
+    return newNetMember(inetAddr, p);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
index abd9f12..9e6922e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSUtil.java
@@ -79,18 +79,4 @@ public class GMSUtil {
     return result;
   }
 
-  public static void insertGemFireAttributes(GMSMember addr, MemberAttributes attr) {
-    addr.setProcessId(attr.getVmPid());
-    addr.setVmKind(attr.getVmKind());
-    addr.setDirectPort(attr.getPort());
-    addr.setBirthViewId(attr.getVmViewId());
-    addr.setName(attr.getName());
-    addr.setGroups(attr.getGroups());
-    addr.setDurableClientAttributes(attr.getDurableClientAttributes());
-  }
-
-  public static void insertDefaultGemFireAttributes(GMSMember addr) {
-    MemberAttributes attr = com.gemstone.gemfire.distributed.internal.membership.MemberAttributes.DEFAULT;
-    insertGemFireAttributes(addr, attr);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
index 8a27cf0..3907660 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
@@ -28,6 +28,7 @@ import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
+import com.gemstone.gemfire.internal.logging.log4j.FastLogger;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 
 public class Services {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 4104833..ab0aa31 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+
 import static com.gemstone.gemfire.internal.DataSerializableFixedID.JOIN_RESPONSE;
 
 import org.apache.logging.log4j.Logger;
@@ -45,6 +46,7 @@ import com.gemstone.gemfire.ForcedDisconnectException;
 import com.gemstone.gemfire.GemFireConfigException;
 import com.gemstone.gemfire.SystemConnectException;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.DurableClientAttributes;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -235,10 +237,17 @@ public class JGroupsMessenger implements Messenger {
     long start = System.currentTimeMillis();
     
     // start the jgroups channel and establish the membership ID
+    boolean reconnecting = false;
     try {
-      InputStream is = new ByteArrayInputStream(properties.getBytes("UTF-8"));
-      myChannel = new JChannel(is);
-      
+      Object oldChannel = services.getConfig().getTransport().getOldDSMembershipInfo();
+      if (oldChannel != null) {
+        myChannel = (JChannel)oldChannel;
+        reconnecting = true;
+      }
+      else {
+        InputStream is = new ByteArrayInputStream(properties.getBytes("UTF-8"));
+        myChannel = new JChannel(is);
+      }
     } catch (Exception e) {
       throw new GemFireConfigException("unable to create jgroups channel", e);
     }
@@ -246,8 +255,9 @@ public class JGroupsMessenger implements Messenger {
     try {
       
       myChannel.setReceiver(new JGroupsReceiver());
-      myChannel.connect("AG"); // apache g***** (whatever we end up calling it)
-      
+      if (!reconnecting) {
+        myChannel.connect("AG"); // apache g***** (whatever we end up calling it)
+      }
     } catch (Exception e) {
       throw new SystemConnectException("unable to create jgroups channel", e);
     }
@@ -330,12 +340,25 @@ public class JGroupsMessenger implements Messenger {
     myChannel.down(new Event(Event.SET_LOCAL_ADDRESS, this.jgAddress));
 
     DistributionConfig config = services.getConfig().getDistributionConfig();
-    boolean isLocator = (MemberAttributes.DEFAULT.getVmKind() == DistributionManager.LOCATOR_DM_TYPE); 
+    boolean isLocator = (services.getConfig().getTransport().getVmKind() == DistributionManager.LOCATOR_DM_TYPE); 
     
     // establish the DistributedSystem's address
+    DurableClientAttributes dca = null;
+    if (config.getDurableClientId() != null) {
+      dca = new DurableClientAttributes(config.getDurableClientId(), config
+          .getDurableClientTimeout());
+    }
+    MemberAttributes attr = new MemberAttributes(
+        -1/*dcPort - not known at this time*/,
+        OSProcess.getId(),
+        services.getConfig().getTransport().getVmKind(),
+        -1/*view id - not known at this time*/,
+        config.getName(),
+        MemberAttributes.parseGroups(config.getRoles(), config.getGroups()),
+        dca);
     localAddress = new InternalDistributedMember(jgAddress.getInetAddress(),
         jgAddress.getPort(), config.getEnableNetworkPartitionDetection(),
-        isLocator, MemberAttributes.DEFAULT);
+        isLocator, attr);
 
     // add the JGroups logical address to the GMSMember
     UUID uuid = this.jgAddress;
@@ -682,6 +705,13 @@ public class JGroupsMessenger implements Messenger {
   public InternalDistributedMember getMemberID() {
     return localAddress;
   }
+  
+  /**
+   * returns the JGroups configuration string
+   */
+  public String getJGroupsStackConfig() {
+    return this.jgStackConfig;
+  }
 
   /**
    * returns the member ID for the given GMSMember object
@@ -759,7 +789,7 @@ public class JGroupsMessenger implements Messenger {
       // admin-only VMs don't have caches, so we ignore cache operations
       // multicast to them, avoiding deserialization cost and classpath
       // problems
-      if ( (MemberAttributes.DEFAULT.getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE)
+      if ( (services.getConfig().getTransport().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE)
            && (msg instanceof DistributedCacheOperation.CacheOperationMessage)) {
         if (logger.isTraceEnabled())
           logger.trace("Membership: admin VM discarding cache operation message {}", jgmsg.getObject());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsQuorumChecker.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsQuorumChecker.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsQuorumChecker.java
index f922275..ed96879 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsQuorumChecker.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsQuorumChecker.java
@@ -21,10 +21,12 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.logging.log4j.Logger;
+import org.jgroups.Message;
 
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.QuorumChecker;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
 import com.gemstone.gemfire.internal.concurrent.ConcurrentHashSet;
 import com.gemstone.gemfire.internal.logging.LogService;
 
@@ -128,6 +130,9 @@ public class JGroupsQuorumChecker implements QuorumChecker {
             logger.debug("quorum check: sending request to {}", addr);
           }
           try {
+            Message msg = new Message();
+//            msg.setDest(new JGAddress((GMSMember)addr.getNetMember()));
+//            msg.setObject(obj)
             DatagramPacket packet = new DatagramPacket(buffer, 0, buffer.length, sockaddr);
             this.sock.send(packet);
           } catch (IOException io) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 833f677..b9425c4 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -874,12 +874,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     
     services.getMessenger().getMemberID().setDirectChannelPort(dcPort);
 
-    MemberAttributes.setDefaults(dcPort,
-        MemberAttributes.DEFAULT.getVmPid(),
-        MemberAttributes.DEFAULT.getVmKind(),
-        MemberAttributes.DEFAULT.getVmViewId(),
-        MemberAttributes.DEFAULT.getName(),
-        MemberAttributes.DEFAULT.getGroups(), MemberAttributes.DEFAULT.getDurableClientAttributes());
   }
   
   
@@ -904,13 +898,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       dcPort = directChannel.getPort();
     }
     
-    MemberAttributes.setDefaults(dcPort,
-        MemberAttributes.DEFAULT.getVmPid(),
-        MemberAttributes.DEFAULT.getVmKind(),
-        address.getVmViewId(),
-        MemberAttributes.DEFAULT.getName(),
-        MemberAttributes.DEFAULT.getGroups(), MemberAttributes.DEFAULT.getDurableClientAttributes());
-    
     if (directChannel != null) {
       directChannel.setLocalAddr(address);
       Stub stub = directChannel.getLocalStub();
@@ -1744,10 +1731,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
 
   
-  public void shutdown()
-  {
-    setShutdown(); // for safety
-    services.stop();
+  public void shutdown() {
+    if (!shutdownInProgress) {
+      setShutdown();
+      services.stop();
+    }
   }
   
   @Override
@@ -1788,11 +1776,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     // members
     services.emergencyClose();
     
-    // we have to clear the view before notifying the membership listener,
-    // so that it won't try sending disconnect messages to members that
-    // aren't there.  Otherwise, it sends the disconnect messages to other
-    // members, they ignore the "surprise" connections, and we hang.
-    //GroupMembershipService.this.clearView();
     if (e != null) {
       try {
         if (membershipTestHooks != null) {
@@ -2045,15 +2028,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
   /**
-   * A quorum checker is used during reconnect to perform quorum
-   * probes.  It is made available here for the UDP protocol to
-   * hand off ping-pong responses to the checker.
-   */
-  public QuorumChecker getQuorumCheckerImpl() {
-    return this.quorumChecker;
-  }
-  
-  /**
    * During jgroups connect the UDP protocol will invoke
    * this method to find the DatagramSocket it should use instead of
    * creating a new one.
@@ -2281,7 +2255,8 @@ public class GMSMembershipManager implements MembershipManager, Manager
   public boolean shutdownInProgress() {
     // Impossible condition (bug36329): make sure that we check DM's
     // view of shutdown here
-    return shutdownInProgress || listener.getDM().shutdownInProgress();
+    DistributionManager dm = listener.getDM();
+    return shutdownInProgress || (dm != null && dm.shutdownInProgress());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteTransportConfig.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteTransportConfig.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteTransportConfig.java
index 28db8cc..ac16dc3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteTransportConfig.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteTransportConfig.java
@@ -7,13 +7,18 @@
  */
 package com.gemstone.gemfire.internal.admin.remote;
 
-import com.gemstone.gemfire.GemFireConfigException;
-import com.gemstone.gemfire.distributed.internal.*;
-import com.gemstone.gemfire.internal.*;
-import com.gemstone.gemfire.internal.admin.*;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Properties;
+import java.util.Set;
 
-import java.util.*;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.admin.SSLConfig;
+import com.gemstone.gemfire.internal.admin.TransportConfig;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
 /**
  * Tranport config for RemoteGfManagerAgent.
@@ -37,6 +42,7 @@ public class RemoteTransportConfig implements TransportConfig {
   private int tcpPort;
   private boolean isReconnectingDS;
   private Object oldDSMembershipInfo;
+  private int vmKind = -1;
 
   // -------------------------------------------------------------------------
   //   Constructor(s)
@@ -50,12 +56,13 @@ public class RemoteTransportConfig implements TransportConfig {
    *
    * @since 3.0
    */
-  public RemoteTransportConfig(DistributionConfig config) {
+  public RemoteTransportConfig(DistributionConfig config, int vmKind) {
     if (config.getBindAddress() == null) {
       this.bindAddress = DistributionConfig.DEFAULT_BIND_ADDRESS;
     } else {
       this.bindAddress = config.getBindAddress();
     }
+    this.vmKind = vmKind;
     this.tcpPort = config.getTcpPort();
     this.membershipPortRange = 
             getMembershipPortRangeString(config.getMembershipPortRange());
@@ -125,7 +132,7 @@ public class RemoteTransportConfig implements TransportConfig {
     String bindAddress, 
     SSLConfig sslConfig,
     Collection ids, String membershipPortRange,
-    int tcpPort)
+    int tcpPort, int vmKind)
   {
     DistributionLocatorId mid = null;
     
@@ -163,6 +170,7 @@ public class RemoteTransportConfig implements TransportConfig {
     }
     this.membershipPortRange = membershipPortRange;
     this.tcpPort = tcpPort;
+    this.vmKind = vmKind;
  }
   
   
@@ -201,6 +209,10 @@ public class RemoteTransportConfig implements TransportConfig {
     return this.mcastId;
   }
   
+  public int getVmKind() {
+    return this.vmKind;
+  }
+  
   public boolean isTcpDisabled() {
     return this.tcpDisabled;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
index 875902c..d3ea781 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
@@ -191,7 +191,7 @@ class ParentLocalizedStrings {
   public static final StringId CacheDistributionAdvisor_WAITING_FOR_CURRENT_OPERATIONS_TO_FINISH_0_ = new StringId(1178, "Waiting for current operations to finish( {0} )");
   public static final StringId CacheFactory_0_AN_OPEN_CACHE_ALREADY_EXISTS = new StringId(1179, "{0}: An open cache already exists.");
   public static final StringId InternalDistributedSystem_shutdownHook_shuttingdown = new StringId(1180, "VM is exiting - shutting down distributed system");
-  public static final StringId GroupMembershipService_entered_into_membership_in_group_0_with_id_1 = new StringId(1181, "Finished joining (took {}ms).");
+  public static final StringId GroupMembershipService_entered_into_membership_in_group_0_with_id_1 = new StringId(1181, "Finished joining (took {0}ms).");
   public static final StringId CacheServerLauncher_CACHE_SERVER_ERROR = new StringId(1182, "Cache server error");
   public static final StringId CacheXmlParser_XML_PARSER_CHARACTERS_APPENDED_CHARACTER_DATA_0 = new StringId(1183, "XML Parser characters, appended character data: {0}");
   public static final StringId CacheXmlParser_XML_PARSER_CHARACTERS_NEW_CHARACTER_DATA_0 = new StringId(1184, "XML Parser characters, new character data: {0}");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientServerTimeSyncDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientServerTimeSyncDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientServerTimeSyncDUnitTest.java
index d980bc8..f9c9374 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientServerTimeSyncDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientServerTimeSyncDUnitTest.java
@@ -97,6 +97,7 @@ public class ClientServerTimeSyncDUnitTest extends CacheTestCase {
     }
   }
 
+  @Ignore("not yet implemented")
   public void testClientTimeSlowsDown() {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0); // Server

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
index 6ad43f8..fd92cf0 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
@@ -30,6 +30,7 @@ import com.gemstone.gemfire.cache.util.GatewayConflictResolver;
 import com.gemstone.gemfire.cache.util.TimestampedEntryEvent;
 import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetMember;
 import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
@@ -242,7 +243,8 @@ public class DistributedAckRegionCCEDUnitTest extends DistributedAckRegionDUnitT
         NetMember nm = CCRegion.getDistributionManager().getDistributionManagerId().getNetMember();
         InternalDistributedMember mbr = null;
         try {
-          mbr = new InternalDistributedMember(nm.getInetAddress().getCanonicalHostName(), nm.getPort()-1, "fake_id", "fake_id_ustring");
+          mbr = new InternalDistributedMember(nm.getInetAddress().getCanonicalHostName(), nm.getPort()-1,
+              "fake_id", "fake_id_ustring", DistributionManager.NORMAL_DM_TYPE, null, null);
           tag.setMemberID(mbr);
         } catch (UnknownHostException e) {
           fail("could not create member id", e);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
index ca3f249..3ebc72a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
@@ -72,7 +72,8 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
     InternalDistributedSystem ds = (InternalDistributedSystem)DistributedSystem.connect(p);
     try {
       // construct a member ID that will represent a departed member
-      InternalDistributedMember mbr = new InternalDistributedMember("localhost", 12345, "", "");
+      InternalDistributedMember mbr = new InternalDistributedMember("localhost", 12345, "", "",
+          DistributionManager.NORMAL_DM_TYPE, null, null);
       final DistributionManager mgr = (DistributionManager)ds.getDistributionManager();
       // schedule a message in order to create a queue for the fake member
       final FakeMessage msg = new FakeMessage(null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ConsoleDistributionManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ConsoleDistributionManagerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ConsoleDistributionManagerDUnitTest.java
index 8ff6b46..e8d5a07 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ConsoleDistributionManagerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ConsoleDistributionManagerDUnitTest.java
@@ -79,7 +79,7 @@ public class ConsoleDistributionManagerDUnitTest
       {
         boolean created = !isConnectedToDS();
         InternalDistributedSystem ds = getSystem();
-        transport = new RemoteTransportConfig(ds.getConfig());
+        transport = new RemoteTransportConfig(ds.getConfig(), DistributionManager.ADMIN_ONLY_DM_TYPE);
         if (created) {
           disconnectFromDS();
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocalDistributionManagerTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocalDistributionManagerTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocalDistributionManagerTest.java
deleted file mode 100644
index 55475b3..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocalDistributionManagerTest.java
+++ /dev/null
@@ -1,475 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
- * This product is protected by U.S. and international copyright
- * and intellectual property laws. Pivotal products are covered by
- * one or more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.distributed.internal;
-
-//import com.gemstone.gemfire.*;
-//import com.gemstone.gemfire.distributed.DistributedSystem;
-import dunit.*;
-
-import java.util.*;
-import java.io.*;
-import com.gemstone.gemfire.distributed.internal.membership.*;
-
-/**
- * This class tests the functionality of a {@link
- * LocalDistributionManager}. 
- *
- * @author David Whitlock
- *
- * @since 2.1
- */
-public class LocalDistributionManagerTest
-  extends DistributedTestCase {
-
-  public LocalDistributionManagerTest(String name) {
-    super(name);
-  }
-
-  /**
-   * Creates a connection to the distributed system in the given
-   * {@link VM}.  Configures the connection for the VM.
-   */
-  protected void createSystem(VM vm) {
-    vm.invoke(new SerializableRunnable("Connect to distributed system") {
-        public void run() {
-          // @todo davidw Remove reliance on shared memory for
-          // configuration
-          Properties props = new Properties();
-          // props.setProperty(DistributionConfig.ENABLE_SHARED_MEMORY_NAME, "true");
-//           File sysDir =
-//             GemFireConnectionFactory.getInstance().getSystemDirectory();
-//           props.setProperty(DistributionConfig.SYSTEM_DIRECTORY_NAME, sysDir.getPath());
-          getSystem(props); 
-        }
-      });
-  }
-
-  ////////  Test Methods
-
-  /**
-   * Do we see all of the distribution managers in the distributed
-   * system?
-   */
-  public void testCountDMs() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-    final int systemCount = 2;
-
-    createSystem(vm0);
-    vm0.invoke(new SerializableRunnable("Count DMs") {
-        public void run() {
-          DM dm = getSystem().getDistributionManager();
-          assertEquals(systemCount + 1,
-                       dm.getNormalDistributionManagerIds().size());
-        }
-      });
-    createSystem(vm1);
-    vm1.invoke(new SerializableRunnable("Count DMs Again") {
-        public void run() {
-          DM dm = getSystem().getDistributionManager();
-          assertEquals(systemCount + 2,
-                       dm.getNormalDistributionManagerIds().size());
-        }
-      });
-  }
-
-  /**
-   * Test that messages that are sent are received in a reasonable
-   * amount of time.
-   */
-  public void testSendMessage() throws InterruptedException {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    createSystem(vm0);
-    createSystem(vm1);
-
-    Thread.sleep(5 * 1000);
-
-    vm0.invoke(new SerializableRunnable("Send message") {
-        public void run() {
-          DM dm = getSystem().getDistributionManager();
-          assertEquals("For DM " + dm.getId(),
-                       3, dm.getOtherNormalDistributionManagerIds().size());
-          FirstMessage message = new FirstMessage();
-          dm.putOutgoing(message);
-        }
-      });
-
-    Thread.sleep(3 * 1000);
-
-    vm1.invoke(new SerializableRunnable("Was message received?") {
-        public void run() {
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              return FirstMessage.received;
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 3 * 1000, 200, true);
-          FirstMessage.received = false;
-        }
-      });
-  }
-
-  /**
-   * Tests the new non-shared {@link ReplyProcessor21}
-   */
-  public void testReplyProcessor() throws InterruptedException {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    createSystem(vm0);
-    createSystem(vm1);
-
-    Thread.sleep(2 * 1000);
-
-    vm0.invoke(new SerializableRunnable("Send request") {
-        public void run() {
-          // Send a request, wait for a response
-          DM dm = getSystem().getDistributionManager();
-          int expected = dm.getOtherNormalDistributionManagerIds().size();
-          assertEquals("For DM " + dm.getId(), 3, expected);
-
-          Response.totalResponses = 0;
-
-          Request request = new Request();
-          ReplyProcessor21 processor = new ReplyProcessor21(getSystem(),
-               dm.getOtherNormalDistributionManagerIds()); 
-          request.processorId = processor.getProcessorId();
-          dm.putOutgoing(request);
-          try {
-            processor.waitForReplies();
-
-          } catch (Exception ex) {
-            fail("While waiting for replies", ex);
-          }
-
-          assertEquals(expected, Response.totalResponses);
-        }
-      });
-    
-  }
-
-  /** A <code>TestMembershipListener</code> used in this VM */
-  protected static TestMembershipListener listener = null;
-
-  /**
-   * Does the {@link MembershipListener#memberJoined} method get
-   * invoked? 
-   */
-  public void testMemberJoinedAndDeparted()
-    throws InterruptedException {
-
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    createSystem(vm0);
-    vm0.invoke(new SerializableRunnable("Install listener") {
-        public void run() {
-          DM dm = getSystem().getDistributionManager();
-          listener = new TestMembershipListener();
-          dm.addMembershipListener(listener);
-        }
-      });
-    createSystem(vm1);
-
-    vm0.invoke(new SerializableRunnable("Verify member joining") {
-        public void run() {
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              return listener.memberJoined();
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 3 * 1000, 200, true);
-        }
-      });
-    vm1.invoke(new SerializableRunnable("Disconnect from system") {
-        public void run() {
-          getSystem().disconnect();
-        }
-      });
-
-    vm0.invoke(new SerializableRunnable("Verify member departing") {
-        public void run() {
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              return listener.memberDeparted();
-            }
-            public String description() {
-              return null;
-            }
-          };
-          DistributedTestCase.waitForCriterion(ev, 3 * 1000, 200, true);
-        }
-      });
-  }
-
-  /**
-   * Tests that the reply processor gets signaled when members go
-   * away. 
-   */
-  public void testMembersDepartWhileWaiting()
-    throws InterruptedException {
-
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    createSystem(vm0);
-    createSystem(vm1);
-
-    Thread.sleep(3 * 1000);
-    
-    AsyncInvocation ai0 =
-      vm0.invokeAsync(new SerializableRunnable("Send message and wait") {
-          public void run() {
-            DM dm = getSystem().getDistributionManager();
-            OnlyGFDMReply message = new OnlyGFDMReply();
-            ReplyProcessor21 processor = new ReplyProcessor21(getSystem(),
-              dm.getOtherNormalDistributionManagerIds());
-            message.processorId = processor.getProcessorId();
-            dm.putOutgoing(message);
-
-            try {
-              processor.waitForReplies();
-              
-            } catch (Exception ex) {
-              fail("While waiting for replies", ex);
-            }
-          }
-        });
-
-    Thread.sleep(3 * 1000);
-    vm1.invoke(new SerializableRunnable("Disconnect from system") {
-        public void run() {
-          getSystem().disconnect();
-        }
-      });
-
-    DistributedTestCase.join(ai0, 30 * 1000, getLogWriter());
-    if (ai0.exceptionOccurred()) {
-      fail("got exception", ai0.getException());
-    }
-  }
-
-  //////////////////////  Inner Classes  //////////////////////
-
-  /**
-   * A message that is send, and when received, sets a
-   * <code>boolean</code> static field.
-   *
-   * @see LocalDistributionManagerTest#testSendMessage
-   */
-  public static class FirstMessage extends SerialDistributionMessage {
-
-    /** Has a <code>FirstMessage</code> be received? */
-    public static boolean received = false;
-
-    public FirstMessage() { }   // For Externalizable
-
-    public void process(DistributionManager dm) {
-      received = true;
-    }
-    public int getDSFID() {
-      return NO_FIXED_ID;
-    }
-  }
-
-  /**
-   * A request that is replied to with a {@link Response}
-   *
-   * @see LocalDistributionManagerTest#testReplyProcessor
-   */
-  public static class Request extends SerialDistributionMessage
-    implements MessageWithReply {
-
-    /** The id of the processor to process the response */
-    int processorId;
-
-    public Request() { }        // For Externizable
-
-    public int getProcessorId() {
-      return this.processorId;
-    }
-
-    /**
-     * Reply with a {@link Response}
-     */
-    public void process(DistributionManager dm) {
-      Response response = new Response();
-      response.processorId = this.processorId;
-      response.setRecipient(this.getSender());
-      dm.putOutgoing(response);
-    }
-
-    public int getDSFID() {
-      return NO_FIXED_ID;
-    }
-
-    public void toData(DataOutput out) throws IOException {
-      super.toData(out);
-      out.writeInt(this.processorId);
-    }
-
-    public void fromData(DataInput in)
-      throws ClassNotFoundException, IOException {
-      super.fromData(in);
-      this.processorId = in.readInt();
-    }
-
-    public String toString() {
-      return "Request with processor " + this.processorId;
-    }
-  }
-
-  /**
-   * A response to a {@link Request}
-   *
-   * @see LocalDistributionManagerTest#testReplyProcessor
-   */
-  public static class Response extends SerialDistributionMessage {
-    /** The total number of responses that have been received */
-    static int totalResponses = 0;
-
-    /** The id of the processor to process the response */
-    int processorId;
-
-    public Response() { }       // For Externalizable
-
-    /**
-     * Alert the {@link ReplyProcess21} that this reply has been
-     * received.
-     */
-    public void process(DistributionManager dm) {
-      // Look up the processor
-      ReplyProcessor21 processor =
-        ReplyProcessor21.getProcessor(this.processorId);
-      assertNotNull("Null processor!", processor);
-      synchronized (Response.class) {
-        totalResponses++;
-      }
-      processor.process(this);
-    }
-
-    public int getDSFID() {
-      return NO_FIXED_ID;
-    }
-
-    public void toData(DataOutput out) throws IOException {
-      super.toData(out);
-      out.writeInt(this.processorId);
-    }
-
-    public void fromData(DataInput in)
-      throws ClassNotFoundException, IOException {
-      super.fromData(in);
-      this.processorId = in.readInt();
-    }
-
-    public String toString() {
-      return "Response with processor " + this.processorId;
-    }
-  }
-
-  /**
-   * A <code>MembershipListener</code> that remembers when members
-   * join and depart.
-   */
-  static class TestMembershipListener implements MembershipListener {
-
-    /** Has a member joined recently? */
-    private boolean joined = false;
-
-    /** Has a member departed recently? */
-    private boolean departed = false;
-
-    public void memberJoined(InternalDistributedMember id) {
-      this.joined = true;
-    }
-
-    public void memberDeparted(InternalDistributedMember id, boolean crashed) {
-      this.departed = true;
-    }
-
-    /**
-     * Gets (and then forgets) whether or not a member has recently
-     * joined the distributed system.
-     */
-    public boolean memberJoined() {
-      boolean b = this.joined;
-      this.joined = false;
-      return b;
-    }
-
-    public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {
-    }
-
-    public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
-    }
-    
-    /**
-     * Gets (and then forgets) whether or not a member has recently
-     * departed the distributed system.
-     */
-    public boolean memberDeparted() {
-      boolean b = this.departed;
-      this.departed = false;
-      return b;
-    }
-  }
-
-  /**
-   * A message that only GemFire distribution managers reply to.
-   */
-  public static class OnlyGFDMReply extends SerialDistributionMessage
-    implements MessageWithReply {
-
-    /** The id of the processor that processes the reply */
-    protected int processorId;
-
-    public int getProcessorId() {
-      return this.processorId;
-    }
-
-    public OnlyGFDMReply() { }  // For Externalizable
-
-    public void process(DistributionManager dm) {
-    }
-
-    public int getDSFID() {
-      return NO_FIXED_ID;
-    }
-
-    public void toData(DataOutput out) throws IOException {
-      super.toData(out);
-      out.writeInt(this.processorId);
-    }
-
-    public void fromData(DataInput in)
-      throws ClassNotFoundException, IOException {
-      super.fromData(in);
-      this.processorId = in.readInt();
-    }
-
-    public String toString() {
-      return "Only GFDM replies with processor " + this.processorId;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
index 4adda2a..7da7780 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
@@ -7,40 +7,58 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership;
 
+import java.io.File;
+import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
+import java.util.Set;
 
 import junit.framework.TestCase;
 
 import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DMStats;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionMessage;
+import com.gemstone.gemfire.distributed.internal.LonerDistributionManager.DummyDMStats;
+import com.gemstone.gemfire.distributed.internal.MembershipListener;
+import com.gemstone.gemfire.distributed.internal.PoolStatHelper;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.SocketCreator;
+import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class MembershipJUnitTest extends TestCase {
-  Level baseLogLevel;
+  static Level baseLogLevel;
   
   public MembershipJUnitTest(String name) {
     super(name);
   }
 
-  @Before
-  protected void setUp() throws Exception {
+  @BeforeClass
+  public static void setupClass() {
     baseLogLevel = LogService.getBaseLogLevel();
-//    LogService.setBaseLogLevel(Level.DEBUG);
-    super.setUp();
+    LogService.setBaseLogLevel(Level.DEBUG);
   }
-
-  @After
+  
+  @AfterClass
   protected void tearDown() throws Exception {
-//    LogService.setBaseLogLevel(baseLogLevel);
-    super.tearDown();
+    LogService.setBaseLogLevel(baseLogLevel);
   }
   
   /**
@@ -106,4 +124,128 @@ public class MembershipJUnitTest extends TestCase {
     assertTrue(!actual.contains(members[members.length-2]));
   }
   
+  /**
+   * This test creates a locator with a colocated
+   * membership manager and then creates a second
+   * manager that joins the system of the first.
+   * 
+   * It then makes assertions about the state of
+   * the membership view, closes one of the managers
+   * and makes more assertions.
+   */
+  @Test
+  public void testJoinLeave() throws Exception {
+    
+    MembershipManager m1=null, m2=null;
+    Locator l = null;
+    
+    try {
+      
+      // boot up a locator
+      int port = AvailablePortHelper.getRandomAvailableTCPPort();
+      InetAddress localHost = SocketCreator.getLocalHost();
+      
+      // this locator will hook itself up with the first MembershipManager
+      // to be created
+      l = Locator.startLocator(port, new File(""), localHost);
+
+      // create configuration objects
+      Properties nonDefault = new Properties();
+      nonDefault.put(DistributionConfig.DISABLE_TCP_NAME, "true");
+      nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
+      nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
+      nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
+      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostName()+'['+port+']');
+      DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
+      RemoteTransportConfig transport = new RemoteTransportConfig(config,
+          DistributionManager.NORMAL_DM_TYPE);
+
+      // start the first membership manager
+      MembershipListener listener1 = new MembershipListener();
+      DMStats stats1 = new MyStats();
+      m1 = MemberFactory.newMembershipManager(listener1, config, transport, stats1);
+
+      // start the second membership manager
+      MembershipListener listener2 = new MembershipListener();
+      DMStats stats2 = new MyStats();
+      m2 = MemberFactory.newMembershipManager(listener2, config, transport, stats2);
+      
+      assert m2.getView().size() == 2;
+      assert m1.getView().size() == 2;
+      assert m1.getView().getViewId() == m2.getView().getViewId();
+      
+      m2.shutdown();
+      assert !m2.isConnected();
+      
+      assert m1.getView().size() == 1;
+    }
+    finally {
+      System.getProperties().remove(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
+      LogService.reconfigure();
+      
+      if (m2 != null) {
+        m2.shutdown();
+      }
+      if (m1 != null) {
+        m1.shutdown();
+      }
+      if (l != null) {
+        l.stop();
+      }
+    }
+    
+  }
+  
+  static class MembershipListener implements DistributedMembershipListener {
+
+    @Override
+    public void viewInstalled(NetView view) {
+    }
+
+    @Override
+    public void quorumLost(Set<InternalDistributedMember> failures,
+        List<InternalDistributedMember> remainingMembers) {
+    }
+
+    @Override
+    public void newMemberConnected(InternalDistributedMember m, Stub stub) {
+    }
+
+    @Override
+    public void memberDeparted(InternalDistributedMember id, boolean crashed,
+        String reason) {
+    }
+
+    @Override
+    public void memberSuspect(InternalDistributedMember suspect,
+        InternalDistributedMember whoSuspected) {
+    }
+
+    @Override
+    public void messageReceived(DistributionMessage o) {
+    }
+
+    @Override
+    public boolean isShutdownMsgSent() {
+      return false;
+    }
+
+    @Override
+    public void membershipFailure(String reason, Throwable t) {
+    }
+
+    @Override
+    public DistributionManager getDM() {
+      return null;
+    }
+    
+  }
+  
+  static class MyStats extends DummyDMStats {
+  }
+
+  public static class DummyPoolStatHelper implements PoolStatHelper {
+    public void startJob() {}
+    public void endJob(){}
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
index 64e087c..029f212 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
@@ -55,12 +55,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   
   private MyBucketOperator bucketOperator;
   
-  @BeforeClass
-  public static void beforeClass() {
-    //bogus initialization to be able to create InternalDistributedMembers
-    MemberAttributes.setDefaults(1, 1, 1, -1, null, null, null);
-  }
-  
   @Before
   public void setUp() {
     this.bucketOperator = new MyBucketOperator();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab68f4e6/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
index 76f8d44..a950c25 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
@@ -188,7 +188,7 @@ public class DurableClientReconnectDUnitTest extends DistributedTestCase
   }
   
   
-  public void testDurableReconnect_DiffernetPrimary() throws Exception
+  public void testDurableReconnect_DifferentPrimary() throws Exception
   {
    //create client cache and Send clientReady message
     createCacheClient();