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

[05/90] [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/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
index 6c11c96..6b91763 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
@@ -16,8 +16,12 @@
  */
 package com.gemstone.gemfire.redis;
 
-import static org.junit.Assert.assertFalse;
-
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -25,12 +29,9 @@ import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertFalse;
 
 @Category(IntegrationTest.class)
 public class ConcurrentStartTest {
@@ -63,8 +64,8 @@ public class ConcurrentStartTest {
   @Test
   public void testCachefulStart() throws InterruptedException {
     CacheFactory cf = new CacheFactory();
-    cf.set("mcast-port", "0");
-    cf.set("locators", "");
+    cf.set(MCAST_PORT, "0");
+    cf.set(LOCATORS, "");
     this.cache = cf.create();
     
     runNServers(numServers);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
index c0da54b..fed4803 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
@@ -16,30 +16,24 @@
  */
 package com.gemstone.gemfire.redis;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import redis.clients.jedis.Jedis;
 
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import java.io.IOException;
+import java.util.*;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.*;
 
 @Category(IntegrationTest.class)
 public class HashesJUnitTest {
@@ -54,9 +48,9 @@ public class HashesJUnitTest {
     rand = new Random();
     CacheFactory cf = new CacheFactory();
     //cf.set("log-file", "redis.log");
-    cf.set("log-level", "error");
-    cf.set("mcast-port", "0");
-    cf.set("locators", "");
+    cf.set(DistributionConfig.LOG_LEVEL_NAME, "error");
+    cf.set(MCAST_PORT, "0");
+    cf.set(LOCATORS, "");
     cache = cf.create();
     port = AvailablePortHelper.getRandomAvailableTCPPort();
     server = new GemFireRedisServer("localhost", port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
index 6548777..17a2c3f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
@@ -16,27 +16,27 @@
  */
 package com.gemstone.gemfire.redis;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import redis.clients.jedis.Jedis;
 
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 @Category(IntegrationTest.class)
 public class ListsJUnitTest {
@@ -52,9 +52,9 @@ public class ListsJUnitTest {
     rand = new Random();
     CacheFactory cf = new CacheFactory();
     //cf.set("log-file", "redis.log");
-    cf.set("log-level", "error");
-    cf.set("mcast-port", "0");
-    cf.set("locators", "");
+    cf.set(DistributionConfig.LOG_LEVEL_NAME, "error");
+    cf.set(MCAST_PORT, "0");
+    cf.set(LOCATORS, "");
     cache = cf.create();
     port = AvailablePortHelper.getRandomAvailableTCPPort();
     server = new GemFireRedisServer("localhost", port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
index 23f202d..0ed99bd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
@@ -16,23 +16,20 @@
  */
 package com.gemstone.gemfire.redis;
 
-import java.util.Random;
-
-import org.junit.experimental.categories.Category;
-import redis.clients.jedis.Jedis;
-
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.SystemConfigurationProperties;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.SerializableCallable;
-import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import org.junit.experimental.categories.Category;
+import redis.clients.jedis.Jedis;
+
+import java.util.Random;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 public class RedisDistDUnitTest extends DistributedTestCase {
 
@@ -84,11 +81,11 @@ public class RedisDistDUnitTest extends DistributedTestCase {
         int port = ports[VM.getCurrentVMNum()];
         CacheFactory cF = new CacheFactory();
         String locator = SocketCreator.getLocalHost().getHostName() + "[" + locatorPort + "]";
-        cF.set("log-level", LogWriterUtils.getDUnitLogLevel());
-        cF.set("redis-bind-address", localHost);
-        cF.set("redis-port", ""+port);
-        cF.set("mcast-port", "0");
-        cF.set("locators", locator);
+        cF.set(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
+        cF.set(SystemConfigurationProperties.REDIS_BIND_ADDRESS, localHost);
+        cF.set(SystemConfigurationProperties.REDIS_PORT, "" + port);
+        cF.set(MCAST_PORT, "0");
+        cF.set(LOCATORS, locator);
         cF.create();
         return Integer.valueOf(port);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
index 9da65e7..28c35bf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
@@ -16,27 +16,28 @@
  */
 package com.gemstone.gemfire.redis;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import redis.clients.jedis.Jedis;
 
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 @Category(IntegrationTest.class)
 public class SetsJUnitTest {
@@ -52,9 +53,9 @@ public class SetsJUnitTest {
     rand = new Random();
     CacheFactory cf = new CacheFactory();
     //cf.set("log-file", "redis.log");
-    cf.set("log-level", "error");
-    cf.set("mcast-port", "0");
-    cf.set("locators", "");
+    cf.set(DistributionConfig.LOG_LEVEL_NAME, "error");
+    cf.set(MCAST_PORT, "0");
+    cf.set(LOCATORS, "");
     cache = cf.create();
     port = AvailablePortHelper.getRandomAvailableTCPPort();
     server = new GemFireRedisServer("localhost", port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
index e7dbec8..90b2827 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
@@ -16,36 +16,26 @@
  */
 package com.gemstone.gemfire.redis;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import redis.clients.jedis.Jedis;
 import redis.clients.jedis.Tuple;
 
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import java.io.IOException;
+import java.util.*;
+import java.util.Map.Entry;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.*;
 
 @Category(IntegrationTest.class)
 public class SortedSetsJUnitTest {
@@ -60,9 +50,9 @@ public class SortedSetsJUnitTest {
     rand = new Random();
     CacheFactory cf = new CacheFactory();
     //cf.set("log-file", "redis.log");
-    cf.set("log-level", "error");
-    cf.set("mcast-port", "0");
-    cf.set("locators", "");
+    cf.set(DistributionConfig.LOG_LEVEL_NAME, "error");
+    cf.set(MCAST_PORT, "0");
+    cf.set(LOCATORS, "");
     cache = cf.create();
     port = AvailablePortHelper.getRandomAvailableTCPPort();
     server = new GemFireRedisServer("localhost", port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
index d273e89..8939e0b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
@@ -16,30 +16,24 @@
  */
 package com.gemstone.gemfire.redis;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import redis.clients.jedis.Jedis;
 
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import java.io.IOException;
+import java.util.*;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.*;
 
 @Category(IntegrationTest.class)
 public class StringsJunitTest {
@@ -55,9 +49,9 @@ public class StringsJunitTest {
     rand = new Random();
     CacheFactory cf = new CacheFactory();
     //cf.set("log-file", "redis.log");
-    cf.set("log-level", "error");
-    cf.set("mcast-port", "0");
-    cf.set("locators", "");
+    cf.set(DistributionConfig.LOG_LEVEL_NAME, "error");
+    cf.set(MCAST_PORT, "0");
+    cf.set(LOCATORS, "");
     cache = cf.create();
     port = AvailablePortHelper.getRandomAvailableTCPPort();
     server = new GemFireRedisServer("localhost", port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestUtils.java
index 89a0a15..1ee8768 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestUtils.java
@@ -16,13 +16,13 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
-import static com.gemstone.gemfire.security.SecurityTestUtils.*;
-import static org.junit.Assert.*;
+import com.gemstone.gemfire.cache.Region;
 
 import java.util.Properties;
 
-import com.gemstone.gemfire.cache.Region;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.SECURITY_CLIENT_AUTHENTICATOR;
+import static com.gemstone.gemfire.security.SecurityTestUtils.*;
+import static org.junit.Assert.assertNotNull;
 
 /**
  * Extracted from ClientAuthenticationDUnitTest
@@ -41,7 +41,7 @@ public abstract class ClientAuthenticationTestUtils {
     }
 
     if (authenticator != null) {
-      authProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator);
+      authProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR, authenticator);
     }
 
     return SecurityTestUtils.createCacheServer(authProps, javaProps, locatorPort, locatorString, 0, NO_EXCEPTION);
@@ -56,7 +56,7 @@ public abstract class ClientAuthenticationTestUtils {
     }
 
     if (authenticator != null) {
-      authProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator);
+      authProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR, authenticator);
     }
     SecurityTestUtils.createCacheServer(authProps, javaProps, locatorPort, locatorString, serverPort, NO_EXCEPTION);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
index 22c4cb2..4131fd4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
@@ -18,44 +18,12 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
-import static com.gemstone.gemfire.internal.AvailablePort.*;
-import static com.gemstone.gemfire.security.SecurityTestUtils.*;
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.test.dunit.Host.*;
-import static com.gemstone.gemfire.test.dunit.Wait.*;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-import com.gemstone.gemfire.cache.Operation;
-import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.Region.Entry;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.client.ServerConnectivityException;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.cache.query.CqAttributes;
-import com.gemstone.gemfire.cache.query.CqAttributesFactory;
-import com.gemstone.gemfire.cache.query.CqEvent;
-import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.CqListener;
-import com.gemstone.gemfire.cache.query.CqQuery;
-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.internal.AvailablePort.*;
+import com.gemstone.gemfire.cache.query.*;
+import com.gemstone.gemfire.internal.AvailablePort.Keeper;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -68,6 +36,17 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.*;
+import static com.gemstone.gemfire.internal.AvailablePort.SOCKET;
+import static com.gemstone.gemfire.internal.AvailablePort.getRandomAvailablePort;
+import static com.gemstone.gemfire.security.SecurityTestUtils.*;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Host.getHost;
+import static com.gemstone.gemfire.test.dunit.Wait.waitForCriterion;
+
 /**
  * Base class for tests for authorization from client to server. It contains
  * utility functions for the authorization tests from client to server.
@@ -186,13 +165,13 @@ public abstract class ClientAuthorizationTestCase extends JUnit4DistributedTestC
   protected static Properties buildProperties(final String authenticator, final String accessor, final boolean isAccessorPP, final Properties extraAuthProps, final Properties extraAuthzProps) {
     Properties authProps = new Properties();
     if (authenticator != null) {
-      authProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator);
+      authProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR, authenticator);
     }
     if (accessor != null) {
       if (isAccessorPP) {
-        authProps.setProperty(SECURITY_CLIENT_ACCESSOR_PP_NAME, accessor);
+        authProps.setProperty(SECURITY_CLIENT_ACCESSOR_PP, accessor);
       } else {
-        authProps.setProperty(SECURITY_CLIENT_ACCESSOR_NAME, accessor);
+        authProps.setProperty(SECURITY_CLIENT_ACCESSOR, accessor);
       }
     }
     return concatProperties(new Properties[] { authProps, extraAuthProps, extraAuthzProps });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
index 6970ef2..cd11c7a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
@@ -18,23 +18,9 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
-import static com.gemstone.gemfire.internal.AvailablePort.*;
-import static com.gemstone.gemfire.security.SecurityTestUtils.*;
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
-import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
-import static com.gemstone.gemfire.test.dunit.Wait.*;
-
-import java.util.Properties;
-import javax.net.ssl.SSLHandshakeException;
-
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.SystemConfigurationProperties;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
@@ -50,6 +36,22 @@ import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import javax.net.ssl.SSLHandshakeException;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.*;
+import static com.gemstone.gemfire.internal.AvailablePort.SOCKET;
+import static com.gemstone.gemfire.internal.AvailablePort.getRandomAvailablePort;
+import static com.gemstone.gemfire.security.SecurityTestUtils.startLocator;
+import static com.gemstone.gemfire.security.SecurityTestUtils.stopLocator;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.IgnoredException.addIgnoredException;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.getIPLiteral;
+import static com.gemstone.gemfire.test.dunit.Wait.pause;
 
 /**
  * Tests peer to peer authentication in Gemfire
@@ -89,10 +91,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     int port = getRandomAvailablePort(SOCKET);
 
     Properties props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "26753");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() + "[" + port + "]");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, UserPasswordAuthInit.class.getName() + ".create");
-    props.setProperty(ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+    props.setProperty(MCAST_PORT, "26753");
+    props.setProperty(SystemConfigurationProperties.LOCATORS, getIPLiteral() + "[" + port + "]");
+    props.setProperty(SystemConfigurationProperties.SECURITY_PEER_AUTH_INIT, UserPasswordAuthInit.class.getName() + ".create");
+    props.setProperty(SystemConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION, "false");
 
     try {
       Locator.startLocatorAndDS(port, null, null, props);
@@ -104,10 +106,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
 
     // Also try setting the authenticator
     props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "26753");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() +"[" + port + "]");
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, LdapUserAuthenticator.class.getName() + ".create");
-    props.setProperty(ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+    props.setProperty(MCAST_PORT, "26753");
+    props.setProperty(LOCATORS, getIPLiteral() + "[" + port + "]");
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, LdapUserAuthenticator.class.getName() + ".create");
+    props.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
 
     try {
       Locator.startLocatorAndDS(port, null, null, props);
@@ -118,8 +120,8 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     }
 
     props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "26753");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, UserPasswordAuthInit.class.getName() + ".create");
+    props.setProperty(MCAST_PORT, "26753");
+    props.setProperty(SECURITY_PEER_AUTH_INIT, UserPasswordAuthInit.class.getName() + ".create");
 
     try {
       getSystem(props);
@@ -131,8 +133,8 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
 
     // Also try setting the authenticator
     props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "26753");
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, LdapUserAuthenticator.class.getName() + ".create");
+    props.setProperty(MCAST_PORT, "26753");
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, LdapUserAuthenticator.class.getName() + ".create");
 
     try {
       getSystem(props);
@@ -155,10 +157,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     assertNull(gen.getJavaProperties());
 
     Properties props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "0");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() + "[" + locatorPort + "]");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, "Incorrect_AuthInitialize");
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+    props.setProperty(SECURITY_PEER_AUTH_INIT, "Incorrect_AuthInitialize");
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
 
     startTheLocator(props, gen.getJavaProperties(), locatorPort);
 
@@ -187,10 +189,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     assertNull(gen.getJavaProperties());
 
     Properties props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "0");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() +"["+locatorPort+"]");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, gen.getAuthInit());
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, "xyz");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+    props.setProperty(SECURITY_PEER_AUTH_INIT, gen.getAuthInit());
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, "xyz");
 
     startTheLocator(props, null, locatorPort);
 
@@ -218,10 +220,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     assertNull(gen.getSystemProperties());
 
     Properties props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "0");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() +"["+locatorPort+"]");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, gen.getAuthInit());
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+    props.setProperty(SECURITY_PEER_AUTH_INIT, gen.getAuthInit());
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
 
     startTheLocator(props, null, locatorPort);
 
@@ -249,10 +251,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     assertNotNull(gen.getValidCredentials(1));
 
     Properties props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "0");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() +"["+locatorPort+"]");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, gen.getAuthInit());
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+    props.setProperty(SECURITY_PEER_AUTH_INIT, gen.getAuthInit());
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
     props.putAll(gen.getValidCredentials(1));
 
     startTheLocator(props, gen.getJavaProperties(), locatorPort);
@@ -283,10 +285,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     assertNotNull(gen.getValidCredentials(3));
 
     Properties props = new Properties();
-    props.setProperty(MCAST_PORT_NAME, "0");
-    props.setProperty(LOCATORS_NAME, getIPLiteral() +"["+locatorPort+"]");
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, gen.getAuthInit());
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+    props.setProperty(SECURITY_PEER_AUTH_INIT, gen.getAuthInit());
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
     props.putAll(gen.getValidCredentials(1));
 
     startTheLocator(props, null, locatorPort);
@@ -354,8 +356,8 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     int port = getRandomAvailablePort(SOCKET);
     final String locators = getIPLiteral() +"["+port+"]";
 
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, authInit);
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, authenticator);
+    props.setProperty(SECURITY_PEER_AUTH_INIT, authInit);
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, authenticator);
     Properties credentials = gen.getValidCredentials(1);
     Properties javaProps = gen.getJavaProperties();
     props.putAll(credentials);
@@ -367,10 +369,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
 
       // Start the first peer with different authenticator
       props = new Properties();
-      props.setProperty(MCAST_PORT_NAME, "0");
-      props.setProperty(LOCATORS_NAME, locators);
-      props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, authInit);
-      props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, authenticator2);
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, locators);
+      props.setProperty(SECURITY_PEER_AUTH_INIT, authInit);
+      props.setProperty(SECURITY_PEER_AUTHENTICATOR, authenticator2);
 
       credentials = gen.getValidCredentials(3);
       Properties javaProps2 = gen2.getJavaProperties();
@@ -381,10 +383,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
 
       // Start the second peer with the same authenticator as locator
       props = new Properties();
-      props.setProperty(MCAST_PORT_NAME, "0");
-      props.setProperty(LOCATORS_NAME, locators);
-      props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, authInit);
-      props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, authenticator);
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, locators);
+      props.setProperty(SECURITY_PEER_AUTH_INIT, authInit);
+      props.setProperty(SECURITY_PEER_AUTHENTICATOR, authenticator);
 
       credentials = gen.getValidCredentials(7);
       javaProps = gen.getJavaProperties();
@@ -457,8 +459,8 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     Properties credentials = gen.getValidCredentials(1);
 
     Properties props = new Properties();
-    props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, authInit);
-    props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+    props.setProperty(SECURITY_PEER_AUTH_INIT, authInit);
+    props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
     props.putAll(credentials);
 
     startTheLocator(props, null, locatorPort);
@@ -466,10 +468,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
     try {
       // Start the first peer with huge credentials
       props = new Properties();
-      props.setProperty(MCAST_PORT_NAME, "0");
-      props.setProperty(LOCATORS_NAME, getIPLiteral() +"["+locatorPort+"]");
-      props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, authInit);
-      props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+      props.setProperty(SECURITY_PEER_AUTH_INIT, authInit);
+      props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
 
       String hugeStr = "20KString";
       for (int i = 0; i <= 20000; i++) {
@@ -489,10 +491,10 @@ public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
 
       // Start the second peer with the same authenticator as locator
       props = new Properties();
-      props.setProperty(MCAST_PORT_NAME, "0");
-      props.setProperty(LOCATORS_NAME, getIPLiteral() +"["+locatorPort+"]");
-      props.setProperty(SECURITY_PEER_AUTH_INIT_NAME, authInit);
-      props.setProperty(SECURITY_PEER_AUTHENTICATOR_NAME, gen.getAuthenticator());
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, getIPLiteral() + "[" + locatorPort + "]");
+      props.setProperty(SECURITY_PEER_AUTH_INIT, authInit);
+      props.setProperty(SECURITY_PEER_AUTHENTICATOR, gen.getAuthenticator());
 
       credentials = gen.getValidCredentials(7);
       props.putAll(credentials);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
index e992dd0..56da24a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
@@ -18,59 +18,8 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.cache30.ClientServerTestCase.*;
-import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
-import static com.gemstone.gemfire.internal.AvailablePort.*;
-import static com.gemstone.gemfire.test.dunit.Assert.assertEquals;
-import static com.gemstone.gemfire.test.dunit.Assert.assertFalse;
-import static com.gemstone.gemfire.test.dunit.Assert.assertNotNull;
-import static com.gemstone.gemfire.test.dunit.Assert.assertNull;
-import static com.gemstone.gemfire.test.dunit.Assert.assertTrue;
-import static com.gemstone.gemfire.test.dunit.Assert.fail;
-import static com.gemstone.gemfire.test.dunit.DistributedTestUtils.*;
-import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
-import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
-import static com.gemstone.gemfire.test.dunit.Wait.*;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import javax.net.ServerSocketFactory;
-import javax.net.SocketFactory;
-import javax.net.ssl.KeyManager;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLContextSpi;
-import javax.net.ssl.SSLServerSocketFactory;
-import javax.net.ssl.SSLSocketFactory;
-import javax.net.ssl.TrustManager;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.NoAvailableServersException;
-import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.client.PoolFactory;
-import com.gemstone.gemfire.cache.client.PoolManager;
-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.*;
+import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ProxyCache;
 import com.gemstone.gemfire.cache.execute.Execution;
@@ -83,6 +32,7 @@ import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.pdx.PdxReader;
 import com.gemstone.gemfire.pdx.PdxSerializable;
 import com.gemstone.gemfire.pdx.PdxWriter;
@@ -90,6 +40,28 @@ import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 
+import javax.net.ServerSocketFactory;
+import javax.net.SocketFactory;
+import javax.net.ssl.*;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static com.gemstone.gemfire.cache30.ClientServerTestCase.configureConnectionPoolWithNameAndFactory;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.*;
+import static com.gemstone.gemfire.internal.AvailablePort.SOCKET;
+import static com.gemstone.gemfire.internal.AvailablePort.getRandomAvailablePort;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.DistributedTestUtils.getDUnitLocatorPort;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.getLogWriter;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.getIPLiteral;
+import static com.gemstone.gemfire.test.dunit.Wait.waitForCriterion;
+
 /**
  * Contains utility methods for setting up servers/clients for authentication
  * and authorization tests.
@@ -213,14 +185,14 @@ public final class SecurityTestUtils {
     if (authProps == null) {
       authProps = new Properties();
     }
-    authProps.setProperty(MCAST_PORT_NAME, "0");
+    authProps.setProperty(MCAST_PORT, "0");
     if (locatorString != null && locatorString.length() > 0) {
-      authProps.setProperty(LOCATORS_NAME, locatorString);
-      authProps.setProperty(START_LOCATOR_NAME, getIPLiteral() + "[" + locatorPort + ']');
+      authProps.setProperty(LOCATORS, locatorString);
+      authProps.setProperty(START_LOCATOR, getIPLiteral() + "[" + locatorPort + ']');
     } else {
-      authProps.setProperty("locators", "localhost["+getDUnitLocatorPort()+"]");
+      authProps.setProperty(LOCATORS, "localhost[" + getDUnitLocatorPort() + "]");
     }
-    authProps.setProperty(SECURITY_LOG_LEVEL_NAME, "finest");
+    authProps.setProperty(SECURITY_LOG_LEVEL, "finest");
 
     getLogWriter().info("Set the server properties to: " + authProps);
     getLogWriter().info("Set the java properties to: " + javaProps);
@@ -308,12 +280,12 @@ public final class SecurityTestUtils {
     if (authProps == null) {
       authProps = new Properties();
     }
-    authProps.setProperty(MCAST_PORT_NAME, "0");
-    authProps.setProperty(LOCATORS_NAME, "");
-    authProps.setProperty(SECURITY_LOG_LEVEL_NAME, "finest");
+    authProps.setProperty(MCAST_PORT, "0");
+    authProps.setProperty(LOCATORS, "");
+    authProps.setProperty(SECURITY_LOG_LEVEL, "finest");
     // TODO (ashetkar) Add " && (!multiUserAuthMode)" below.
     if (authInitModule != null) {
-      authProps.setProperty(SECURITY_CLIENT_AUTH_INIT_NAME, authInitModule);
+      authProps.setProperty(SECURITY_CLIENT_AUTH_INIT, authInitModule);
     }
 
     SecurityTestUtils tmpInstance = new SecurityTestUtils("temp");
@@ -418,20 +390,20 @@ public final class SecurityTestUtils {
     if (authProps[0] == null) {
       authProps[0] = new Properties();
     }
-    authProps[0].setProperty(MCAST_PORT_NAME, "0");
-    authProps[0].setProperty(LOCATORS_NAME, "");
-    authProps[0].setProperty(SECURITY_LOG_LEVEL_NAME, "finest");
+    authProps[0].setProperty(MCAST_PORT, "0");
+    authProps[0].setProperty(LOCATORS, "");
+    authProps[0].setProperty(SECURITY_LOG_LEVEL, "finest");
 
     Properties props = new Properties();
 
     if (authInitModule != null) {
-      authProps[0].setProperty(SECURITY_CLIENT_AUTH_INIT_NAME, authInitModule);
-      props.setProperty(SECURITY_CLIENT_AUTH_INIT_NAME, authInitModule);
+      authProps[0].setProperty(SECURITY_CLIENT_AUTH_INIT, authInitModule);
+      props.setProperty(SECURITY_CLIENT_AUTH_INIT, authInitModule);
     }
 
     if (durableClientId != null) {
-      props.setProperty(DURABLE_CLIENT_ID_NAME, durableClientId);
-      props.setProperty(DURABLE_CLIENT_TIMEOUT_NAME, String.valueOf(DEFAULT_DURABLE_CLIENT_TIMEOUT));
+      props.setProperty(DURABLE_CLIENT_ID, durableClientId);
+      props.setProperty(DURABLE_CLIENT_TIMEOUT, String.valueOf(DistributionConfig.DEFAULT_DURABLE_CLIENT_TIMEOUT));
     }
 
     SecurityTestUtils tmpInstance = new SecurityTestUtils("temp");
@@ -516,9 +488,9 @@ public final class SecurityTestUtils {
       if (extraProps != null) {
         authProps.putAll(extraProps);
       }
-      authProps.setProperty(MCAST_PORT_NAME, "0");
-      authProps.setProperty(LOCATORS_NAME, getIPLiteral() + "[" + port + "]");
-      authProps.setProperty(ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+      authProps.setProperty(MCAST_PORT, "0");
+      authProps.setProperty(LOCATORS, getIPLiteral() + "[" + port + "]");
+      authProps.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
 
       clearStaticSSLContext();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
index 7bdcd9a..2b3a427 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
@@ -16,12 +16,6 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import java.security.Principal;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -29,6 +23,11 @@ import com.gemstone.gemfire.security.templates.LdapUserAuthenticator;
 import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.util.test.TestUtil;
+import org.apache.logging.log4j.Logger;
+
+import java.security.Principal;
+import java.util.Properties;
+import java.util.Random;
 
 public class LdapUserCredentialGenerator extends CredentialGenerator {
 
@@ -67,7 +66,7 @@ public class LdapUserCredentialGenerator extends CredentialGenerator {
     if (serverAuthEnabled) {
       String keyStoreFile = TestUtil.getResourcePath(LdapUserCredentialGenerator.class, PKCSCredentialGenerator.keyStoreDir + "/gemfire1.keystore");
       extraProps.setProperty(HandShake.PRIVATE_KEY_FILE_PROP, keyStoreFile);
-      extraProps.setProperty(HandShake.PRIVATE_KEY_ALIAS_PROP, "gemfire1");
+      extraProps.setProperty(HandShake.PRIVATE_KEY_ALIAS_PROP, DistributionConfig.GEMFIRE_PREFIX);
       extraProps.setProperty(HandShake.PRIVATE_KEY_PASSWD_PROP, "gemfire");
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
index 6f7cf29..14f1f80 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
@@ -16,16 +16,16 @@
  */
 package com.gemstone.gemfire.security.generator;
 
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+import org.apache.logging.log4j.Logger;
+
 import java.io.File;
 import java.io.IOException;
 import java.security.Principal;
 import java.util.Properties;
 
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-
 public class SSLCredentialGenerator extends CredentialGenerator {
 
   private static final Logger logger = LogService.getLogger();
@@ -113,10 +113,10 @@ public class SSLCredentialGenerator extends CredentialGenerator {
 
   private Properties getSSLProperties() {
     Properties props = new Properties();
-    props.setProperty("ssl-enabled", "true");
-    props.setProperty("ssl-require-authentication", "true");
-    props.setProperty("ssl-ciphers", "SSL_RSA_WITH_3DES_EDE_CBC_SHA");
-    props.setProperty("ssl-protocols", "TLSv1");
+    props.setProperty(DistributionConfig.SSL_ENABLED_NAME, "true");
+    props.setProperty(DistributionConfig.SSL_REQUIRE_AUTHENTICATION_NAME, "true");
+    props.setProperty(DistributionConfig.SSL_CIPHERS_NAME, "SSL_RSA_WITH_3DES_EDE_CBC_SHA");
+    props.setProperty(DistributionConfig.SSL_PROTOCOLS_NAME, "TLSv1");
     return props;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
index 06a4997..1240533 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
@@ -16,14 +16,15 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import java.util.Iterator;
-import java.util.Properties;
-
 import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.security.AuthInitialize;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 
+import java.util.Iterator;
+import java.util.Properties;
+
 /**
  * An {@link AuthInitialize} implementation that obtains the user name and
  * password as the credentials from the given set of properties. If 
@@ -35,7 +36,7 @@ import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
  */
 public class UserPasswordWithExtraPropsAuthInit extends UserPasswordAuthInit {
 
-  public static final String SECURITY_PREFIX = "security-";
+  public static final String SECURITY_PREFIX = DistributionConfig.SECURITY_PREFIX_NAME;
   public static final String EXTRA_PROPS = "security-keep-extra-props";
 
   public static AuthInitialize create() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
index 9f69dab..fa54b7a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
@@ -16,13 +16,6 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -30,6 +23,13 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManage
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalInstantiator;
 
+import java.io.File;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+
 /**
  * <code>DistributedTestUtils</code> provides static utility methods that 
  * affect the runtime environment or artifacts generated by a DistributedTest.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/LogWriterUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/LogWriterUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/LogWriterUtils.java
index 9556af2..0a482d4 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/LogWriterUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/LogWriterUtils.java
@@ -16,10 +16,6 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
-import java.util.Properties;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
@@ -28,6 +24,9 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LogWriterFactory;
 import com.gemstone.gemfire.internal.logging.ManagerLogWriter;
 import com.gemstone.gemfire.internal.logging.log4j.LogWriterLogger;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Properties;
 
 /**
  * <code>LogWriterUtils</code> provides static utility methods to access a

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/NetworkUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/NetworkUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/NetworkUtils.java
index d83aecd..dec882e 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/NetworkUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/NetworkUtils.java
@@ -16,10 +16,11 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
-import java.net.UnknownHostException;
-
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.SocketCreator;
 
+import java.net.UnknownHostException;
+
 /**
  * <code>NetworkUtils</code> provides static utility methods to perform
  * network DNS lookups or similar actions.
@@ -63,7 +64,7 @@ public class NetworkUtils {
    * @return the host name
    */
   public static String getServerHostName(final Host host) {
-    String serverBindAddress = System.getProperty("gemfire.server-bind-address");
+    String serverBindAddress = System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "server-bind-address");
     return serverBindAddress != null ? serverBindAddress : host.getHostName();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
index 1d517bc..ea8e02c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
@@ -21,6 +21,7 @@ import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionMessageObserver;
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -41,6 +42,8 @@ import java.util.Arrays;
 import java.util.Map;
 import java.util.Properties;
 
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+
 /**
  * This class is the base class for all distributed tests using JUnit 4 that
  * require the creation of a {@link Cache}.
@@ -89,11 +92,11 @@ public class JUnit4CacheTestCase extends JUnit4DistributedTestCase implements Ca
   private final void createCache(final boolean client, final CacheFactory factory) {
     synchronized(JUnit4CacheTestCase.class) {
       try {
-        System.setProperty("gemfire.DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE", "true");
+        System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE", "true");
         Cache newCache;
         if (client) {
-          System.setProperty("gemfire.locators", "");
-          System.setProperty("gemfire.mcast-port", "0");
+          System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "locators", "");
+          System.setProperty(DistributionConfig.GEMFIRE_PREFIX + MCAST_PORT, "0");
           newCache = (Cache)new ClientCacheFactory(getSystem().getProperties()).create();
         } else {
           if(factory == null) {
@@ -116,9 +119,9 @@ public class JUnit4CacheTestCase extends JUnit4DistributedTestCase implements Ca
       } catch (Exception ex) {
         Assert.fail("Checked exception while initializing cache??", ex);
       } finally {
-        System.clearProperty("gemfire.DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE");
-        System.clearProperty("gemfire.locators");
-        System.clearProperty("gemfire.mcast-port");
+        System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE");
+        System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "locators");
+        System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + MCAST_PORT);
       }
     }
   }
@@ -130,7 +133,7 @@ public class JUnit4CacheTestCase extends JUnit4DistributedTestCase implements Ca
   public final Cache createLonerCache() {
     synchronized(JUnit4CacheTestCase.class) {
       try {
-        System.setProperty("gemfire.DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE", "true");
+        System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE", "true");
         Cache newCache = CacheFactory.create(getLonerSystem());
         cache = newCache;
       } catch (CacheExistsException e) {
@@ -142,7 +145,7 @@ public class JUnit4CacheTestCase extends JUnit4DistributedTestCase implements Ca
       } catch (Exception ex) {
         Assert.fail("Checked exception while initializing cache??", ex);
       } finally {
-        System.clearProperty("gemfire.DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE");
+        System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE");
       }
       return cache;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
index e5c3df7..06019b8 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
@@ -16,22 +16,6 @@
  */
 package com.gemstone.gemfire.test.dunit.internal;
 
-import static org.junit.Assert.*;
-
-import java.io.Serializable;
-import java.text.DecimalFormat;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-
 import com.gemstone.gemfire.admin.internal.AdminDistributedSystemImpl;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
@@ -47,11 +31,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionMessageObserver;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.admin.ClientStatsManager;
-import com.gemstone.gemfire.internal.cache.DiskStoreObserver;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.HARegion;
-import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
@@ -59,13 +39,22 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
-import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.Invoke;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 import com.gemstone.gemfire.test.junit.rules.serializable.SerializableTestName;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+
+import java.io.Serializable;
+import java.text.DecimalFormat;
+import java.util.*;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertNotNull;
 
 /**
  * This class is the base class for all distributed tests using JUnit 4.
@@ -249,8 +238,8 @@ public abstract class JUnit4DistributedTestCase implements DistributedTestFixtur
    */
   public final InternalDistributedSystem getLonerSystem() {
     Properties props = getDistributedSystemProperties();
-    props.put(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.put(DistributionConfig.LOCATORS_NAME, "");
+    props.put(MCAST_PORT, "0");
+    props.put(LOCATORS, "");
     return getSystem(props);
   }
 
@@ -565,7 +554,7 @@ public abstract class JUnit4DistributedTestCase implements DistributedTestFixtur
     Message.MAX_MESSAGE_SIZE = Message.DEFAULT_MAX_MESSAGE_SIZE;
 
     // clear system properties -- keep alphabetized
-    System.clearProperty("gemfire.log-level");
+    System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "log-level");
     System.clearProperty("jgroups.resolve_dns");
 
     if (InternalDistributedSystem.systemAttemptingReconnect != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
index 8249996..094095f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
@@ -16,43 +16,11 @@
  */
 package com.gemstone.gemfire.test.dunit.standalone;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.net.URISyntaxException;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.rmi.AccessException;
-import java.rmi.AlreadyBoundException;
-import java.rmi.NotBoundException;
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
-import java.rmi.server.UnicastRemoteObject;
-import java.util.List;
-import java.util.Properties;
-
-import com.gemstone.gemfire.distributed.internal.InternalLocator;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.logging.log4j.core.appender.FileAppender;
-import org.apache.logging.log4j.core.config.LoggerConfig;
-import org.apache.logging.log4j.core.layout.PatternLayout;
-import org.junit.Assert;
-
 import batterytest.greplogs.ExpectedStrings;
 import batterytest.greplogs.LogConsumer;
-import hydra.MethExecutorResult;
-
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -60,6 +28,30 @@ import com.gemstone.gemfire.test.dunit.DUnitEnv;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
+import hydra.MethExecutorResult;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.FileAppender;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.junit.Assert;
+
+import java.io.*;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.URISyntaxException;
+import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
+import java.rmi.*;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.List;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 /**
  * A class to build a fake test configuration and launch some DUnit VMS.
@@ -92,10 +84,10 @@ public class DUnitLauncher {
   public static final boolean LOCATOR_LOG_TO_DISK = Boolean.getBoolean("locatorLogToDisk");
 
   static final String MASTER_PARAM = "DUNIT_MASTER";
-  public static final String RMI_PORT_PARAM = "gemfire.DUnitLauncher.RMI_PORT";
-  static final String VM_NUM_PARAM = "gemfire.DUnitLauncher.VM_NUM";
+  public static final String RMI_PORT_PARAM = DistributionConfig.GEMFIRE_PREFIX + "DUnitLauncher.RMI_PORT";
+  static final String VM_NUM_PARAM = DistributionConfig.GEMFIRE_PREFIX + "DUnitLauncher.VM_NUM";
 
-  private static final String LAUNCHED_PROPERTY = "gemfire.DUnitLauncher.LAUNCHED";
+  private static final String LAUNCHED_PROPERTY = DistributionConfig.GEMFIRE_PREFIX + "DUnitLauncher.LAUNCHED";
 
   private static Master master;
 
@@ -218,11 +210,11 @@ public class DUnitLauncher {
   
   public static Properties getDistributedSystemProperties() {
     Properties p = new Properties();
-    p.setProperty("locators", getLocatorString());
-    p.setProperty("mcast-port", "0");
-    p.setProperty("enable-cluster-configuration", "false");
-    p.setProperty("use-cluster-configuration", "false");
-    p.setProperty("log-level", LOG_LEVEL);
+    p.setProperty(LOCATORS, getLocatorString());
+    p.setProperty(MCAST_PORT, "0");
+    p.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+    p.setProperty(DistributionConfig.USE_CLUSTER_CONFIGURATION_NAME, "false");
+    p.setProperty(DistributionConfig.LOG_LEVEL_NAME, LOG_LEVEL);
     return p;
   }
 
@@ -259,10 +251,10 @@ public class DUnitLauncher {
         Properties p = getDistributedSystemProperties();
         // I never want this locator to end up starting a jmx manager
         // since it is part of the unit test framework
-        p.setProperty("jmx-manager", "false");
+        p.setProperty(DistributionConfig.JMX_MANAGER_NAME, "false");
         //Disable the shared configuration on this locator.
         //Shared configuration tests create their own locator
-        p.setProperty("enable-cluster-configuration", "false");
+        p.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
         //Tell the locator it's the first in the system for
         //faster boot-up
         System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
@@ -295,7 +287,7 @@ public class DUnitLauncher {
     addSuspectFileAppender(workspaceDir);
     
     //Free off heap memory when disconnecting from the distributed system
-    System.setProperty("gemfire.free-off-heap-memory", "true");
+    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "free-off-heap-memory", "true");
     
     //indicate that this CM is controlled by the eclipse dunit.
     System.setProperty(LAUNCHED_PROPERTY, "true");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
index 1e78bff..797e96a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
@@ -16,12 +16,12 @@
  */
 package com.gemstone.gemfire.test.dunit.standalone;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintStream;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.internal.FileUtil;
+import org.apache.commons.io.FileUtils;
+
+import java.io.*;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
 import java.rmi.AccessException;
@@ -33,12 +33,6 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
-import com.gemstone.gemfire.distributed.internal.InternalLocator;
-import org.apache.commons.io.FileUtils;
-
-import com.gemstone.gemfire.internal.FileUtil;
-import com.gemstone.gemfire.internal.logging.LogService;
-
 /**
  *
  */
@@ -55,28 +49,28 @@ public class ProcessManager {
     this.namingPort = namingPort;
     this.registry = registry;
   }
-  
+
   public synchronized void launchVM(int vmNum) throws IOException {
-    if(processes.containsKey(vmNum)) {
+    if (processes.containsKey(vmNum)) {
       throw new IllegalStateException("VM " + vmNum + " is already running.");
     }
-    
+
     String[] cmd = buildJavaCommand(vmNum, namingPort);
     System.out.println("Executing " + Arrays.asList(cmd));
     File workingDir = getVMDir(vmNum);
     try {
       FileUtil.delete(workingDir);
-    } catch(IOException e) {
+    } catch (IOException e) {
       //This delete is occasionally failing on some platforms, maybe due to a lingering
       //process. Allow the process to be launched anyway.
-      System.err.println("Unable to delete " + workingDir + ". Currently contains " 
-                          + Arrays.asList(workingDir.list()));
+      System.err.println("Unable to delete " + workingDir + ". Currently contains "
+          + Arrays.asList(workingDir.list()));
     }
     workingDir.mkdirs();
     if (log4jConfig != null) {
       FileUtils.copyFileToDirectory(log4jConfig, workingDir);
     }
-    
+
     //TODO - delete directory contents, preferably with commons io FileUtils
     Process process = Runtime.getRuntime().exec(cmd, null, workingDir);
     pendingVMs++;
@@ -89,26 +83,26 @@ public class ProcessManager {
   public static File getVMDir(int vmNum) {
     return new File(DUnitLauncher.DUNIT_DIR, "vm" + vmNum);
   }
-  
+
   public synchronized void killVMs() {
-    for(ProcessHolder process : processes.values()) {
-      if(process != null) {
+    for (ProcessHolder process : processes.values()) {
+      if (process != null) {
         process.kill();
       }
     }
   }
-  
+
   public synchronized boolean hasLiveVMs() {
-    for(ProcessHolder process : processes.values()) {
-      if(process != null && process.isAlive()) {
+    for (ProcessHolder process : processes.values()) {
+      if (process != null && process.isAlive()) {
         return true;
       }
     }
     return false;
   }
-  
+
   public synchronized void bounce(int vmNum) {
-    if(!processes.containsKey(vmNum)) {
+    if (!processes.containsKey(vmNum)) {
       throw new IllegalStateException("No such process " + vmNum);
     }
     try {
@@ -120,15 +114,15 @@ public class ProcessManager {
       throw new RuntimeException("Unable to restart VM " + vmNum, e);
     }
   }
-   
+
   private void linkStreams(final int vmNum, final ProcessHolder holder, final InputStream in, final PrintStream out) {
     Thread ioTransport = new Thread() {
       public void run() {
         BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-        String vmName = (vmNum==-2)? "[locator]" : "[vm_"+vmNum+"]";
+        String vmName = (vmNum == -2) ? "[locator]" : "[vm_" + vmNum + "]";
         try {
           String line = reader.readLine();
-          while(line != null) {
+          while (line != null) {
             if (line.length() == 0) {
               out.println();
             } else {
@@ -137,8 +131,8 @@ public class ProcessManager {
             }
             line = reader.readLine();
           }
-        } catch(Exception e) {
-          if(!holder.isKilled()) {
+        } catch (Exception e) {
+          if (!holder.isKilled()) {
             out.println("Error transporting IO from child process");
             e.printStackTrace(out);
           }
@@ -151,7 +145,7 @@ public class ProcessManager {
   }
 
   private String[] buildJavaCommand(int vmNum, int namingPort) {
-    String cmd = System.getProperty( "java.home" ) + File.separator + "bin" + File.separator + "java";
+    String cmd = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
     String classPath = System.getProperty("java.class.path");
     //String tmpDir = System.getProperty("java.io.tmpdir");
     String agent = getAgentString();
@@ -165,7 +159,7 @@ public class ProcessManager {
     String jdkSuspend = vmNum == suspendVM ? "y" : "n";
     ArrayList<String> cmds = new ArrayList<String>();
     cmds.add(cmd);
-    cmds.add("-classpath"); 
+    cmds.add("-classpath");
     cmds.add(classPath);
     cmds.add("-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort);
     cmds.add("-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum);
@@ -174,15 +168,15 @@ public class ProcessManager {
       cmds.add("-D" + InternalLocator.INHIBIT_DM_BANNER + "=true");
     }
     cmds.add("-DlogLevel=" + DUnitLauncher.LOG_LEVEL);
-    if (DUnitLauncher.LOG4J!=null) {
-      cmds.add("-Dlog4j.configurationFile="+DUnitLauncher.LOG4J);
+    if (DUnitLauncher.LOG4J != null) {
+      cmds.add("-Dlog4j.configurationFile=" + DUnitLauncher.LOG4J);
     }
     cmds.add("-Djava.library.path=" + System.getProperty("java.library.path"));
     cmds.add("-Xrunjdwp:transport=dt_socket,server=y,suspend=" + jdkSuspend + jdkDebug);
     cmds.add("-XX:+HeapDumpOnOutOfMemoryError");
     cmds.add("-Xmx512m");
-    cmds.add("-Dgemfire.DEFAULT_MAX_OPLOG_SIZE=10");
-    cmds.add("-Dgemfire.disallowMcastDefaults=true");
+    cmds.add("-D" + DistributionConfig.GEMFIRE_PREFIX + "DEFAULT_MAX_OPLOG_SIZE=10");
+    cmds.add("-D" + DistributionConfig.GEMFIRE_PREFIX + "disallowMcastDefaults=true");
     cmds.add("-ea");
     cmds.add("-XX:+PrintGC");
     cmds.add("-XX:+PrintGCDetails");
@@ -191,10 +185,10 @@ public class ProcessManager {
     cmds.add(ChildVM.class.getName());
     String[] rst = new String[cmds.size()];
     cmds.toArray(rst);
-    
+
     return rst;
   }
-  
+
   /**
    * Get the java agent passed to this process and pass it to the child VMs.
    * This was added to support jacoco code coverage reports
@@ -202,8 +196,8 @@ public class ProcessManager {
   private String getAgentString() {
     RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
     if (runtimeBean != null) {
-      for(String arg: runtimeBean.getInputArguments()) {
-        if(arg.contains("-javaagent:")) {
+      for (String arg : runtimeBean.getInputArguments()) {
+        if (arg.contains("-javaagent:")) {
           //HACK for gradle bug  GRADLE-2859. Jacoco is passing a relative path
           //That won't work when we pass this to dunit VMs in a different 
           //directory
@@ -213,7 +207,7 @@ public class ProcessManager {
         }
       }
     }
-    
+
     return "-DdummyArg=true";
   }
 
@@ -221,24 +215,24 @@ public class ProcessManager {
     pendingVMs--;
     this.notifyAll();
   }
-  
+
   public synchronized boolean waitForVMs(long timeout) throws InterruptedException {
     long end = System.currentTimeMillis() + timeout;
-    while(pendingVMs > 0) {
+    while (pendingVMs > 0) {
       long remaining = end - System.currentTimeMillis();
-      if(remaining <= 0) {
+      if (remaining <= 0) {
         return false;
       }
       this.wait(remaining);
     }
-    
+
     return true;
   }
-  
+
   private static class ProcessHolder {
     private final Process process;
     private volatile boolean killed = false;
-    
+
     public ProcessHolder(Process process) {
       this.process = process;
     }
@@ -246,7 +240,7 @@ public class ProcessManager {
     public void kill() {
       this.killed = true;
       process.destroy();
-      
+
     }
 
     public Process getProcess() {
@@ -256,7 +250,7 @@ public class ProcessManager {
     public boolean isKilled() {
       return killed;
     }
-    
+
     public boolean isAlive() {
       return !killed && process.isAlive();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
index 113099d..ed6fe0c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
@@ -16,17 +16,16 @@
  */
 package com.gemstone.gemfire.test.dunit.tests;
 
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.test.dunit.Invoke.*;
-
-import java.util.Properties;
-
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.invokeInEveryVM;
 
 /**
  * Verifies that overriding {@code getDistributedSystemProperties} results
@@ -48,7 +47,7 @@ public class JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest extends JUnit4
   @Override
   public final Properties getDistributedSystemProperties() {
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     return props;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
index fb4b4fe..bde503b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
@@ -16,15 +16,12 @@
  */
 package com.gemstone.gemfire.test.dunit.tests;
 
-import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 
 import java.util.Properties;
 
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static com.gemstone.gemfire.test.dunit.Invoke.invokeInEveryVM;
 
 /**
  * Verifies that overriding {@code getDistributedSystemProperties} results
@@ -49,7 +46,7 @@ public class OverridingGetPropertiesDisconnectsAllDUnitTest extends DistributedT
   @Override
   public final Properties getDistributedSystemProperties() {
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     return props;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
index 16734b5..323c281 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
@@ -16,13 +16,6 @@
  */
 package com.gemstone.gemfire.test.fake;
 
-import static org.mockito.Mockito.*;
-
-import java.io.File;
-import java.net.UnknownHostException;
-
-import org.junit.Assert;
-
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.Statistics;
@@ -37,6 +30,11 @@ import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 
+import java.io.File;
+import java.net.UnknownHostException;
+
+import static org.mockito.Mockito.*;
+
 /**
  * Factory methods for fake objects for use in test.
  * 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
index 1ea5fe6..674437e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
@@ -16,16 +16,18 @@
  */
 package com.gemstone.gemfire.test.golden;
 
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.process.ProcessWrapper;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.junit.After;
+import org.junit.Before;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.junit.After;
-import org.junit.Before;
-
-import com.gemstone.gemfire.test.process.ProcessWrapper;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 /**
  * Test framework for launching processes and comparing output to expected golden output.
@@ -121,8 +123,8 @@ public abstract class GoldenTestCase {
 
   protected final Properties createProperties() {
     Properties properties = new Properties();
-    properties.setProperty("gemfire.mcast-port", "0");
-    properties.setProperty("gemfire.log-level", "warning");
+    properties.setProperty(DistributionConfig.GEMFIRE_PREFIX + MCAST_PORT, "0");
+    properties.setProperty(DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.LOG_LEVEL_NAME, "warning");
     properties.setProperty("file.encoding", "UTF-8");
     return editProperties(properties);
   }