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

[2/2] geode git commit: GEODE-1893 remove ability to create a locator without all location services

GEODE-1893 remove ability to create a locator without all location services

Deprecated API options for creating a locator without peerLocation or
serverLocation handlers.  There will be documentation work to do since
the docs mention this in at least one place.


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

Branch: refs/heads/develop
Commit: c33ce7db8148fe0b300066581f26582e3e044e96
Parents: 2f14e20
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 20 14:40:01 2016 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 20 15:53:35 2016 -0800

----------------------------------------------------------------------
 .../distributed/ConfigurationProperties.java    | 17 ++---
 .../org/apache/geode/distributed/Locator.java   | 15 ++--
 .../geode/distributed/LocatorLauncher.java      | 15 +---
 .../internal/InternalDistributedSystem.java     | 24 ++----
 .../distributed/internal/InternalLocator.java   | 50 +++++--------
 .../geode/internal/DistributionLocator.java     |  3 +-
 .../admin/remote/DistributionLocatorId.java     | 30 +++-----
 .../internal/beans/LocatorMBeanBridge.java      |  4 +-
 .../geode/distributed/LocatorJUnitTest.java     | 28 -------
 .../InternalDistributedSystemJUnitTest.java     |  3 -
 .../locator/GMSLocatorRecoveryJUnitTest.java    |  2 +-
 .../management/LocatorManagementDUnitTest.java  | 79 +++++++++++---------
 .../codeAnalysis/sanctionedSerializables.txt    |  0
 .../internal/locator/wan/LocatorHelper.java     |  3 -
 14 files changed, 94 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java b/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
index 536e4b6..63f6505 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
@@ -1114,8 +1114,7 @@ public interface ConfigurationProperties {
    * name="ssl-locator-alias"/a>
    * </p>
    * <U>Description</U>: This property is to be used if a specific key is to be used for the SSL
-   * communications for the locator. These Communications would be either locator-client or
-   * locator-server
+   * communications for locators and for communicating with locators.
    * </p>
    * <U><i>Optional</i></U> <U>Default</U>: ""
    * </p>
@@ -1725,16 +1724,12 @@ public interface ConfigurationProperties {
   /**
    * The static String definition of the <i>"start-locator"</i> property <a name="start-locator"/a>
    * </p>
-   * <U>Description</U>: A host name or bind-address and port
-   * ("host[port],peer=<true|false>,server=<true|false>") that are used to start a locator in the
-   * same process as the DistributedSystem. The locator is started when the DistributedSystem
-   * connects, and is stopped when the DistributedSystem disconnects. To start a locator that is not
-   * tied to the DistributedSystem's lifecycle, see the {@link Locator} class in this same package.
+   * <U>Description</U>: A host name or bind-address and port ("host[port]") that are used to start
+   * a locator in the same process as the DistributedSystem. The locator is started when the
+   * DistributedSystem connects, and is stopped when the DistributedSystem disconnects. To start a
+   * locator that is not tied to the DistributedSystem's lifecycle, see the {@link Locator} class in
+   * this same package.
    * <p>
-   * <p>
-   * The peer and server parameters are optional. They specify whether the locator can be used for
-   * peers to discover each other, or for clients to discover peers. By default both are true.
-   * </p>
    * <U>Default</U>: "" (doesn't start a locator)
    */
   String START_LOCATOR = "start-locator";

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/Locator.java b/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
index 049ead4..2c7c251 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
@@ -74,8 +74,6 @@ public abstract class Locator {
    */
   protected String hostnameForClients;
 
-  protected static boolean loadSharedConfig = false;
-
   ////////////////////// Static Methods //////////////////////
 
   /**
@@ -104,7 +102,7 @@ public abstract class Locator {
    */
   public static Locator startLocator(int port, File logFile) throws IOException {
 
-    return startLocator(port, logFile, false, (InetAddress) null, (Properties) null, true, false,
+    return startLocator(port, logFile, false, (InetAddress) null, (Properties) null, true, true,
         null);
   }
 
@@ -169,7 +167,7 @@ public abstract class Locator {
   public static Locator startLocator(int port, File logFile, InetAddress bindAddress)
       throws IOException {
 
-    return startLocator(port, logFile, false, bindAddress, (Properties) null, true, false, null);
+    return startLocator(port, logFile, false, bindAddress, (Properties) null, true, true, null);
   }
 
 
@@ -211,6 +209,7 @@ public abstract class Locator {
    * system with a long-running process that can be relied on for stable membership information. The
    * locator will provide provide peer and cache server location services.
    *
+   * @deprecated use a different startup method peerLocator and serverLocator parameters are ignored
    * @since GemFire 5.7
    *
    * @param port The port on which the locator will listen for membership information requests from
@@ -241,8 +240,7 @@ public abstract class Locator {
   public static Locator startLocatorAndDS(int port, File logFile, InetAddress bindAddress,
       java.util.Properties dsProperties, boolean peerLocator, boolean serverLocator,
       String hostnameForClients) throws IOException {
-    return startLocator(port, logFile, bindAddress, dsProperties, peerLocator, serverLocator,
-        hostnameForClients);
+    return startLocator(port, logFile, bindAddress, dsProperties, true, true, hostnameForClients);
   }
 
   /**
@@ -253,7 +251,7 @@ public abstract class Locator {
       java.util.Properties dsProperties, boolean peerLocator, boolean serverLocator,
       String hostnameForClients) throws IOException {
     return InternalLocator.startLocator(port, logFile, null, null, null, bindAddress, dsProperties,
-        peerLocator, serverLocator, hostnameForClients, loadSharedConfig);
+        hostnameForClients);
   }
 
   /**
@@ -264,8 +262,7 @@ public abstract class Locator {
       InetAddress bindAddress, java.util.Properties dsProperties, boolean peerLocator,
       boolean serverLocator, String hostnameForClients) throws IOException {
     return InternalLocator.startLocator(port, logFile, null, null, null, bindAddress,
-        startDistributedSystem, dsProperties, peerLocator, serverLocator, hostnameForClients,
-        loadSharedConfig);
+        startDistributedSystem, dsProperties, hostnameForClients);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java b/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
index d704804..3a3ee82 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
@@ -120,18 +120,6 @@ public final class LocatorLauncher extends AbstractLauncher<String> {
    * @deprecated This is specific to the internal implementation and may go away in a future
    *             release.
    */
-  public static final boolean DEFAULT_ENABLE_PEER_LOCATION = true;
-
-  /**
-   * @deprecated This is specific to the internal implementation and may go away in a future
-   *             release.
-   */
-  public static final boolean DEFAULT_ENABLE_SERVER_LOCATION = true;
-
-  /**
-   * @deprecated This is specific to the internal implementation and may go away in a future
-   *             release.
-   */
   public static final String DEFAULT_LOCATOR_PID_FILE = "vf.gf.locator.pid";
 
   private static final String DEFAULT_LOCATOR_LOG_EXT = ".log";
@@ -640,8 +628,7 @@ public final class LocatorLauncher extends AbstractLauncher<String> {
         // TODO : remove the extra param for loadFromSharedConfigDir
         try {
           this.locator = InternalLocator.startLocator(getPort(), getLogFile(), null, null, null,
-              getBindAddress(), getDistributedSystemProperties(), DEFAULT_ENABLE_PEER_LOCATION,
-              DEFAULT_ENABLE_SERVER_LOCATION, getHostnameForClients(), false);
+              getBindAddress(), getDistributedSystemProperties(), getHostnameForClients());
         } finally {
           ProcessLauncherContext.remove();
         }

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
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 95aa7df..3d51fb9 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
@@ -795,15 +795,13 @@ public class InternalDistributedSystem extends DistributedSystem
       // if locator is started this way, cluster config is not enabled, set the flag correctly
       this.startedLocator.getConfig().setEnableClusterConfiguration(false);
 
-      if (locId.isPeerLocator()) {
-        boolean startedPeerLocation = false;
-        try {
-          this.startedLocator.startPeerLocation(true);
-          startedPeerLocation = true;
-        } finally {
-          if (!startedPeerLocation) {
-            this.startedLocator.stop();
-          }
+      boolean startedPeerLocation = false;
+      try {
+        this.startedLocator.startPeerLocation(true);
+        startedPeerLocation = true;
+      } finally {
+        if (!startedPeerLocation) {
+          this.startedLocator.stop();
         }
       }
     } catch (IOException e) {
@@ -820,17 +818,9 @@ public class InternalDistributedSystem extends DistributedSystem
   private void endInitLocator() throws IOException {
     InternalLocator loc = this.startedLocator;
     if (loc != null) {
-      String locatorString = this.originalConfig.getStartLocator();
-      // DistributionLocatorId locId = new DistributionLocatorId(locatorString);
       boolean finished = false;
       try {
-        // during the period when the product is using only paper licenses we always
-        // start server location services in order to be able to log information
-        // about the use of cache servers
-        // if(locId.isServerLocator()) {
         loc.startServerLocation(this);
-        // }
-
         loc.endStartLocator(this);
         finished = true;
       } finally {

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
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 18eee70..6bae8ba 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
@@ -306,22 +306,15 @@ public class InternalLocator extends Locator implements ConnectListener {
    * @param securityLogger the logger to be used for security related log messages
    * @param dsProperties optional properties to configure the distributed system (e.g., mcast
    *        addr/port, other locators)
-   * @param peerLocator enable peer location services
-   * @param enableServerLocator enable server location services
    * @param hostnameForClients the name to give to clients for connecting to this locator
-   * @param loadSharedConfigFromDir load the shared configuration from the shared configuration
-   *        directory
-   *
    * @throws IOException
    * @since GemFire 7.0
    */
   public static InternalLocator startLocator(int port, File logFile, File stateFile,
       InternalLogWriter logger, InternalLogWriter securityLogger, InetAddress bindAddress,
-      java.util.Properties dsProperties, boolean peerLocator, boolean enableServerLocator,
-      String hostnameForClients, boolean loadSharedConfigFromDir) throws IOException {
+      Properties dsProperties, String hostnameForClients) throws IOException {
     return startLocator(port, logFile, stateFile, logger, securityLogger, bindAddress, true,
-        dsProperties, peerLocator, enableServerLocator, hostnameForClients,
-        loadSharedConfigFromDir);
+        dsProperties, hostnameForClients);
   }
 
 
@@ -337,25 +330,15 @@ public class InternalLocator extends Locator implements ConnectListener {
    * @param startDistributedSystem if true, a distributed system is started
    * @param dsProperties optional properties to configure the distributed system (e.g., mcast
    *        addr/port, other locators)
-   * @param peerLocator enable peer location services
-   * @param enableServerLocator enable server location services
    * @param hostnameForClients the name to give to clients for connecting to this locator
-   * @param loadSharedConfigFromDir TODO:CONFIG
    *
    * @throws IOException
    */
   public static InternalLocator startLocator(int port, File logFile, File stateFile,
       InternalLogWriter logger, InternalLogWriter securityLogger, InetAddress bindAddress,
-      boolean startDistributedSystem, java.util.Properties dsProperties, boolean peerLocator,
-      boolean enableServerLocator, String hostnameForClients, boolean loadSharedConfigFromDir)
+      boolean startDistributedSystem, Properties dsProperties, String hostnameForClients)
       throws IOException {
 
-    if (!peerLocator && !enableServerLocator) {
-      throw new IllegalArgumentException(
-          LocalizedStrings.InternalLocator_EITHER_PEER_LOCATOR_OR_SERVER_LOCATOR_MUST_BE_ENABLED
-              .toLocalizedString());
-    }
-
     System.setProperty(FORCE_LOCATOR_DM_TYPE, "true");
     InternalLocator slocator = null;
 
@@ -368,13 +351,9 @@ public class InternalLocator extends Locator implements ConnectListener {
       // TODO:GEODE-1243: this.server is now a TcpServer and it should store or return its non-zero
       // port in a variable to use here
 
-      if (enableServerLocator) {
-        slocator.handler.willHaveServerLocator = true;
-      }
+      slocator.handler.willHaveServerLocator = true;
       try {
-        if (peerLocator) {
-          slocator.startPeerLocation(startDistributedSystem);
-        }
+        slocator.startPeerLocation(startDistributedSystem);
         if (startDistributedSystem) {
           try {
             slocator.startDistributedSystem(); // TODO:GEODE-1243: throws Exception if TcpServer
@@ -395,12 +374,6 @@ public class InternalLocator extends Locator implements ConnectListener {
       }
 
 
-      // during the period when the product is using only paper licenses we always
-      // start server location services in order to be able to log information
-      // about the use of cache servers
-      // if(enableServerLocator) {
-      // slocator.startServerLocation(InternalDistributedSystem.getConnectedInstance());
-      // }
       InternalDistributedSystem sys = InternalDistributedSystem.getConnectedInstance();
       if (sys != null) {
         try {
@@ -647,6 +620,19 @@ public class InternalLocator extends Locator implements ConnectListener {
     return this.locatorImpl;
   }
 
+  /**
+   * For backward-compatibility we retain this method
+   * 
+   * @deprecated use a form of the method that does not have peerLocator/serverLocator parameters
+   */
+  public static InternalLocator startLocator(int locatorPort, File logFile, File stateFile,
+      InternalLogWriter logger, InternalLogWriter logger1, InetAddress addr,
+      Properties dsProperties, boolean peerLocator, boolean serverLocator, String s, boolean b1)
+      throws IOException {
+    return startLocator(locatorPort, logFile, stateFile, logger, logger1, addr, dsProperties, s);
+
+  }
+
   class SharedConfigurationRunnable implements Runnable {
 
     private final InternalLocator locator = InternalLocator.this;

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java b/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
index e0de2bd..f861515 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
@@ -167,8 +167,7 @@ public class DistributionLocator {
       try {
 
         InternalLocator locator = InternalLocator.startLocator(port, new File(DEFAULT_LOG_FILE),
-            null, null, null, address, (Properties) null, peerLocator, serverLocator,
-            hostnameForClients, LOAD_SHARED_CONFIGURATION);
+            null, null, null, address, (Properties) null, hostnameForClients);
 
         ManagerInfo.setLocatorStarted(directory, port, address);
         locator.waitToStop();

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DistributionLocatorId.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DistributionLocatorId.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DistributionLocatorId.java
index f724292..5ab1bed 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DistributionLocatorId.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DistributionLocatorId.java
@@ -42,6 +42,8 @@ public class DistributionLocatorId implements java.io.Serializable {
   private final int port;
   private final String bindAddress;
   transient private SSLConfig sslConfig;
+  // the following two fields are not used but are retained for backward compatibility
+  // as this class is Serializable and is used in WAN locator information exchange
   private boolean peerLocator = true;
   private boolean serverLocator = true;
   private String hostnameForClients;
@@ -78,13 +80,11 @@ public class DistributionLocatorId implements java.io.Serializable {
   }
 
   public DistributionLocatorId(InetAddress host, int port, String bindAddress, SSLConfig sslConfig,
-      boolean peerLocator, boolean serverLocator, String hostnameForClients) {
+      String hostnameForClients) {
     this.host = host;
     this.port = port;
     this.bindAddress = validateBindAddress(bindAddress);
     this.sslConfig = validateSSLConfig(sslConfig);
-    this.peerLocator = peerLocator;
-    this.serverLocator = serverLocator;
     this.hostnameForClients = hostnameForClients;
   }
 
@@ -155,14 +155,16 @@ public class DistributionLocatorId implements java.io.Serializable {
         String[] optionFields = options[i].split("=");
         if (optionFields.length == 2) {
           if (optionFields[0].equalsIgnoreCase("peer")) {
-            this.peerLocator = Boolean.valueOf(optionFields[1]).booleanValue();
+            // this setting is deprecated
+            // this.peerLocator = Boolean.valueOf(optionFields[1]).booleanValue();
           } else if (optionFields[0].equalsIgnoreCase("server")) {
-            this.serverLocator = Boolean.valueOf(optionFields[1]).booleanValue();
+            // this setting is deprecated
+            // this.serverLocator = Boolean.valueOf(optionFields[1]).booleanValue();
           } else if (optionFields[0].equalsIgnoreCase("hostname-for-clients")) {
             this.hostnameForClients = optionFields[1];
           } else {
             throw new IllegalArgumentException(marshalled + " invalid option " + optionFields[0]
-                + ". valid options are \"peer\" or \"server\"");
+                + ". valid options are \"peer\", \"server\" and \"hostname-for-clients\"");
           }
         }
       }
@@ -172,7 +174,7 @@ public class DistributionLocatorId implements java.io.Serializable {
   public DistributionLocatorId(InetAddress address, Locator locator) {
     this(address, locator.getPort(),
         locator.getBindAddress() == null ? null : locator.getBindAddress().getHostAddress(), null,
-        locator.isPeerLocator(), locator.isServerLocator(), locator.getHostnameForClients());
+        locator.getHostnameForClients());
   }
 
   /**
@@ -233,20 +235,6 @@ public class DistributionLocatorId implements java.io.Serializable {
   /**
    * @since GemFire 5.7
    */
-  public boolean isPeerLocator() {
-    return this.peerLocator;
-  }
-
-  /**
-   * @since GemFire 5.7
-   */
-  public boolean isServerLocator() {
-    return this.serverLocator;
-  }
-
-  /**
-   * @since GemFire 5.7
-   */
   public String getHostnameForClients() {
     return this.hostnameForClients;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/main/java/org/apache/geode/management/internal/beans/LocatorMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/LocatorMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/LocatorMBeanBridge.java
index 9544a96..33b3175 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/LocatorMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/LocatorMBeanBridge.java
@@ -71,12 +71,12 @@ public class LocatorMBeanBridge {
 
 
   public boolean isPeerLocator() {
-    return loc.isPeerLocator();
+    return true;
   }
 
 
   public boolean isServerLocator() {
-    return loc.isServerLocator();
+    return true;
   }
 
   public String[] listManagers() {

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/test/java/org/apache/geode/distributed/LocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorJUnitTest.java
index 8be0e7a..65f0947 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorJUnitTest.java
@@ -126,8 +126,6 @@ public class LocatorJUnitTest {
   @Test
   public void testBasicInfo() throws Exception {
     locator = Locator.startLocator(port, tmpFile);
-    assertTrue(locator.isPeerLocator());
-    assertFalse(locator.isServerLocator());
     int boundPort = (port == 0) ? locator.getPort() : port;
     TcpClient client = new TcpClient();
     String[] info = client.getInfo(InetAddress.getLocalHost(), boundPort);
@@ -165,32 +163,6 @@ public class LocatorJUnitTest {
     }
   }
 
-  @Test
-  public void testServerOnly() throws Exception {
-    Properties props = new Properties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION, "false");
-    locator = Locator.startLocatorAndDS(port, tmpFile, null, props, false, true, null);
-    assertFalse(locator.isPeerLocator());
-    assertTrue(locator.isServerLocator());
-    Thread.sleep(1000);
-    doServerLocation(locator.getPort());
-  }
-
-  @Test
-  public void testBothPeerAndServer() throws Exception {
-    Properties props = new Properties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION, "false");
-
-    locator = Locator.startLocatorAndDS(port, new File(""), null, props);
-    assertTrue(locator.isPeerLocator());
-    assertTrue(locator.isServerLocator());
-    Thread.sleep(1000);
-    doServerLocation(locator.getPort());
-    locator.stop();
-  }
-
   /**
    * Make sure two ServerLocation objects on different hosts but with the same port are not equal
    * <p/>

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalDistributedSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalDistributedSystemJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalDistributedSystemJUnitTest.java
index 07c6f38..5a191bb 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalDistributedSystemJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalDistributedSystemJUnitTest.java
@@ -602,9 +602,6 @@ public class InternalDistributedSystemJUnitTest {
     Collection locators = Locator.getLocators();
     Assert.assertEquals(1, locators.size());
     Locator locator = (Locator) locators.iterator().next();
-    Assert.assertTrue(locator.isPeerLocator());
-    // Assert.assertFalse(locator.isServerLocator()); server location is forced on while licensing
-    // is disabled in GemFire
     // Assert.assertIndexDetailsEquals("127.0.0.1", locator.getBindAddress().getHostAddress());
     // removed this check for ipv6 testing
     Assert.assertEquals(unusedPort, locator.getPort().intValue());

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index 489d4b8..423d7e1 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -136,7 +136,7 @@ public class GMSLocatorRecoveryJUnitTest {
       // to be created
       // l = Locator.startLocator(port, new File(""), localHost);
       l = InternalLocator.startLocator(port, new File(""), null, null, null, localHost, false,
-          new Properties(), true, false, null, false);
+          new Properties(), null);
 
       // create configuration objects
       Properties nonDefault = new Properties();

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/test/java/org/apache/geode/management/LocatorManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/LocatorManagementDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/LocatorManagementDUnitTest.java
index 336a6b1..205ada5 100644
--- a/geode-core/src/test/java/org/apache/geode/management/LocatorManagementDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/LocatorManagementDUnitTest.java
@@ -14,17 +14,17 @@
  */
 package org.apache.geode.management;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Properties;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_HTTP_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.Locator;
@@ -41,6 +41,14 @@ import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.dunit.WaitCriterion;
 import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Properties;
 
 /**
  * Test cases
@@ -70,8 +78,8 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
   @Test
   public void testPeerLocation() throws Exception {
     int locPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    startLocator(locator, true, locPort);
-    locatorMBeanExist(locator, locPort, true);
+    startLocator(locator, locPort);
+    locatorMBeanExist(locator, locPort);
 
     Host host = Host.getHost(0);
     String host0 = getServerHostName(host);
@@ -92,8 +100,8 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
   @Test
   public void testPeerLocationWithPortZero() throws Exception {
     // Start the locator with port=0
-    int locPort = startLocator(locator, true, 0);
-    locatorMBeanExist(locator, locPort, true);
+    int locPort = startLocator(locator, 0);
+    locatorMBeanExist(locator, locPort);
 
     Host host = Host.getHost(0);
     String host0 = getServerHostName(host);
@@ -118,16 +126,16 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
   public void testColocatedLocator() throws Exception {
     initManagement(false);
     int locPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    startLocator(locator, false, locPort);
-    locatorMBeanExist(locator, locPort, false);
+    startLocator(locator, locPort);
+    locatorMBeanExist(locator, locPort);
 
   }
 
   @Test
   public void testColocatedLocatorWithPortZero() throws Exception {
     initManagement(false);
-    int locPort = startLocator(locator, false, 0);
-    locatorMBeanExist(locator, locPort, false);
+    int locPort = startLocator(locator, 0);
+    locatorMBeanExist(locator, locPort);
 
   }
 
@@ -135,21 +143,21 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
   public void testListManagers() throws Exception {
     initManagement(false);
     int locPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    startLocator(locator, false, locPort);
-    listManagers(locator, locPort, false);
+    startLocator(locator, locPort);
+    listManagers(locator, locPort);
   }
 
   @Test
   public void testListManagersWithPortZero() throws Exception {
     initManagement(false);
-    int locPort = startLocator(locator, false, 0);
-    listManagers(locator, locPort, false);
+    int locPort = startLocator(locator, 0);
+    listManagers(locator, locPort);
   }
 
   @Test
   public void testWillingManagers() throws Exception {
     int locPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    startLocator(locator, true, locPort);
+    startLocator(locator, locPort);
 
     Host host = Host.getHost(0);
     String host0 = getServerHostName(host);
@@ -162,12 +170,12 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
     createCache(managedNode2, props);
     createCache(managedNode3, props);
 
-    listWillingManagers(locator, locPort, false);
+    listWillingManagers(locator);
   }
 
   @Test
   public void testWillingManagersWithPortZero() throws Exception {
-    int locPort = startLocator(locator, true, 0);
+    int locPort = startLocator(locator, 0);
 
     Host host = Host.getHost(0);
     String host0 = getServerHostName(host);
@@ -180,7 +188,7 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
     createCache(managedNode2, props);
     createCache(managedNode3, props);
 
-    listWillingManagers(locator, locPort, false);
+    listWillingManagers(locator);
   }
 
   /**
@@ -188,7 +196,7 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
    * 
    * @param vm reference to VM
    */
-  protected Integer startLocator(final VM vm, final boolean isPeer, final int port) {
+  protected Integer startLocator(final VM vm, final int port) {
 
     return (Integer) vm.invoke(new SerializableCallable("Start Locator In VM") {
 
@@ -212,7 +220,7 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
         Locator locator = null;
         try {
           File logFile = new File(getTestMethodName() + "-locator" + port + ".log");
-          locator = Locator.startLocatorAndDS(port, logFile, bindAddr, props, isPeer, true, null);
+          locator = Locator.startLocatorAndDS(port, logFile, bindAddr, props);
         } catch (IOException ex) {
           Assert.fail("While starting locator on port " + port, ex);
         }
@@ -243,10 +251,10 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
 
   /**
    * Creates a persistent region
-   * 
+   *
    * @param vm reference to VM
    */
-  protected void locatorMBeanExist(VM vm, final int locPort, final boolean isPeer) {
+  protected void locatorMBeanExist(VM vm, final int locPort) {
 
     vm.invoke(new SerializableCallable("Locator MBean created") {
 
@@ -260,7 +268,6 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
         assertEquals(locPort, bean.getPort());
         LogWriterUtils.getLogWriter().info("Log of Locator" + bean.viewLog());
         LogWriterUtils.getLogWriter().info("BindAddress" + bean.getBindAddress());
-        assertEquals(isPeer, bean.isPeerLocator());
         return null;
       }
     });
@@ -292,10 +299,10 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
 
   /**
    * Creates a persistent region
-   * 
+   *
    * @param vm reference to VM
    */
-  protected void listManagers(VM vm, final int locPort, final boolean isPeer) {
+  protected void listManagers(VM vm, final int locPort) {
 
     vm.invoke(new SerializableCallable("List Managers") {
 
@@ -328,10 +335,10 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
 
   /**
    * Creates a persistent region
-   * 
+   *
    * @param vm reference to VM
    */
-  protected void listWillingManagers(VM vm, final int locPort, final boolean isPeer) {
+  protected void listWillingManagers(VM vm) {
 
     vm.invoke(new SerializableCallable("List Willing Managers") {
 

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/geode/blob/c33ce7db/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/LocatorHelper.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/LocatorHelper.java b/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/LocatorHelper.java
index 9038bb8..5b91a4c 100644
--- a/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/LocatorHelper.java
+++ b/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/LocatorHelper.java
@@ -79,9 +79,6 @@ public class LocatorHelper {
    */
   private static void addServerLocator(Integer distributedSystemId,
       LocatorMembershipListener locatorListener, DistributionLocatorId locator) {
-    if (!locator.isServerLocator()) {
-      return;
-    }
     ConcurrentHashMap<Integer, Set<String>> allServerLocatorsInfo =
         (ConcurrentHashMap<Integer, Set<String>>) locatorListener.getAllServerLocatorsInfo();