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

[37/67] [abbrv] incubator-geode git commit: GEODE-1377: Initial move of system properties from private to public

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerBlackList.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerBlackList.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerBlackList.java
index c6e8b75..0edd260 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerBlackList.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerBlackList.java
@@ -16,23 +16,19 @@
  */
 package com.gemstone.gemfire.cache.client.internal;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl.PoolTask;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.logging.LogService;
+import org.apache.logging.log4j.Logger;
+
+import java.util.*;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.cache.client.internal.PoolImpl.PoolTask;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.logging.LogService;
-
 /**
  * This class is designed to prevent the client from spinning
  * and reconnected to the same failed server over and over.
@@ -60,7 +56,7 @@ public class ServerBlackList {
   protected final ListenerBroadcaster broadcaster = new ListenerBroadcaster();
   
   //not final for tests.
-  static int THRESHOLD = Integer.getInteger("gemfire.ServerBlackList.THRESHOLD", 3).intValue();
+  static int THRESHOLD = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "ServerBlackList.THRESHOLD", 3).intValue();
   protected final long pingInterval;
   
   public ServerBlackList(long pingInterval) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/locator/wan/LocatorMembershipListener.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/locator/wan/LocatorMembershipListener.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/locator/wan/LocatorMembershipListener.java
index 3bb0a1c..3ee08a1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/locator/wan/LocatorMembershipListener.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/locator/wan/LocatorMembershipListener.java
@@ -16,12 +16,12 @@
  */
 package com.gemstone.gemfire.cache.client.internal.locator.wan;
 
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.admin.remote.DistributionLocatorId;
 
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
 /**
  * A listener to handle membership when new locator is added to remote locator
  * metadata. This listener is expected to inform all other locators in remote

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerImpl.java
index f1b3eca..a3f5e3b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerImpl.java
@@ -20,20 +20,13 @@ import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.GatewayConfigurationException;
-import com.gemstone.gemfire.cache.client.AllConnectionsInUseException;
-import com.gemstone.gemfire.cache.client.NoAvailableServersException;
-import com.gemstone.gemfire.cache.client.ServerConnectivityException;
-import com.gemstone.gemfire.cache.client.ServerOperationException;
-import com.gemstone.gemfire.cache.client.ServerRefusedConnectionException;
-import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.cache.client.internal.ConnectionFactory;
-import com.gemstone.gemfire.cache.client.internal.Endpoint;
-import com.gemstone.gemfire.cache.client.internal.EndpointManager;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.*;
+import com.gemstone.gemfire.cache.client.internal.*;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl.PoolTask;
-import com.gemstone.gemfire.cache.client.internal.QueueConnectionImpl;
 import com.gemstone.gemfire.distributed.PoolCancelledException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.cache.PoolManagerImpl;
 import com.gemstone.gemfire.internal.cache.PoolStats;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -41,27 +34,14 @@ import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.security.GemFireSecurityException;
-import com.gemstone.gemfire.i18n.StringId;
+import org.apache.logging.log4j.Logger;
 
 import java.net.SocketException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
+import java.util.*;
+import java.util.concurrent.*;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.logging.log4j.Logger;
-
 /**
  * Manages client to server connections for the connection pool. This class contains
  * all of the pooling logic to checkout/checkin connections.
@@ -71,8 +51,8 @@ import org.apache.logging.log4j.Logger;
  */
 public class ConnectionManagerImpl implements ConnectionManager {
   private static final Logger logger = LogService.getLogger();
-  
-  static long AQUIRE_TIMEOUT = Long.getLong("gemfire.ConnectionManager.AQUIRE_TIMEOUT", 10 * 1000).longValue();
+
+  static long AQUIRE_TIMEOUT = Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "ConnectionManager.AQUIRE_TIMEOUT", 10 * 1000).longValue();
   private final String poolName;
   private final PoolStats poolStats;
   protected final long prefillRetry; // ms // make this an int

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
index 3a3e261..91108c1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
@@ -16,19 +16,6 @@
  */
 package com.gemstone.gemfire.cache.execute.internal;
 
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionService;
 import com.gemstone.gemfire.cache.client.ClientCache;
@@ -44,18 +31,16 @@ import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.InternalEntity;
-import com.gemstone.gemfire.internal.cache.AbstractRegion;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionExecutor;
-import com.gemstone.gemfire.internal.cache.execute.MemberFunctionExecutor;
-import com.gemstone.gemfire.internal.cache.execute.PartitionedRegionFunctionExecutor;
-import com.gemstone.gemfire.internal.cache.execute.ServerFunctionExecutor;
-import com.gemstone.gemfire.internal.cache.execute.ServerRegionFunctionExecutor;
+import com.gemstone.gemfire.internal.cache.execute.*;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
 /**
  * Provides the entry point into execution of user defined {@linkplain Function}s.
  * <p>
@@ -74,7 +59,7 @@ public final class FunctionServiceManager {
   /**
    * use when the optimization to execute onMember locally is not desired.
    */
-  public static final boolean RANDOM_onMember = Boolean.getBoolean("gemfire.randomizeOnMember");
+  public static final boolean RANDOM_onMember = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "randomizeOnMember");
 
   public FunctionServiceManager() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/AbstractGroupOrRangeJunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/AbstractGroupOrRangeJunction.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/AbstractGroupOrRangeJunction.java
index 02bd924..a14dda8 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/AbstractGroupOrRangeJunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/AbstractGroupOrRangeJunction.java
@@ -20,25 +20,16 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-
-import com.gemstone.gemfire.cache.query.FunctionDomainException;
-import com.gemstone.gemfire.cache.query.NameResolutionException;
-import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
-import com.gemstone.gemfire.cache.query.QueryService;
-import com.gemstone.gemfire.cache.query.SelectResults;
-import com.gemstone.gemfire.cache.query.Struct;
-import com.gemstone.gemfire.cache.query.TypeMismatchException;
+import com.gemstone.gemfire.cache.query.*;
 import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
 import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl;
 import com.gemstone.gemfire.cache.query.types.ObjectType;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
+import java.util.*;
+
 /**
  * 
  */
@@ -47,7 +38,7 @@ public abstract class AbstractGroupOrRangeJunction extends
   /** left operand */
   final CompiledValue[] _operands;
   private static  final int INDEX_RESULT_THRESHOLD_DEFAULT = 100;
-  public static final String INDX_THRESHOLD_PROP_STR = "gemfire.Query.INDEX_THRESHOLD_SIZE";
+  public static final String INDX_THRESHOLD_PROP_STR = DistributionConfig.GEMFIRE_PREFIX + "Query.INDEX_THRESHOLD_SIZE";
   private static final int indexThresholdSize = Integer.getInteger(INDX_THRESHOLD_PROP_STR, INDEX_RESULT_THRESHOLD_DEFAULT).intValue();
   private int _operator = 0;
   private CompiledValue iterOperands;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledValue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledValue.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledValue.java
index 7b5cf4b..b31a4f6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledValue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledValue.java
@@ -16,10 +16,13 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import java.util.*;
 import com.gemstone.gemfire.cache.query.*;
 import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
-import com.gemstone.gemfire.cache.query.types.*;
+import com.gemstone.gemfire.cache.query.types.ObjectType;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+
+import java.util.List;
+import java.util.Set;
 
 /**
  * Class Description
@@ -47,7 +50,7 @@ public interface CompiledValue {
   public final static int FIELD = -16;
   public final static int GROUP_BY_SELECT = -17;
   public static  final int INDEX_RESULT_THRESHOLD_DEFAULT = 100;
-  public static final String INDX_THRESHOLD_PROP_STR = "gemfire.Query.INDEX_THRESHOLD_SIZE";
+  public static final String INDX_THRESHOLD_PROP_STR = DistributionConfig.GEMFIRE_PREFIX + "Query.INDEX_THRESHOLD_SIZE";
   public static final String INDEX_INFO = "index_info";
   public static final int indexThresholdSize = Integer.getInteger(INDX_THRESHOLD_PROP_STR, INDEX_RESULT_THRESHOLD_DEFAULT).intValue();
   public static final String RESULT_TYPE = "result_type";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
index 9e56e53..665c5a4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
@@ -17,17 +17,6 @@
 
 package com.gemstone.gemfire.cache.query.internal;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.CacheRuntimeException;
@@ -38,29 +27,16 @@ import com.gemstone.gemfire.cache.client.internal.UserAttributes;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
-import com.gemstone.gemfire.cache.query.FunctionDomainException;
-import com.gemstone.gemfire.cache.query.NameResolutionException;
-import com.gemstone.gemfire.cache.query.Query;
-import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.cache.query.QueryInvalidException;
-import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
-import com.gemstone.gemfire.cache.query.QueryService;
-import com.gemstone.gemfire.cache.query.QueryStatistics;
-import com.gemstone.gemfire.cache.query.RegionNotFoundException;
-import com.gemstone.gemfire.cache.query.SelectResults;
-import com.gemstone.gemfire.cache.query.TypeMismatchException;
+import com.gemstone.gemfire.cache.query.*;
 import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.NanoTimer;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.CachePerfStats;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalDataSet;
-import com.gemstone.gemfire.internal.cache.PRQueryProcessor;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.TXManagerImpl;
-import com.gemstone.gemfire.internal.cache.TXStateProxy;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLong;
+
 
 /**
  * Thread-safe implementation of com.gemstone.persistence.query.Query
@@ -86,7 +62,7 @@ public class DefaultQuery implements Query {
   private static final Object[] EMPTY_ARRAY = new Object[0];
 
   public static boolean QUERY_VERBOSE =
-    Boolean.getBoolean("gemfire.Query.VERBOSE");
+      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "Query.VERBOSE");
 
   /**
    * System property to cleanup the compiled query. The compiled query
@@ -94,7 +70,7 @@ public class DefaultQuery implements Query {
    * By default its set to 10 minutes, the time is set in MilliSecs.
    */
   public static final int COMPILED_QUERY_CLEAR_TIME = Integer.getInteger(
-      "gemfire.Query.COMPILED_QUERY_CLEAR_TIME", 10 * 60 * 1000).intValue();
+      DistributionConfig.GEMFIRE_PREFIX + "Query.COMPILED_QUERY_CLEAR_TIME", 10 * 60 * 1000).intValue();
 
   public static int TEST_COMPILED_QUERY_CLEAR_TIME = -1;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQueryService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQueryService.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQueryService.java
index 392cb9d..cbfc051 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQueryService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQueryService.java
@@ -16,19 +16,6 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.LowMemoryException;
 import com.gemstone.gemfire.cache.Region;
@@ -37,35 +24,13 @@ import com.gemstone.gemfire.cache.client.internal.InternalPool;
 import com.gemstone.gemfire.cache.client.internal.ProxyCache;
 import com.gemstone.gemfire.cache.client.internal.ServerProxy;
 import com.gemstone.gemfire.cache.client.internal.UserAttributes;
-import com.gemstone.gemfire.cache.query.AmbiguousNameException;
-import com.gemstone.gemfire.cache.query.CqAttributes;
-import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.CqExistsException;
-import com.gemstone.gemfire.cache.query.CqQuery;
-import com.gemstone.gemfire.cache.query.CqServiceStatistics;
-import com.gemstone.gemfire.cache.query.Index;
-import com.gemstone.gemfire.cache.query.IndexCreationException;
-import com.gemstone.gemfire.cache.query.IndexExistsException;
-import com.gemstone.gemfire.cache.query.IndexNameConflictException;
-import com.gemstone.gemfire.cache.query.IndexType;
-import com.gemstone.gemfire.cache.query.MultiIndexCreationException;
-import com.gemstone.gemfire.cache.query.NameResolutionException;
-import com.gemstone.gemfire.cache.query.Query;
-import com.gemstone.gemfire.cache.query.QueryExecutionLowMemoryException;
-import com.gemstone.gemfire.cache.query.QueryInvalidException;
-import com.gemstone.gemfire.cache.query.QueryService;
-import com.gemstone.gemfire.cache.query.RegionNotFoundException;
-import com.gemstone.gemfire.cache.query.TypeMismatchException;
+import com.gemstone.gemfire.cache.query.*;
 import com.gemstone.gemfire.cache.query.internal.cq.ClientCQ;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
-import com.gemstone.gemfire.cache.query.internal.index.AbstractIndex;
-import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
-import com.gemstone.gemfire.cache.query.internal.index.IndexData;
-import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
-import com.gemstone.gemfire.cache.query.internal.index.IndexUtils;
-import com.gemstone.gemfire.cache.query.internal.index.PartitionedIndex;
+import com.gemstone.gemfire.cache.query.internal.index.*;
 import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.InternalCache;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -74,6 +39,10 @@ import com.gemstone.gemfire.internal.cache.control.MemoryThresholds;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import org.apache.logging.log4j.Logger;
+
+import java.util.*;
+import java.util.Map.Entry;
 
 /**
  * @version $Revision: 1.2 $
@@ -85,11 +54,11 @@ public class DefaultQueryService implements QueryService {
    * System property to allow query on region with heterogeneous objects. 
    * By default its set to false.
    */
-  public static final boolean QUERY_HETEROGENEOUS_OBJECTS = 
-    Boolean.valueOf(System.getProperty("gemfire.QueryService.QueryHeterogeneousObjects", "true")).booleanValue(); 
+  public static final boolean QUERY_HETEROGENEOUS_OBJECTS =
+      Boolean.valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "QueryService.QueryHeterogeneousObjects", "true")).booleanValue();
 
-  public static boolean COPY_ON_READ_AT_ENTRY_LEVEL = 
-      Boolean.valueOf(System.getProperty("gemfire.QueryService.CopyOnReadAtEntryLevel", "false")).booleanValue(); 
+  public static boolean COPY_ON_READ_AT_ENTRY_LEVEL =
+      Boolean.valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "QueryService.CopyOnReadAtEntryLevel", "false")).booleanValue();
 
   
   /** Test purpose only */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/CqServiceProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/CqServiceProvider.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/CqServiceProvider.java
index 81d8532..d31ec8a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/CqServiceProvider.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/CqServiceProvider.java
@@ -16,21 +16,22 @@
  */
 package com.gemstone.gemfire.cache.query.internal.cq;
 
+import com.gemstone.gemfire.cache.query.internal.cq.spi.CqServiceFactory;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.ServiceLoader;
 
-import com.gemstone.gemfire.cache.query.internal.cq.spi.CqServiceFactory;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-
 public class CqServiceProvider {
   
   private static final CqServiceFactory factory;
   // System property to maintain the CQ event references for optimizing the updates.
   // This will allows to run the CQ query only once during update events.   
-  public static boolean MAINTAIN_KEYS = 
-    Boolean.valueOf(System.getProperty("gemfire.cq.MAINTAIN_KEYS", "true")).booleanValue();
+  public static boolean MAINTAIN_KEYS =
+      Boolean.valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "cq.MAINTAIN_KEYS", "true")).booleanValue();
   /**
    * A debug flag used for testing vMotion during CQ registration
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/IndexManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/IndexManager.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/IndexManager.java
index 0e29178..82b8860 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/IndexManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/IndexManager.java
@@ -21,62 +21,23 @@
  */
 package com.gemstone.gemfire.cache.query.internal.index;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.query.AmbiguousNameException;
-import com.gemstone.gemfire.cache.query.Index;
-import com.gemstone.gemfire.cache.query.IndexExistsException;
-import com.gemstone.gemfire.cache.query.IndexInvalidException;
-import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
-import com.gemstone.gemfire.cache.query.IndexNameConflictException;
-import com.gemstone.gemfire.cache.query.IndexStatistics;
-import com.gemstone.gemfire.cache.query.IndexType;
-import com.gemstone.gemfire.cache.query.MultiIndexCreationException;
-import com.gemstone.gemfire.cache.query.NameResolutionException;
-import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.cache.query.TypeMismatchException;
-import com.gemstone.gemfire.cache.query.internal.CompiledPath;
-import com.gemstone.gemfire.cache.query.internal.CompiledValue;
-import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
-import com.gemstone.gemfire.cache.query.internal.ExecutionContext;
-import com.gemstone.gemfire.cache.query.internal.MapIndexable;
-import com.gemstone.gemfire.cache.query.internal.NullToken;
-import com.gemstone.gemfire.cache.query.internal.QueryMonitor;
-import com.gemstone.gemfire.cache.query.internal.QueryObserver;
-import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
+import com.gemstone.gemfire.cache.query.*;
+import com.gemstone.gemfire.cache.query.internal.*;
 import com.gemstone.gemfire.cache.query.internal.index.AbstractIndex.InternalIndexStatistics;
 import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.CachePerfStats;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.RegionEntry;
-import com.gemstone.gemfire.internal.cache.TXManagerImpl;
-import com.gemstone.gemfire.internal.cache.TXStateProxy;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
+import org.apache.logging.log4j.Logger;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  */
@@ -109,9 +70,9 @@ public class IndexManager  {
   private IndexUpdaterThread updater;
 
   // Threshold for Queue.
-  private final int INDEX_MAINTENANCE_BUFFER = Integer.getInteger("gemfire.AsynchIndexMaintenanceThreshold", -1).intValue();
+  private final int INDEX_MAINTENANCE_BUFFER = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "AsynchIndexMaintenanceThreshold", -1).intValue();
 
-  public static boolean JOIN_OPTIMIZATION = !Boolean.getBoolean("gemfire.index.DisableJoinOptimization");
+  public static boolean JOIN_OPTIMIZATION = !Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "index.DisableJoinOptimization");
   
   // Added for test purposes only.
   public static boolean INPLACE_OBJECT_MODIFICATION_FOR_TEST = false;    
@@ -129,14 +90,14 @@ public class IndexManager  {
    * In case of in-place modification the EntryEvent will not have the old-value, without this
    * the old-values are not removed from the index-maps thus resulting in inconsistent results.
    */
-  public static final boolean INPLACE_OBJECT_MODIFICATION =     
-    Boolean.valueOf(System.getProperty("gemfire.index.INPLACE_OBJECT_MODIFICATION", "false")).booleanValue(); 
+  public static final boolean INPLACE_OBJECT_MODIFICATION =
+      Boolean.valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "index.INPLACE_OBJECT_MODIFICATION", "false")).booleanValue();
 
   /**
    * System property to turn-off the compact-index support.
    */
-  public static final boolean RANGEINDEX_ONLY =     
-    Boolean.valueOf(System.getProperty("gemfire.index.RANGEINDEX_ONLY", "false")).booleanValue();
+  public static final boolean RANGEINDEX_ONLY =
+      Boolean.valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "index.RANGEINDEX_ONLY", "false")).booleanValue();
 
   /** For test purpose only */
   public static boolean TEST_RANGEINDEX_ONLY = false;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewayReceiver.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewayReceiver.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewayReceiver.java
index ba757ad..a80fbd4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewayReceiver.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewayReceiver.java
@@ -16,11 +16,12 @@
  */
 package com.gemstone.gemfire.cache.wan;
 
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+
 import java.io.IOException;
 import java.util.List;
 
-import com.gemstone.gemfire.cache.server.CacheServer;
-
 /**
  * A GatewayReceiver that receives the events from a <code>GatewaySender</code>.
  * GatewayReceiver is used in conjunction with a {@link GatewaySender} to
@@ -76,7 +77,7 @@ public interface GatewayReceiver {
    * re-applied or not is decided by this attribute.  
    */
   public static final boolean APPLY_RETRIES = Boolean
-      .getBoolean("gemfire.GatewayReceiver.ApplyRetries");
+      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "GatewayReceiver.ApplyRetries");
 
   /**
    * Starts this receiver.  Once the receiver is running, its

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
index bc6c184..d559a1a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.wan;
 
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+
 import java.util.List;
 
 /**
@@ -47,7 +49,7 @@ public interface GatewaySender {
    * block.
    */
   public static final int DEFAULT_SOCKET_READ_TIMEOUT = Integer.getInteger(
-      "gemfire.cache.gatewaySender.default-socket-read-timeout", 0).intValue();
+      DistributionConfig.GEMFIRE_PREFIX + "cache.gatewaySender.default-socket-read-timeout", 0).intValue();
 
   /**
    * The default minimum socket read timeout.
@@ -58,7 +60,7 @@ public interface GatewaySender {
    * Size of the oplog file used for the persistent queue in bytes
    */
   static public final int QUEUE_OPLOG_SIZE
-  = Integer.getInteger("gemfire.cache.gatewaySender.queueOpLogSize", 1024*1024*100).intValue();
+      = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "cache.gatewaySender.queueOpLogSize", 1024 * 1024 * 100).intValue();
 
   
   /**
@@ -89,7 +91,7 @@ public interface GatewaySender {
 
   public static final int DEFAULT_PARALLELISM_REPLICATED_REGION = Integer
       .getInteger(
-          "gemfire.cache.gatewaySender.defaultParallelismForReplicatedRegion",
+          DistributionConfig.GEMFIRE_PREFIX + "cache.gatewaySender.defaultParallelismForReplicatedRegion",
           113).intValue();  
   
   public static final int DEFAULT_DISTRIBUTED_SYSTEM_ID = -1;
@@ -110,26 +112,27 @@ public interface GatewaySender {
    * dead and should be aborted
    */
   public static final long GATEWAY_SENDER_TIMEOUT
-  = Integer.getInteger("gemfire.GATEWAY_SENDER_TIMEOUT", 30).intValue();
+      = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "GATEWAY_SENDER_TIMEOUT", 30).intValue();
 
   
   /**
    * The obsolete socket read timeout java system property. Since customers have
    * been given this property, it is used to log a warning.
    */
-  public static final String GATEWAY_CONNECTION_READ_TIMEOUT_PROPERTY = "gemfire.GatewaySender.GATEWAY_CONNECTION_READ_TIMEOUT";
-  
-  public static final int GATEWAY_CONNECTION_IDLE_TIMEOUT = Integer.getInteger("gemfire.GatewaySender.GATEWAY_CONNECTION_IDLE_TIMEOUT", -1).intValue();
+  public static final String GATEWAY_CONNECTION_READ_TIMEOUT_PROPERTY = DistributionConfig.GEMFIRE_PREFIX + "GatewaySender.GATEWAY_CONNECTION_READ_TIMEOUT";
+
+  public static final int GATEWAY_CONNECTION_IDLE_TIMEOUT = Integer
+      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "GatewaySender.GATEWAY_CONNECTION_IDLE_TIMEOUT", -1).intValue();
   
   /**
    * If the System property is set, use it. Otherwise, set default to 'true'.
    */
-  public static final boolean REMOVE_FROM_QUEUE_ON_EXCEPTION = 
-    (System.getProperty("gemfire.GatewaySender.REMOVE_FROM_QUEUE_ON_EXCEPTION") != null) 
-      ? Boolean.getBoolean("gemfire.GatewaySender.REMOVE_FROM_QUEUE_ON_EXCEPTION")
+  public static final boolean REMOVE_FROM_QUEUE_ON_EXCEPTION =
+      (System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "GatewaySender.REMOVE_FROM_QUEUE_ON_EXCEPTION") != null)
+          ? Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "GatewaySender.REMOVE_FROM_QUEUE_ON_EXCEPTION")
       : true;
-  
-  public static final boolean EARLY_ACK = Boolean.getBoolean("gemfire.GatewaySender.EARLY_ACK");
+
+  public static final boolean EARLY_ACK = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "GatewaySender.EARLY_ACK");
 
   public static final boolean DEFAULT_IS_PARALLEL = false;
   
@@ -140,7 +143,7 @@ public interface GatewaySender {
    * in case receiver is not up and running. Default is set to 1000 milliseconds i.e. 1 second.
    */
   public static final int CONNECTION_RETRY_INTERVAL = Integer.getInteger(
-      "gemfire.gateway-connection-retry-interval", 1000).intValue();
+      DistributionConfig.GEMFIRE_PREFIX + "gateway-connection-retry-interval", 1000).intValue();
   
   /**
    * The order policy. This enum is applicable only when concurrency-level is > 1.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
index c29185a..9be157c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
@@ -17,29 +17,6 @@
 
 package com.gemstone.gemfire.distributed;
 
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.net.BindException;
-import java.net.InetAddress;
-import java.net.URL;
-import java.sql.Timestamp;
-import java.text.SimpleDateFormat;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.logging.FileHandler;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.unsafe.RegisterSignalHandlerSupport;
@@ -56,6 +33,19 @@ import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.internal.util.SunAPINotFoundException;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonObject;
 
+import java.io.*;
+import java.net.BindException;
+import java.net.InetAddress;
+import java.net.URL;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.FileHandler;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
 /**
  * The AbstractLauncher class is a base class for implementing various launchers to construct and run different GemFire
  * processes, like Cache Servers, Locators, Managers, HTTP servers and so on.
@@ -81,7 +71,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
   );
 
   public static final String DEFAULT_WORKING_DIRECTORY = SystemUtils.CURRENT_DIRECTORY;
-  public static final String SIGNAL_HANDLER_REGISTRATION_SYSTEM_PROPERTY = "gemfire.launcher.registerSignalHandlers";
+  public static final String SIGNAL_HANDLER_REGISTRATION_SYSTEM_PROPERTY = DistributionConfig.GEMFIRE_PREFIX + "launcher.registerSignalHandlers";
 
   protected static final String OPTION_PREFIX = "-";
 
@@ -295,7 +285,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
     }
 
     if (!StringUtils.isBlank(getMemberName())) {
-      distributedSystemProperties.setProperty(DistributionConfig.NAME_NAME, getMemberName());
+      distributedSystemProperties.setProperty(SystemConfigurationProperties.NAME, getMemberName());
     }
 
     // Set any other GemFire Distributed System/Distribution Config directory-based properties as necessary

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
index eaf1ee7..96ea8ea 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
@@ -17,18 +17,6 @@
 
 package com.gemstone.gemfire.distributed;
 
-import java.io.File;
-import java.net.InetAddress;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.StatisticsFactory;
@@ -48,6 +36,13 @@ import com.gemstone.gemfire.memcached.GemFireMemcachedServer;
 import com.gemstone.gemfire.redis.GemFireRedisServer;
 import com.gemstone.gemfire.security.GemFireSecurityException;
 
+import java.io.File;
+import java.net.InetAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
 /**
  * A "connection" to a GemFire distributed system.  A
  * <code>DistributedSystem</code> is created by invoking the {@link
@@ -1974,7 +1969,7 @@ public abstract class DistributedSystem implements StatisticsFactory {
    * @see #getPropertiesFile()
    * @since Geode 1.0
    */
-  public static final String PROPERTIES_FILE_DEFAULT = "gemfire.properties";
+  public static final String PROPERTIES_FILE_DEFAULT = DistributionConfig.GEMFIRE_PREFIX + "properties";
 
   /**
    * Returns the current value of {@link #PROPERTIES_FILE_PROPERTY} system 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
index a0cfc4f..52e9c34 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
@@ -17,29 +17,6 @@
 
 package com.gemstone.gemfire.distributed;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.logging.Level;
-
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusResponse;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
@@ -50,20 +27,7 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.ObjectUtils;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.lang.SystemUtils;
-import com.gemstone.gemfire.internal.process.ConnectionFailedException;
-import com.gemstone.gemfire.internal.process.ControlNotificationHandler;
-import com.gemstone.gemfire.internal.process.ControllableProcess;
-import com.gemstone.gemfire.internal.process.FileAlreadyExistsException;
-import com.gemstone.gemfire.internal.process.MBeanInvocationFailedException;
-import com.gemstone.gemfire.internal.process.PidUnavailableException;
-import com.gemstone.gemfire.internal.process.ProcessController;
-import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
-import com.gemstone.gemfire.internal.process.ProcessControllerParameters;
-import com.gemstone.gemfire.internal.process.ProcessLauncherContext;
-import com.gemstone.gemfire.internal.process.ProcessType;
-import com.gemstone.gemfire.internal.process.ProcessUtils;
-import com.gemstone.gemfire.internal.process.StartupStatusListener;
-import com.gemstone.gemfire.internal.process.UnableToControlProcessException;
+import com.gemstone.gemfire.internal.process.*;
 import com.gemstone.gemfire.internal.util.IOUtils;
 import com.gemstone.gemfire.lang.AttachAPINotFoundException;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonArray;
@@ -73,6 +37,22 @@ import joptsimple.OptionException;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
 
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.logging.Level;
+
 /**
  * The LocatorLauncher class is a launcher for a GemFire Locator.
  * 
@@ -1729,9 +1709,9 @@ public final class LocatorLauncher extends AbstractLauncher<String> {
     protected void validateOnStart() {
       if (Command.START.equals(getCommand())) {
         if (StringUtils.isBlank(getMemberName())
-          && !isSet(System.getProperties(), DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME)
-          && !isSet(getDistributedSystemProperties(), DistributionConfig.NAME_NAME)
-          && !isSet(loadGemFireProperties(DistributedSystem.getPropertyFileURL()), DistributionConfig.NAME_NAME))
+            && !isSet(System.getProperties(), DistributionConfig.GEMFIRE_PREFIX + SystemConfigurationProperties.NAME)
+            && !isSet(getDistributedSystemProperties(), SystemConfigurationProperties.NAME)
+            && !isSet(loadGemFireProperties(DistributedSystem.getPropertyFileURL()), SystemConfigurationProperties.NAME))
         {
           throw new IllegalStateException(LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE
             .toLocalizedString("Locator"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
index c95e301..252f701 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
@@ -17,31 +17,8 @@
 
 package com.gemstone.gemfire.distributed;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.ServiceLoader;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.internal.DefaultServerLauncherCacheProvider;
@@ -49,30 +26,13 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.GemFireVersion;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.cache.AbstractCacheServer;
-import com.gemstone.gemfire.internal.cache.CacheConfig;
-import com.gemstone.gemfire.internal.cache.CacheServerLauncher;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.ObjectUtils;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.lang.SystemUtils;
-import com.gemstone.gemfire.internal.process.ClusterConfigurationNotAvailableException;
-import com.gemstone.gemfire.internal.process.ConnectionFailedException;
-import com.gemstone.gemfire.internal.process.ControlNotificationHandler;
-import com.gemstone.gemfire.internal.process.ControllableProcess;
-import com.gemstone.gemfire.internal.process.FileAlreadyExistsException;
-import com.gemstone.gemfire.internal.process.MBeanInvocationFailedException;
-import com.gemstone.gemfire.internal.process.PidUnavailableException;
-import com.gemstone.gemfire.internal.process.ProcessController;
-import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
-import com.gemstone.gemfire.internal.process.ProcessControllerParameters;
-import com.gemstone.gemfire.internal.process.ProcessLauncherContext;
-import com.gemstone.gemfire.internal.process.ProcessType;
-import com.gemstone.gemfire.internal.process.StartupStatusListener;
-import com.gemstone.gemfire.internal.process.UnableToControlProcessException;
+import com.gemstone.gemfire.internal.process.*;
 import com.gemstone.gemfire.internal.util.IOUtils;
 import com.gemstone.gemfire.lang.AttachAPINotFoundException;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
@@ -80,11 +40,26 @@ import com.gemstone.gemfire.management.internal.cli.json.GfJsonArray;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonObject;
 import com.gemstone.gemfire.pdx.PdxSerializer;
-
 import joptsimple.OptionException;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
 
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.SERVER_BIND_ADDRESS;
+
 /**
  * The ServerLauncher class is a launcher class with main method to start a GemFire Server (implying a GemFire Cache
  * Server process).
@@ -119,7 +94,7 @@ public class ServerLauncher extends AbstractLauncher<String> {
     helpMap.put("pid", LocalizedStrings.ServerLauncher_SERVER_PID_HELP.toLocalizedString());
     helpMap.put("rebalance", LocalizedStrings.ServerLauncher_SERVER_REBALANCE_HELP.toLocalizedString());
     helpMap.put("redirect-output", LocalizedStrings.ServerLauncher_SERVER_REDIRECT_OUTPUT_HELP.toLocalizedString());
-    helpMap.put("server-bind-address", LocalizedStrings.ServerLauncher_SERVER_BIND_ADDRESS_HELP.toLocalizedString());
+    helpMap.put(SERVER_BIND_ADDRESS, LocalizedStrings.ServerLauncher_SERVER_BIND_ADDRESS_HELP.toLocalizedString());
     helpMap.put("hostname-for-clients", LocalizedStrings.ServerLauncher_SERVER_HOSTNAME_FOR_CLIENT_HELP.toLocalizedString());
     helpMap.put("server-port", LocalizedStrings.ServerLauncher_SERVER_PORT_HELP.toLocalizedString(String.valueOf(getDefaultServerPort())));
   }
@@ -1438,7 +1413,7 @@ public class ServerLauncher extends AbstractLauncher<String> {
       parser.accepts("pid").withRequiredArg().ofType(Integer.class);
       parser.accepts("rebalance");
       parser.accepts("redirect-output");
-      parser.accepts("server-bind-address").withRequiredArg().ofType(String.class);
+      parser.accepts(SERVER_BIND_ADDRESS).withRequiredArg().ofType(String.class);
       parser.accepts("server-port").withRequiredArg().ofType(Integer.class);
       parser.accepts("spring-xml-location").withRequiredArg().ofType(String.class);
       parser.accepts("version");
@@ -1522,8 +1497,8 @@ public class ServerLauncher extends AbstractLauncher<String> {
             setPid((Integer) options.valueOf("pid"));
           }
 
-          if (options.has("server-bind-address")) {
-            setServerBindAddress(ObjectUtils.toString(options.valueOf("server-bind-address")));
+          if (options.has(SERVER_BIND_ADDRESS)) {
+            setServerBindAddress(ObjectUtils.toString(options.valueOf(SERVER_BIND_ADDRESS)));
           }
 
           if (options.has("server-port")) {
@@ -2287,9 +2262,9 @@ public class ServerLauncher extends AbstractLauncher<String> {
     protected void validateOnStart() {
       if (Command.START.equals(getCommand())) {
         if (StringUtils.isBlank(getMemberName())
-          && !isSet(System.getProperties(), DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME)
-          && !isSet(getDistributedSystemProperties(), DistributionConfig.NAME_NAME)
-          && !isSet(loadGemFireProperties(DistributedSystem.getPropertyFileURL()), DistributionConfig.NAME_NAME))
+            && !isSet(System.getProperties(), DistributionConfig.GEMFIRE_PREFIX + SystemConfigurationProperties.NAME)
+            && !isSet(getDistributedSystemProperties(), SystemConfigurationProperties.NAME)
+            && !isSet(loadGemFireProperties(DistributedSystem.getPropertyFileURL()), SystemConfigurationProperties.NAME))
         {
           throw new IllegalStateException(LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE
             .toLocalizedString("Server"));
@@ -2342,7 +2317,7 @@ public class ServerLauncher extends AbstractLauncher<String> {
    * An enumerated type representing valid commands to the Server launcher.
    */
   public static enum Command {
-    START("start", "assign-buckets", "disable-default-server", "rebalance", "server-bind-address", "server-port", "force", "debug", "help"),
+    START("start", "assign-buckets", "disable-default-server", "rebalance", SERVER_BIND_ADDRESS, "server-port", "force", "debug", "help"),
     STATUS("status", "member", "pid", "dir", "debug", "help"),
     STOP("stop", "member", "pid", "dir", "debug", "help"),
     UNSPECIFIED("unspecified"),

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/SystemConfigurationProperties.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/SystemConfigurationProperties.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/SystemConfigurationProperties.java
new file mode 100644
index 0000000..40a11bb
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/SystemConfigurationProperties.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import com.gemstone.gemfire.distributed.internal.ConfigAttribute;
+
+/**
+ * Created by ukohlmeyer on 26/05/2016.
+ */
+public interface SystemConfigurationProperties {
+  String ACK_SEVERE_ALERT_THRESHOLD = "ack-severe-alert-threshold";
+  String ACK_WAIT_THRESHOLD = "ack-wait-threshold";
+  String ARCHIVE_DISK_SPACE_LIMIT = "archive-disk-space-limit";
+  String ARCHIVE_FILE_SIZE_LIMIT = "archive-file-size-limit";
+  String ASYNC_DISTRIBUTION_TIMEOUT = "async-distribution-timeout";
+  String ASYNC_MAX_QUEUE_SIZE = "async-max-queue-size";
+  String ASYNC_QUEUE_TIMEOUT = "async-queue-timeout";
+  String BIND_ADDRESS = "bind-address";
+  String CACHE_XML_FILE = "cache-xml-file";
+  String CLUSTER_CONFIGURATION_DIR = "cluster-configuration-dir";
+  String CLUSTER_SSL_CIPHERS = "cluster-ssl-ciphers";
+  String CLUSTER_SSL_ENABLED = "cluster-ssl-enabled";
+  String CLUSTER_SSL_KEYSTORE = "cluster-ssl-keystore";
+  String CLUSTER_SSL_KEYSTORE_PASSWORD = "cluster-ssl-keystore-password";
+  String CLUSTER_SSL_KEYSTORE_TYPE = "cluster-ssl-keystore-type";
+  String CLUSTER_SSL_PROTOCOLS = "cluster-ssl-protocols";
+  String CLUSTER_SSL_REQUIRE_AUTHENTICATION = "cluster-ssl-require-authentication";
+  String CLUSTER_SSL_TRUSTSTORE = "cluster-ssl-truststore";
+  String CLUSTER_SSL_TRUSTSTORE_PASSWORD = "cluster-ssl-truststore-password";
+  String CONFLATE_EVENTS = "conflate-events";
+  String CONSERVE_SOCKETS = "conserve-sockets";
+  String DELTA_PROPAGATION = "delta-propagation";
+  String DEPLOY_WORKING_DIR = "deploy-working-dir";
+  String DISABLE_AUTO_RECONNECT = "disable-auto-reconnect";
+  String DISABLE_TCP = "disable-tcp";
+  String DISTRIBUTED_SYSTEM_ID = "distributed-system-id";
+  String DURABLE_CLIENT_ID = "durable-client-id";
+  String DURABLE_CLIENT_TIMEOUT = "durable-client-timeout";
+  String ENABLE_CLUSTER_CONFIGURATION = "enable-cluster-configuration";
+  String ENABLE_NETWORK_PARTITION_DETECTION = "enable-network-partition-detection";
+  String ENABLE_TIME_STATISTICS = "enable-time-statistics";
+  String ENFORE_UNIQUE_HOST = "enforce-unique-host";
+  String GATEWAY_SSL_CIPHERS = "gateway-ssl-ciphers";
+  String GATEWAY_SSL_ENABLED = "gateway-ssl-enabled";
+  String GATEWAY_SSL_KEYSTORE = "gateway-ssl-keystore";
+  String GATEWAY_SSL_KEYSTORE_PASSWORD = "gateway-ssl-keystore-password";
+  String GATEWAY_SSL_KEYSTORE_TYPE = "gateway-ssl-keystore-type";
+  String GATEWAY_SSL_PROTOCOLS = "gateway-ssl-protocols";
+  String GATEWAY_SSL_REQUIRE_AUTHENTICATION = "gateway-ssl-require-authentication";
+  String GATEWAY_SSL_TRUSTSTORE = "gateway-ssl-truststore";
+  String GATEWAY_SSL_TRUSTSTORE_PASSWORD = "gateway-ssl-truststore-password";
+  String GROUPS = "groups";
+  String HTTP_SERVICE_BIND_ADDRESS = "http-service-bind-address";
+  String HTTP_SERVICE_PORT = "http-service-port";
+  String HTTP_SERVICE_SSL_CIPHERS = "http-service-ssl-ciphers";
+  String HTTP_SERVICE_SSL_ENABLED = "http-service-ssl-enabled";
+  String HTTP_SERVICE_SSL_KEYSTORE = "http-service-ssl-keystore";
+  String HTTP_SERVICE_SSL_KEYSTORE_PASSWORD = "http-service-ssl-keystore-password";
+  String HTTP_SERVICE_SSL_KEYSTORE_TYPE = "http-service-ssl-keystore-type";
+  String HTTP_SERVICE_SSL_PROTOCOLS = "http-service-ssl-protocols";
+  String HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION = "http-service-ssl-require-authentication";
+  String HTTP_SERVICE_SSL_TRUSTSTORE = "http-service-ssl-truststore";
+  String HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD = "http-service-ssl-truststore-password";
+  String JMX_MANAGER = "jmx-manager";
+  String JMX_MANAGER_ACCESS_FILE = "jmx-manager-access-file";
+  String JMX_MANAGER_BIND_ADDRESS = "jmx-manager-bind-address";
+  String JMX_MANAGER_HOSTNAME_FOR_CLIENTS = "jmx-manager-hostname-for-clients";
+  String JMX_MANAGER_HTTP_PORT = "jmx-manager-http-port";
+  String JMX_MANAGER_PASSWORD_FILE = "jmx-manager-password-file";
+  String JMX_MANAGER_PORT = "jmx-manager-port";
+  String JMX_MANAGER_SSL = "jmx-manager-ssl";
+  String JMX_MANAGER_START = "jmx-manager-start";
+  String JMX_MANAGER_UPDATE_RATE = "jmx-manager-update-rate";
+  String JMX_MANAGER_SSL_CIPHERS = "jmx-manager-ssl-ciphers";
+  String JMX_MANAGER_SSL_ENABLED = "jmx-manager-ssl-enabled";
+  String JMX_MANAGER_SSL_KEYSTORE = "jmx-manager-ssl-keystore";
+  String JMX_MANAGER_SSL_KEYSTORE_PASSWORD = "jmx-manager-ssl-keystore-password";
+  String JMX_MANAGER_SSL_KEYSTORE_TYPE = "jmx-manager-ssl-keystore-type";
+  String JMX_MANAGER_SSL_PROTOCOLS = "jmx-manager-ssl-protocols";
+  String JMX_MANAGER_SSL_REQUIRE_AUTHENTICATION = "jmx-manager-ssl-require-authentication";
+  String JMX_MANAGER_SSL_TRUSTSTORE = "jmx-manager-ssl-truststore";
+  String JMX_MANAGER_SSL_TRUSTSTORE_PASSWORD = "jmx-manager-ssl-truststore-password";
+  String LICENCE_APPLICATION_CACHE = "license-application-cache";
+  String LICENCE_DATA_MANAGEMENT = "license-data-management";
+  String LICENCE_SERVER_TIMEOUT = "license-server-timeout";
+  String LICENCE_WORKING_DIR = "license-working-dir";
+  String LOAD_CLUSTER_CONFIGURATION_FROM_DIR = "load-cluster-configuration-from-dir";
+  String LOCATOR_WAIT_TIME = "locator-wait-time";
+  String LOCATORS = "locators";
+  String LOG_DISK_SPACE_LIMIT = "log-disk-space-limit";
+  String LOG_FILE = "log-file";
+  String LOG_FILE_SIZE_LIMIT = "log-file-size-limit";
+  String LOG_LEVEL = "log-level";
+  String MAX_NUM_RECONNECT_TRIES = "max-num-reconnect-tries";
+  String MAX_WAIT_TIME_RECONNECT = "max-wait-time-reconnect";
+  String MCAST_ADDRESS = "mcast-address";
+  String MCAST_FLOW_CONTROL = "mcast-flow-control";
+  String MCAST_PORT = "mcast-port";
+  String MCAST_RECV_BUFFER_SIZE = "mcast-recv-buffer-size";
+  String MCAST_SEND_BUFFER_SIZE = "mcast-send-buffer-size";
+  String MCAST_TTL = "mcast-ttl";
+  String MEMBER_TIMEOUT = "member-timeout";
+  String MEMBERSHIP_PORT_RANGE = "membership-port-range";
+  String MEMCACHED_BIND_ADDRESS = "memcached-bind-address";
+  String MEMCACHED_PORT = "memcached-port";
+  String MEMCACHED_PROTOCOL = "memcached-protocol";
+
+  /**
+   * The name of the "name" property
+   */
+  @ConfigAttribute(type = String.class)
+  String NAME = "name";
+  String REDUNDANCY_ZONE = "redundancy-zone";
+  String REMOTE_LOCATORS = "remote-locators";
+  String REMOVE_UNRESPONSIVE_CLIENT = "remove-unresponsive-client";
+  String ROLES = "roles";
+  String SECURITY_PREFIX = "security-";
+  String SECURITY_CLIENT_ACCESSOR = SECURITY_PREFIX + "client-accessor";
+  String SECURITY_CLIENT_ACCESSOR_PP = SECURITY_PREFIX + "client-accessor-pp";
+  String SECURITY_CLIENT_AUTH_INIT = SECURITY_PREFIX + "client-auth-init";
+  String SECURITY_CLIENT_AUTHENTICATOR = SECURITY_PREFIX + "client-authenticator";
+  String SECURITY_CLIENT_DHALGO = SECURITY_PREFIX + "client-dhalgo";
+  String SECURITY_LOG_FILE = SECURITY_PREFIX + "log-file";
+  String SECURITY_LOG_LEVEL = SECURITY_PREFIX + "log-level";
+  String SECURITY_PEER_AUTH_INIT = SECURITY_PREFIX + "peer-auth-init";
+  String SECURITY_PEER_AUTHENTICATOR = SECURITY_PREFIX + "peer-authenticator";
+  String SECURITY_PEER_VERIFY_MEMBER_TIMEOUT = SECURITY_PREFIX + "peer-verifymember-timeout";
+  String SERVER_BIND_ADDRESS = "server-bind-address";
+  String SERVER_SSL_CIPHERS = "server-ssl-ciphers";
+  String SERVER_SSL_ENABLED = "server-ssl-enabled";
+  String SERVER_SSL_KEYSTORE = "server-ssl-keystore";
+  String SERVER_SSL_KEYSTORE_PASSWORD = "server-ssl-keystore-password";
+  String SERVER_SSL_KEYSTORE_TYPE = "server-ssl-keystore-type";
+  String SERVER_SSL_PROTOCOLS = "server-ssl-protocols";
+  String SERVER_SSL_REQUIRE_AUTHENTICATION = "server-ssl-require-authentication";
+  String SERVER_SSL_TRUSTSTORE = "server-ssl-truststore";
+  String SERVER_SSL_TRUSTSTORE_PASSWORD = "server-ssl-truststore-password";
+  String SOCKET_BUFFER_SIZE = "socket-buffer-size";
+  String SOCKET_LEASE_TIME = "socket-lease-time";
+  String START_DEV_REST_API = "start-dev-rest-api";
+  String START_LOCATOR = "start-locator";
+  String STATISTIC_ARCHIVE_FILE = "statistic-archive-file";
+  String STATISTIC_SAMPLE_RATE = "statistic-sample-rate";
+  String STATISTIC_SAMPLING_ENABLED = "statistic-sampling-enabled";
+  String TCP_PORT = "tcp-port";
+  String UDP_FRAGMENT_SIZE = "udp-fragment-size";
+  String UDP_RECV_BUFFER_SIZE = "udp-recv-buffer-size";
+  String UDP_SEND_BUFFER_SIZE = "udp-send-buffer-size";
+  String USE_CLUSTER_CONFIGURATION = "use-cluster-configuration";
+  String USER_COMMAND_PACKAGES = "user-command-packages";
+  String OFF_HEAP_MEMORY_SIZE = "off-heap-memory-size";
+
+  String REDIS_PORT = "redis-port";
+  String REDIS_BIND_ADDRESS = "redis-bind-address";
+  String REDIS_PASSWORD = "redis-password";
+  String LOCK_MEMORY = "lock-memory";
+  String SECURITY_SHIRO_INIT = SECURITY_PREFIX + "shiro-init";
+  String DISTRIBUTED_TRANSACTIONS = "distributed-transactions";
+
+  @Deprecated
+  String SSL_ENABLED = "ssl-enabled";
+  @Deprecated
+  String SSL_PROTOCOLS = "ssl-protocols";
+  @Deprecated
+  String SSL_CIPHERS = "ssl-ciphers";
+  @Deprecated
+  String SSL_REQUIRE_AUTHENTICATION = "ssl-require-authentication";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
index ae11e5d..3b08779 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
@@ -16,20 +16,10 @@
  */
 package com.gemstone.gemfire.distributed.internal;
 
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.StringTokenizer;
-
 import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.InvalidValueException;
 import com.gemstone.gemfire.UnmodifiableException;
+import com.gemstone.gemfire.distributed.SystemConfigurationProperties;
 import com.gemstone.gemfire.internal.AbstractConfig;
 import com.gemstone.gemfire.internal.ConfigSource;
 import com.gemstone.gemfire.internal.SocketCreator;
@@ -38,6 +28,11 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.memcached.GemFireMemcachedServer;
 
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.*;
+
 /**
  * Provides an implementation of <code>DistributionConfig</code> that
  * knows how to read the configuration file.
@@ -101,8 +96,7 @@ public abstract class AbstractDistributionConfig
     }
   }
 
-
-  @ConfigAttributeChecker(name=START_LOCATOR_NAME)
+  @ConfigAttributeChecker(name = START_LOCATOR)
   protected String checkStartLocator(String value) {
     if (value != null && value.trim().length() > 0) {
       // throws IllegalArgumentException if string is malformed
@@ -111,39 +105,42 @@ public abstract class AbstractDistributionConfig
     return value;
   }
 
-
-  @ConfigAttributeChecker(name=TCP_PORT_NAME)
+  @ConfigAttributeChecker(name = TCP_PORT)
   protected int checkTcpPort(int value) {
     if ( getSSLEnabled() && value != 0 ) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE.toLocalizedString(new Object[] {TCP_PORT_NAME, Integer.valueOf(value), SSL_ENABLED_NAME}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE
+          .toLocalizedString(new Object[] { TCP_PORT, Integer.valueOf(value), SSL_ENABLED_NAME }));
     }
     if ( getClusterSSLEnabled() && value != 0 ) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE.toLocalizedString(new Object[] {TCP_PORT_NAME, Integer.valueOf(value), CLUSTER_SSL_ENABLED_NAME}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE
+          .toLocalizedString(new Object[] { TCP_PORT, Integer.valueOf(value), CLUSTER_SSL_ENABLED_NAME }));
     }
     return value;
   }
 
-  @ConfigAttributeChecker(name=MCAST_PORT_NAME)
+  @ConfigAttributeChecker(name = MCAST_PORT)
   protected int checkMcastPort(int value) {
     if ( getSSLEnabled() && value != 0 ) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE.toLocalizedString(new Object[] {MCAST_PORT_NAME, Integer.valueOf(value), SSL_ENABLED_NAME}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE
+          .toLocalizedString(new Object[] { MCAST_PORT, Integer.valueOf(value), SSL_ENABLED_NAME }));
     }
     if ( getClusterSSLEnabled() && value != 0 ) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE.toLocalizedString(new Object[] {MCAST_PORT_NAME, Integer.valueOf(value), CLUSTER_SSL_ENABLED_NAME}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_0_WHEN_2_IS_TRUE
+          .toLocalizedString(new Object[] { MCAST_PORT, Integer.valueOf(value), CLUSTER_SSL_ENABLED_NAME }));
     }
     return value;
   }
 
-
-  @ConfigAttributeChecker(name=MCAST_ADDRESS_NAME)
+  @ConfigAttributeChecker(name = MCAST_ADDRESS)
   protected InetAddress checkMcastAddress(InetAddress value) {
     if (!value.isMulticastAddress()) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_IT_WAS_NOT_A_MULTICAST_ADDRESS.toLocalizedString(new Object[] {MCAST_ADDRESS_NAME, value}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_IT_WAS_NOT_A_MULTICAST_ADDRESS
+          .toLocalizedString(new Object[] { MCAST_ADDRESS, value }));
     }
     return value;
   }
 
-  @ConfigAttributeChecker(name=BIND_ADDRESS_NAME)
+  @ConfigAttributeChecker(name = BIND_ADDRESS)
   protected String checkBindAddress(String value) {
     if (value != null && value.length() > 0 &&
         !SocketCreator.isLocalHost(value)) {
@@ -155,8 +152,7 @@ public abstract class AbstractDistributionConfig
     return value;
   }
 
-
-  @ConfigAttributeChecker(name=SERVER_BIND_ADDRESS_NAME)
+  @ConfigAttributeChecker(name = SERVER_BIND_ADDRESS)
   protected String checkServerBindAddress(String value) {
     if (value != null && value.length() > 0 &&
         !SocketCreator.isLocalHost(value)) {
@@ -171,7 +167,8 @@ public abstract class AbstractDistributionConfig
   @ConfigAttributeChecker(name=SSL_ENABLED_NAME)
   protected Boolean checkSSLEnabled(Boolean value) {
     if ( value.booleanValue() && (getMcastPort() != 0) ) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_FALSE_WHEN_2_IS_NOT_0.toLocalizedString(new Object[] {SSL_ENABLED_NAME, value, MCAST_PORT_NAME}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_FALSE_WHEN_2_IS_NOT_0
+          .toLocalizedString(new Object[] { SSL_ENABLED_NAME, value, MCAST_PORT }));
     }
     return value;
   }
@@ -179,7 +176,8 @@ public abstract class AbstractDistributionConfig
   @ConfigAttributeChecker(name=CLUSTER_SSL_ENABLED_NAME)
   protected Boolean checkClusterSSLEnabled(Boolean value) {
     if ( value.booleanValue() && (getMcastPort() != 0) ) {
-      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_FALSE_WHEN_2_IS_NOT_0.toLocalizedString(new Object[] {CLUSTER_SSL_ENABLED_NAME, value, MCAST_PORT_NAME}));
+      throw new IllegalArgumentException(LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_ITS_VALUE_MUST_BE_FALSE_WHEN_2_IS_NOT_0
+          .toLocalizedString(new Object[] { CLUSTER_SSL_ENABLED_NAME, value, MCAST_PORT }));
     }
     return value;
   }
@@ -200,7 +198,7 @@ public abstract class AbstractDistributionConfig
   @ConfigAttributeChecker(name=DISTRIBUTED_SYSTEM_ID_NAME)
   protected int checkDistributedSystemId(int value) {
     String distributedSystemListener = System
-    .getProperty("gemfire.DistributedSystemListener");
+        .getProperty(DistributionConfig.GEMFIRE_PREFIX + "DistributedSystemListener");
     //this check is specific for Jayesh's use case of WAN BootStraping
     if(distributedSystemListener == null){
       if (value < MIN_DISTRIBUTED_SYSTEM_ID) {
@@ -235,7 +233,7 @@ public abstract class AbstractDistributionConfig
    *         If <code>value</code> is not a valid locator
    *         configuration
    */
-  @ConfigAttributeChecker(name=LOCATORS_NAME)
+  @ConfigAttributeChecker(name = LOCATORS)
   protected String checkLocators(String value) {
     // validate locators value
     StringBuffer sb = new StringBuffer();
@@ -403,7 +401,7 @@ public abstract class AbstractDistributionConfig
   @ConfigAttributeChecker(name=SECURITY_PEER_AUTH_INIT_NAME)
   protected String checkSecurityPeerAuthInit(String value) {
     if (value != null && value.length() > 0 && getMcastPort() != 0) {
-      String mcastInfo = MCAST_PORT_NAME + "[" + getMcastPort() + "]";
+      String mcastInfo = MCAST_PORT + "[" + getMcastPort() + "]";
       throw new IllegalArgumentException(
         LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_2_MUST_BE_0_WHEN_SECURITY_IS_ENABLED
           .toLocalizedString(new Object[] {
@@ -416,7 +414,7 @@ public abstract class AbstractDistributionConfig
   @ConfigAttributeChecker(name=SECURITY_PEER_AUTHENTICATOR_NAME)
   protected String checkSecurityPeerAuthenticator(String value) {
     if (value != null && value.length() > 0 && getMcastPort() != 0) {
-       String mcastInfo = MCAST_PORT_NAME + "[" + getMcastPort() + "]";
+      String mcastInfo = MCAST_PORT + "[" + getMcastPort() + "]";
       throw new IllegalArgumentException(
         LocalizedStrings.AbstractDistributionConfig_COULD_NOT_SET_0_TO_1_BECAUSE_2_MUST_BE_0_WHEN_SECURITY_IS_ENABLED
         .toLocalizedString(
@@ -710,35 +708,35 @@ public abstract class AbstractDistributionConfig
       LocalizedStrings.AbstractDistributionConfig_LOG_DISK_SPACE_LIMIT_NAME
         .toLocalizedString());
 
-    m.put(LOCATORS_NAME, 
+    m.put(LOCATORS,
       LocalizedStrings.AbstractDistributionConfig_LOCATORS_NAME_0
         .toLocalizedString(DEFAULT_LOCATORS));
-    
-    m.put(LOCATOR_WAIT_TIME_NAME,
+
+    m.put(LOCATOR_WAIT_TIME,
       LocalizedStrings.AbstractDistributionConfig_LOCATOR_WAIT_TIME_NAME_0
         .toLocalizedString(Integer.valueOf(DEFAULT_LOCATOR_WAIT_TIME)));
 
-    m.put(TCP_PORT_NAME, 
+    m.put(TCP_PORT,
       LocalizedStrings.AbstractDistributionConfig_TCP_PORT_NAME_0_1_2
         .toLocalizedString( new Object[] {
           Integer.valueOf(DEFAULT_TCP_PORT),
           Integer.valueOf(MIN_TCP_PORT),
           Integer.valueOf(MAX_TCP_PORT)}));
 
-    m.put(MCAST_PORT_NAME, 
+    m.put(MCAST_PORT,
       LocalizedStrings.AbstractDistributionConfig_MCAST_PORT_NAME_0_1_2
        .toLocalizedString(new Object[] {
           Integer.valueOf(DEFAULT_MCAST_PORT),
           Integer.valueOf(MIN_MCAST_PORT), 
           Integer.valueOf(MAX_MCAST_PORT)}));
 
-    m.put(MCAST_ADDRESS_NAME, 
+    m.put(MCAST_ADDRESS,
       LocalizedStrings.AbstractDistributionConfig_MCAST_ADDRESS_NAME_0_1
        .toLocalizedString(new Object[] {
           Integer.valueOf(DEFAULT_MCAST_PORT),
           DEFAULT_MCAST_ADDRESS}));
 
-    m.put(MCAST_TTL_NAME, 
+    m.put(MCAST_TTL,
       LocalizedStrings.AbstractDistributionConfig_MCAST_TTL_NAME_0_1_2
        .toLocalizedString(new Object[] {
           Integer.valueOf(DEFAULT_MCAST_TTL),
@@ -803,15 +801,15 @@ public abstract class AbstractDistributionConfig
       LocalizedStrings.AbstractDistributionConfig_ROLES_NAME_0
         .toLocalizedString(DEFAULT_ROLES));
 
-    m.put(BIND_ADDRESS_NAME, 
+    m.put(BIND_ADDRESS,
       LocalizedStrings.AbstractDistributionConfig_BIND_ADDRESS_NAME_0
         .toLocalizedString(DEFAULT_BIND_ADDRESS));
 
-    m.put(SERVER_BIND_ADDRESS_NAME, 
+    m.put(SERVER_BIND_ADDRESS,
       LocalizedStrings.AbstractDistributionConfig_SERVER_BIND_ADDRESS_NAME_0
         .toLocalizedString(DEFAULT_BIND_ADDRESS));
 
-    m.put(NAME_NAME, "A name that uniquely identifies a member in its distributed system." +
+    m.put(SystemConfigurationProperties.NAME, "A name that uniquely identifies a member in its distributed system." +
         " Multiple members in the same distributed system can not have the same name." +
         " Defaults to \"\".");
 
@@ -904,9 +902,9 @@ public abstract class AbstractDistributionConfig
         .toLocalizedString( new Object[] {
           Integer.valueOf(DEFAULT_ASYNC_MAX_QUEUE_SIZE),
           Integer.valueOf(MIN_ASYNC_MAX_QUEUE_SIZE),
-          Integer.valueOf(MAX_ASYNC_MAX_QUEUE_SIZE)}));       
+            Integer.valueOf(MAX_ASYNC_MAX_QUEUE_SIZE) }));
 
-    m.put(START_LOCATOR_NAME, 
+    m.put(START_LOCATOR,
       LocalizedStrings.AbstractDistributionConfig_START_LOCATOR_NAME
         .toLocalizedString());
 
@@ -1148,7 +1146,7 @@ public abstract class AbstractDistributionConfig
     return dcAttDescriptions;
   }
 
-  static final InetAddress _getDefaultMcastAddress() {
+  public static final InetAddress _getDefaultMcastAddress() {
     String ipLiteral;
     if ( SocketCreator.preferIPv6Addresses() ) {
       ipLiteral = "FF38::1234"; // fix for bug 30014

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DSClock.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DSClock.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DSClock.java
index ff92d55..d13610a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DSClock.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DSClock.java
@@ -16,11 +16,6 @@
  */
 package com.gemstone.gemfire.distributed.internal;
 
-import java.util.Date;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.SystemTimer.SystemTimerTask;
@@ -29,6 +24,10 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.DateFormatter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Date;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * DSClock tracks the system time.  The most useful method is 
@@ -115,7 +114,7 @@ public class DSClock {
    * @since GemFire 8.0
    */
   public void setCacheTimeOffset(DistributedMember coord, long offset, boolean isJoin) {
-    if (Boolean.getBoolean("gemfire.disable-distributed-clock")) {
+    if (Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "disable-distributed-clock")) {
       return;
     }
     if (isLoner) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
index 08cf39d..cefe9da 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
@@ -17,23 +17,6 @@
 
 package com.gemstone.gemfire.distributed.internal;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.GemFireIOException;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
@@ -47,11 +30,19 @@ import com.gemstone.gemfire.internal.cache.UpdateAttributesProcessor;
 import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
 import com.gemstone.gemfire.internal.cache.versions.VersionSource;
 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.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Provides advice on sending distribution messages. For a given operation,
@@ -83,13 +74,13 @@ public class DistributionAdvisor  {
    * profileVersionSequencer.
    */
   public static final int START_VERSION_NUMBER = Integer.getInteger(
-      "gemfire.DistributionAdvisor.startVersionNumber", 1).intValue();
+      DistributionConfig.GEMFIRE_PREFIX + "DistributionAdvisor.startVersionNumber", 1).intValue();
   
   /**
    * Specifies the starting serial number for the serialNumberSequencer.
    */
   public static final int START_SERIAL_NUMBER = Integer.getInteger(
-      "gemfire.Cache.startSerialNumber",
+      DistributionConfig.GEMFIRE_PREFIX + "Cache.startSerialNumber",
       1 // Integer.MAX_VALUE-10
       ).intValue();
 
@@ -109,7 +100,7 @@ public class DistributionAdvisor  {
    * if a rollover has occurred.
    */
   public static final int ROLLOVER_THRESHOLD = Integer.getInteger(
-      "gemfire.CacheDistributionAdvisor.rolloverThreshold", 
+      DistributionConfig.GEMFIRE_PREFIX + "CacheDistributionAdvisor.rolloverThreshold",
       1000).intValue();
   
   /**