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:47 UTC

[10/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/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
index 26cfdfa..38be211 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
@@ -16,25 +16,12 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-
-import org.junit.experimental.categories.Category;
-
-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.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.QueueStateImpl.SequenceIdAndExpirationObject;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.ClientServerObserver;
 import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
@@ -42,16 +29,16 @@ import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.ha.HAHelper;
 import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
 import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-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.NetworkUtils;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import org.junit.experimental.categories.Category;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 /**
  * Tests the reliable messaging functionality - Client sends a periodic
@@ -380,8 +367,8 @@ public class ReliableMessagingDUnitTest extends DistributedTestCase {
   {
     ReliableMessagingDUnitTest test = new ReliableMessagingDUnitTest("temp");
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
     cache = test.createCache(props);
     String host = NetworkUtils.getServerHostName(Host.getHost(0));
     PoolImpl p = (PoolImpl)PoolManager.createFactory()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UnregisterInterestDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UnregisterInterestDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UnregisterInterestDUnitTest.java
index b7980d4..ee88dea 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UnregisterInterestDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UnregisterInterestDUnitTest.java
@@ -19,14 +19,7 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.util.ArrayList;
-import java.util.Properties;
-
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
@@ -37,13 +30,13 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.FilterProfile;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.dunit.Assert;
-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.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.*;
+
+import java.util.ArrayList;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 /**
  */
@@ -313,7 +306,7 @@ public class UnregisterInterestDUnitTest extends DistributedTestCase {
     DistributedSystem ds = new UnregisterInterestDUnitTest("UnregisterInterestDUnitTest").getSystem();
     ds.disconnect();
     Properties props = new Properties();
-    props.setProperty("locators", "localhost["+DistributedTestUtils.getDUnitLocatorPort()+"]");
+    props.setProperty(LOCATORS, "localhost[" + DistributedTestUtils.getDUnitLocatorPort() + "]");
     CacheFactory cf = new CacheFactory(props);
     cache = cf.create();
     RegionFactory rf = ((GemFireCacheImpl)cache).createRegionFactory(RegionShortcut.REPLICATE);
@@ -329,8 +322,8 @@ public class UnregisterInterestDUnitTest extends DistributedTestCase {
     ds.disconnect();
 
     Properties props = new Properties();
-    props.setProperty("locators", "");
-    props.setProperty("mcast-port", "0");
+    props.setProperty(LOCATORS, "");
+    props.setProperty(MCAST_PORT, "0");
     ClientCacheFactory ccf = new ClientCacheFactory(props);
     ccf.setPoolSubscriptionEnabled(true);
     ccf.addPoolServer(host.getHostName(), port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
index 08d964a..0b91136 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
@@ -16,29 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import static junit.framework.TestCase.assertNotNull;
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.CacheWriterException;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
@@ -51,15 +29,21 @@ import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
-import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import com.jayway.awaitility.Awaitility;
-import com.jayway.awaitility.core.ConditionTimeoutException;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static junit.framework.TestCase.assertNotNull;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Start client 1
@@ -245,12 +229,12 @@ public class UpdatePropagationDUnitTest extends JUnit4CacheTestCase {
   {
     ClientCache cache;
     try {
-      System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "true");
+      System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.DISABLE_RANDOM", "true");
       int PORT1 = port1.intValue() ;
       int PORT2 = port2.intValue();
       Properties props = new Properties();
-      props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-      props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, "");
       ClientCacheFactory cf = new ClientCacheFactory();
       cf.addPoolServer(host, PORT1)
       .addPoolServer(host, PORT2)
@@ -262,7 +246,7 @@ public class UpdatePropagationDUnitTest extends JUnit4CacheTestCase {
       .setPoolPingInterval(300);
        cache = getClientCache(cf);
     } finally {
-      System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "false");
+      System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.DISABLE_RANDOM", "false");
       CacheServerTestUtil.enableShufflingOfEndpoints();
     }
     cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
index 639fb33..bb947d5 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
@@ -16,33 +16,23 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-
-import java.util.Properties;
-
-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.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.cache.client.*;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
+import com.gemstone.gemfire.test.dunit.*;
+
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.LOCATORS;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 /**
  * One Client , two servers.
@@ -160,8 +150,8 @@ public class VerifyUpdatesFromNonInterestEndPointDUnitTest extends DistributedTe
   {
     VerifyUpdatesFromNonInterestEndPointDUnitTest test = new VerifyUpdatesFromNonInterestEndPointDUnitTest("temp");
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
     cache = test.createCache(props);
     Pool p;
     try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
index 7da8d90..2da8391 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
@@ -16,43 +16,9 @@
  */
 package com.gemstone.gemfire.internal.cache.wan;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-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.Properties;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheClosedException;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.CacheLoader;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.Declarable;
-import com.gemstone.gemfire.cache.DiskStore;
-import com.gemstone.gemfire.cache.DiskStoreFactory;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.LoaderHelper;
-import com.gemstone.gemfire.cache.PartitionAttributesFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
@@ -77,15 +43,18 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.RegionQueue;
 import com.gemstone.gemfire.internal.cache.lru.Sizeable;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-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.Host;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.*;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.*;
 
 public class AsyncEventQueueTestBase extends DistributedTestCase {
 
@@ -153,11 +122,11 @@ public class AsyncEventQueueTestBase extends DistributedTestCase {
     AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(getTestMethodName());
     int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
     Properties props = test.getDistributedSystemProperties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     //props.setProperty(DistributionConfig.DISTRIBUTED_SYSTEM_ID_NAME, "" + dsId);
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + port
+    props.setProperty(LOCATORS, "localhost[" + port
         + "]");
-    props.setProperty(DistributionConfig.START_LOCATOR_NAME, "localhost["
+    props.setProperty(START_LOCATOR, "localhost["
         + port + "],server=true,peer=true,hostname-for-clients=localhost");
     test.getSystem(props);
     return port;
@@ -167,11 +136,11 @@ public class AsyncEventQueueTestBase extends DistributedTestCase {
     AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(getTestMethodName());
     int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
     Properties props = test.getDistributedSystemProperties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     props.setProperty(DistributionConfig.DISTRIBUTED_SYSTEM_ID_NAME, "" + dsId);
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + port
+    props.setProperty(LOCATORS, "localhost[" + port
         + "]");
-    props.setProperty(DistributionConfig.START_LOCATOR_NAME, "localhost["
+    props.setProperty(START_LOCATOR, "localhost["
         + port + "],server=true,peer=true,hostname-for-clients=localhost");
     props.setProperty(DistributionConfig.REMOTE_LOCATORS_NAME, "localhost["
         + remoteLocPort + "]");
@@ -686,8 +655,8 @@ public class AsyncEventQueueTestBase extends DistributedTestCase {
   protected static void createCache(Integer locPort) {
     AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(getTestMethodName());
     Properties props = test.getDistributedSystemProperties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "localhost[" + locPort
         + "]");
     InternalDistributedSystem ds = test.getSystem(props);
     cache = CacheFactory.create(ds);
@@ -696,7 +665,7 @@ public class AsyncEventQueueTestBase extends DistributedTestCase {
   public static void createCacheWithoutLocator(Integer mCastPort) {
     AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(getTestMethodName());
     Properties props = test.getDistributedSystemProperties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "" + mCastPort);
+    props.setProperty(MCAST_PORT, "" + mCastPort);
     InternalDistributedSystem ds = test.getSystem(props);
     cache = CacheFactory.create(ds);
   }
@@ -907,8 +876,8 @@ public class AsyncEventQueueTestBase extends DistributedTestCase {
   public static int createReceiver(int locPort) {
     AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(getTestMethodName());
     Properties props = test.getDistributedSystemProperties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "localhost[" + locPort
         + "]");
 
     InternalDistributedSystem ds = test.getSystem(props);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
index a099617..dbed7ad 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
@@ -16,24 +16,11 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
 
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.Declarable;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.wan.GatewayEventSubstitutionFilter;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueTestBase;
@@ -41,6 +28,15 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 public class AsyncEventListenerDUnitTest extends AsyncEventQueueTestBase {
 
@@ -57,7 +53,7 @@ public class AsyncEventListenerDUnitTest extends AsyncEventQueueTestBase {
   public void testCreateAsyncEventQueueWithNullListener() {
     AsyncEventQueueTestBase test = new AsyncEventQueueTestBase(getTestMethodName());
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     InternalDistributedSystem ds = test.getSystem(props);
     cache = CacheFactory.create(ds);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
index ddcc9cb..19cd517 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
@@ -15,26 +15,22 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.*;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
 import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
 import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueConfigurationException;
-import com.gemstone.gemfire.internal.cache.wan.MyAsyncEventListener;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  *
@@ -43,10 +39,10 @@ import junit.framework.TestCase;
 public class AsyncEventQueueValidationsJUnitTest {
 
   private Cache cache;
-  
+
   @Test
   public void testConcurrentParallelAsyncEventQueueAttributesWrongDispatcherThreads() {
-    cache = new CacheFactory().set("mcast-port", "0").create();
+    cache = new CacheFactory().set(MCAST_PORT, "0").create();
     try {
       AsyncEventQueueFactory fact = cache.createAsyncEventQueueFactory();
       fact.setParallel(true);
@@ -55,15 +51,14 @@ public class AsyncEventQueueValidationsJUnitTest {
       fact.create("id", new com.gemstone.gemfire.internal.cache.wan.MyAsyncEventListener());
       fail("Expected AsyncEventQueueConfigurationException.");
     } catch (AsyncEventQueueConfigurationException e) {
-        assertTrue(e.getMessage()
-            .contains(" can not be created with dispatcher threads less than 1"));
+      assertTrue(e.getMessage()
+          .contains(" can not be created with dispatcher threads less than 1"));
     }
   }
-  
-  
+
   @Test
   public void testConcurrentParallelAsyncEventQueueAttributesOrderPolicyThread() {
-    cache = new CacheFactory().set("mcast-port", "0").create();
+    cache = new CacheFactory().set(MCAST_PORT, "0").create();
     try {
       AsyncEventQueueFactory fact = cache.createAsyncEventQueueFactory();
       fact.setParallel(true);
@@ -72,10 +67,9 @@ public class AsyncEventQueueValidationsJUnitTest {
       fact.create("id", new com.gemstone.gemfire.internal.cache.wan.MyAsyncEventListener());
       fail("Expected AsyncEventQueueConfigurationException.");
     } catch (AsyncEventQueueConfigurationException e) {
-        assertTrue(e.getMessage()
-            .contains("can not be created with OrderPolicy"));
+      assertTrue(e.getMessage()
+          .contains("can not be created with OrderPolicy"));
     }
   }
-  
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
index 5aa9278..11be69d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheConfigDUnitTest.java
@@ -16,22 +16,18 @@
  */
 package com.gemstone.gemfire.internal.compression;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.Properties;
-
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.SerializableCallable;
-import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Properties;
 
 /**
  * Tests configured and badly configured cache.xml files with regards to compression.
@@ -127,7 +123,7 @@ public class CompressionCacheConfigDUnitTest extends CacheTestCase {
         try {
           disconnectFromDS();
           Properties props = new Properties();
-          props.setProperty("cache-xml-file",cacheXml);
+          props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, cacheXml);
           LogWriterUtils.getLogWriter().info("<ExpectedException action=add>ClassNotFoundException</ExpectedException>");
           getSystem(props);
           assertNotNull(getCache());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
index 0100a05..6f64bd1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
@@ -16,14 +16,14 @@
  */
 package com.gemstone.gemfire.internal.compression;
 
-import java.util.Properties;
-
 import com.gemstone.gemfire.compression.SnappyCompressor;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.OffHeapTestUtil;
 import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 
+import java.util.Properties;
+
 @SuppressWarnings("serial")
 public class CompressionCacheListenerOffHeapDUnitTest extends
     CompressionCacheListenerDUnitTest {
@@ -33,10 +33,10 @@ public class CompressionCacheListenerOffHeapDUnitTest extends
   }
   
   public static void caseSetUp() {
-    System.setProperty("gemfire.trackOffHeapRefCounts", "true");
+    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "trackOffHeapRefCounts", "true");
   }
   public static void caseTearDown() {
-    System.clearProperty("gemfire.trackOffHeapRefCounts");
+    System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "trackOffHeapRefCounts");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
index 0cc3bdf..efd8e57 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
@@ -16,8 +16,6 @@
  */
 package com.gemstone.gemfire.internal.compression;
 
-import java.util.Properties;
-
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -25,6 +23,8 @@ import com.gemstone.gemfire.internal.cache.OffHeapTestUtil;
 import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 
+import java.util.Properties;
+
 public class CompressionRegionOperationsOffHeapDUnitTest extends
     CompressionRegionOperationsDUnitTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCacheJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCacheJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCacheJUnitTest.java
index f3ca340..582c5e3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCacheJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/AbstractPoolCacheJUnitTest.java
@@ -22,32 +22,28 @@
  */
 package com.gemstone.gemfire.internal.datasource;
 
-import java.sql.Connection;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import junit.framework.TestCase;
-
-import javax.naming.Context;
-import javax.sql.PooledConnection;
-import javax.sql.XAConnection;
-import javax.transaction.xa.XAResource;
-
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
+import javax.naming.Context;
+import javax.sql.PooledConnection;
+import javax.sql.XAConnection;
+import javax.transaction.xa.XAResource;
+import java.sql.Connection;
+import java.util.*;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.util.test.TestUtil;
-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.assertNotNull;
+import static org.junit.Assert.fail;
 
 /**
  */
@@ -61,11 +57,11 @@ public class AbstractPoolCacheJUnitTest {
   @Before
   public void setUp() throws Exception {
     props = new Properties();
-    props.setProperty("log-level", "info");
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
+    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
     String path = TestUtil.getResourcePath(AbstractPoolCacheJUnitTest.class, "/jta/cachejta.xml");
-    props.setProperty("cache-xml-file", path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/CleanUpJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/CleanUpJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/CleanUpJUnitTest.java
index 95d6224..ece05b3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/CleanUpJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/CleanUpJUnitTest.java
@@ -19,24 +19,25 @@
  */
 package com.gemstone.gemfire.internal.datasource;
 
-import static org.junit.Assert.fail;
-
-import java.sql.Connection;
-import java.util.Properties;
-
-import javax.naming.Context;
-
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import javax.naming.Context;
+import java.sql.Connection;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.fail;
+
 //import javax.sql.PooledConnection;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.util.test.TestUtil;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * 
@@ -53,9 +54,9 @@ public class CleanUpJUnitTest {
   @Before
   public void setUp() {
     props = new Properties();
-    props.setProperty("mcast-port","0");
+    props.setProperty(MCAST_PORT, "0");
     String path = TestUtil.getResourcePath(CleanUpJUnitTest.class, "/jta/cachejta.xml");
-    props.setProperty("cache-xml-file", path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolCacheImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolCacheImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolCacheImplJUnitTest.java
index c63466c..b961d8d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolCacheImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolCacheImplJUnitTest.java
@@ -21,26 +21,26 @@
  */
 package com.gemstone.gemfire.internal.datasource;
 
-import static org.junit.Assert.fail;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+import javax.naming.Context;
+import javax.sql.PooledConnection;
 import java.sql.Connection;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import javax.naming.Context;
-import javax.sql.PooledConnection;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-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.util.test.TestUtil;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.fail;
 
 /*
  * this is a Multithreaded test for datasource connection pool.
@@ -63,9 +63,9 @@ public class ConnectionPoolCacheImplJUnitTest {
   public void setUp() {
     try {
       props = new Properties();
-      props.setProperty("mcast-port", "0");
+      props.setProperty(MCAST_PORT, "0");
       String path = TestUtil.getResourcePath(ConnectionPoolCacheImplJUnitTest.class, "/jta/cachejta.xml");
-      props.setProperty("cache-xml-file", path);
+      props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
       ds1 = DistributedSystem.connect(props);
       cache = CacheFactory.create(ds1);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolingJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolingJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolingJUnitTest.java
index e9ef04b..d2a377a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolingJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/ConnectionPoolingJUnitTest.java
@@ -16,9 +16,24 @@
  */
 package com.gemstone.gemfire.internal.datasource;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.sql.DataSource;
+import javax.transaction.UserTransaction;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -26,25 +41,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import javax.naming.Context;
-import javax.naming.NamingException;
-import javax.sql.DataSource;
-import javax.transaction.UserTransaction;
-
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-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.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.util.test.TestUtil;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 /**
  * 
@@ -68,9 +67,9 @@ public class ConnectionPoolingJUnitTest {
   public void setUp() throws Exception {
     encounteredException = false;
     props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     String path = TestUtil.getResourcePath(ConnectionPoolingJUnitTest.class, "/jta/cachejta.xml");
-    props.setProperty("cache-xml-file", path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/DataSourceFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/DataSourceFactoryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/DataSourceFactoryJUnitTest.java
index 783cda3..9384d43 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/DataSourceFactoryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/DataSourceFactoryJUnitTest.java
@@ -16,25 +16,23 @@
  */
 package com.gemstone.gemfire.internal.datasource;
 
-import java.sql.Connection;
-import java.util.Properties;
-
-import javax.naming.Context;
-
-import junit.framework.TestCase;
-
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
+import javax.naming.Context;
+import java.sql.Connection;
+import java.util.Properties;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.util.test.TestUtil;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.fail;
 
 /*
  *  
@@ -49,9 +47,9 @@ public class DataSourceFactoryJUnitTest {
   @Before
   public void setUp() {
     props = new Properties();
-    props.setProperty("mcast-port","0");
+    props.setProperty(MCAST_PORT, "0");
     String path = TestUtil.getResourcePath(DataSourceFactoryJUnitTest.class, "/jta/cachejta.xml");
-    props.setProperty("cache-xml-file",path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/RestartJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/RestartJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/RestartJUnitTest.java
index 6f6dacf..7d5aba3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/RestartJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/datasource/RestartJUnitTest.java
@@ -19,23 +19,24 @@
  */
 package com.gemstone.gemfire.internal.datasource;
 
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.fail;
-
-import java.util.Properties;
-
-//import com.gemstone.gemfire.internal.jta.CacheUtils;
-import javax.transaction.TransactionManager;
-
-import org.junit.Test;
-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.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.jndi.JNDIInvoker;
-import com.gemstone.gemfire.util.test.TestUtil;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import javax.transaction.TransactionManager;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.fail;
+
+//import com.gemstone.gemfire.internal.jta.CacheUtils;
 
 /**
  * 
@@ -57,9 +58,9 @@ public class RestartJUnitTest {
 	TransactionManager tm2 = null;
     try{
     props = new Properties();
-    props.setProperty("mcast-port","0");
+      props.setProperty(MCAST_PORT, "0");
     String path = TestUtil.getResourcePath(RestartJUnitTest.class, "/jta/cachejta.xml");
-    props.setProperty("cache-xml-file",path);
+      props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
 
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/BlockingTimeOutJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/BlockingTimeOutJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/BlockingTimeOutJUnitTest.java
index 64afaf2..4ee947b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/BlockingTimeOutJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/BlockingTimeOutJUnitTest.java
@@ -16,37 +16,33 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import static org.junit.Assert.assertNotNull;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.OutputStreamWriter;
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.sql.DataSource;
+import java.io.*;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Properties;
 import java.util.Random;
 
-import javax.naming.Context;
-import javax.naming.NamingException;
-import javax.sql.DataSource;
-
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-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.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.util.test.TestUtil;
-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.assertNotNull;
 
 @Category(IntegrationTest.class)
 public class BlockingTimeOutJUnitTest {
@@ -147,9 +143,9 @@ public class BlockingTimeOutJUnitTest {
     wr.write(modified_file_str);
     wr.flush();
     wr.close();
-    props.setProperty("cache-xml-file", path);
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
     String tableName = "";
     cache = new CacheFactory(props).create();
     if (className != null && !className.equals("")) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/CacheUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/CacheUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/CacheUtils.java
index 29a88da..121b9aa 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/CacheUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/CacheUtils.java
@@ -24,22 +24,27 @@ package com.gemstone.gemfire.internal.jta;
 
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.LogWriter;
-//import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-//import com.gemstone.gemfire.cache.Region;
-//import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.query.QueryService;
 import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.util.test.TestUtil;
 
-import java.util.Properties;
-import java.sql.*;
-
-import com.gemstone.gemfire.cache.query.QueryService;
-
 import javax.naming.Context;
 import javax.naming.NamingException;
 import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+
+//import com.gemstone.gemfire.cache.AttributesFactory;
+//import com.gemstone.gemfire.cache.Region;
+//import com.gemstone.gemfire.cache.RegionAttributes;
 
 /**
  * 
@@ -60,9 +65,9 @@ public class CacheUtils {
 
   public static String init(String className) throws Exception{
     Properties props = new Properties();
-    props.setProperty("cache-xml-file",TestUtil.getResourcePath(CacheUtils.class, "cachejta.xml"));
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, TestUtil.getResourcePath(CacheUtils.class, "cachejta.xml"));
     String tableName = "";
-    props.setProperty("mcast-port", "0");
+    props.setProperty(MCAST_PORT, "0");
     
     ds = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/DataSourceJTAJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/DataSourceJTAJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/DataSourceJTAJUnitTest.java
index abe5a51..6b9b228 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/DataSourceJTAJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/DataSourceJTAJUnitTest.java
@@ -16,29 +16,29 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Properties;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import javax.naming.Context;
 import javax.naming.NamingException;
 import javax.sql.DataSource;
 import javax.transaction.UserTransaction;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.jta.CacheUtils;
-import com.gemstone.gemfire.internal.jta.JTAUtils;
-import com.gemstone.gemfire.util.test.TestUtil;
-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;
 
 /**
  * This test case is to test the following test scenarios: 1) Get Simple DS
@@ -93,7 +93,7 @@ public class DataSourceJTAJUnitTest {
     try {
       Properties props = new Properties();
       String path= TestUtil.getResourcePath(CacheUtils.class, "cachejta.xml");
-      props.setProperty("cache-xml-file", path);
+      props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
       ds = connect(props);
       tableName = CacheUtils.init(ds, "JTATest");
       // System.out.println ("Table name: " + tableName);
@@ -274,7 +274,7 @@ public class DataSourceJTAJUnitTest {
     try {
       Properties props = new Properties();
       String path= TestUtil.getResourcePath(CacheUtils.class, "cachejta.xml");
-      props.setProperty("cache-xml-file", path);
+      props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
       ds = connect(props);
       tableName = CacheUtils.init(ds, "JTATest");
       System.out.println("Table name: " + tableName);
@@ -585,7 +585,7 @@ public class DataSourceJTAJUnitTest {
     try {
       Properties props = new Properties();
       String path= TestUtil.getResourcePath(CacheUtils.class, "cachejta.xml");
-      props.setProperty("cache-xml-file", path);
+      props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
       ds = connect(props);
       tableName = CacheUtils.init(ds, "JTATest");
       // System.out.println ("Table name: " + tableName);
@@ -749,7 +749,7 @@ public class DataSourceJTAJUnitTest {
     try {
       Properties props = new Properties();
       String path= TestUtil.getResourcePath(CacheUtils.class, "cachejta.xml");
-      props.setProperty("cache-xml-file", path);
+      props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
       ds = connect(props);
       tableName = CacheUtils.init(ds, "JTATest");
       // System.out.println ("Table name: " + tableName);
@@ -1008,8 +1008,8 @@ public class DataSourceJTAJUnitTest {
    */
 
   private static DistributedSystem connect(Properties props) {
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
     return DistributedSystem.connect(props);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/ExceptionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/ExceptionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/ExceptionJUnitTest.java
index 4ff2896..78426c6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/ExceptionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/ExceptionJUnitTest.java
@@ -16,24 +16,16 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import static org.junit.Assert.*;
-
-import java.util.Properties;
-
-import junit.framework.TestCase;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.*;
+import org.junit.experimental.categories.Category;
 
 import javax.transaction.*;
+import java.util.Properties;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.fail;
 
 /**
  * Check if the correct expectations are being thrown when they are supposed to.
@@ -50,7 +42,7 @@ public class ExceptionJUnitTest {
   @BeforeClass
   public static void beforeClass() throws Exception {
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     ds = DistributedSystem.connect(props);
     tm = TransactionManagerImpl.getTransactionManager();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/GlobalTransactionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/GlobalTransactionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/GlobalTransactionJUnitTest.java
index a3c47e5..b78cdf7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/GlobalTransactionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/GlobalTransactionJUnitTest.java
@@ -16,31 +16,24 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import junit.framework.TestCase;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.util.Properties;
-
-import javax.naming.Context;
-import javax.transaction.RollbackException;
-import javax.transaction.Synchronization;
-import javax.transaction.Transaction;
-import javax.transaction.TransactionManager;
-import javax.transaction.UserTransaction;
-import javax.transaction.Status;
-
-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.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.datasource.GemFireBasicDataSource;
 import com.gemstone.gemfire.internal.datasource.GemFireTransactionDataSource;
-import com.gemstone.gemfire.internal.datasource.RestartJUnitTest;
-import com.gemstone.gemfire.util.test.TestUtil;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import junit.framework.TestCase;
+import org.junit.experimental.categories.Category;
+
+import javax.naming.Context;
+import javax.transaction.*;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 @Category(IntegrationTest.class)
 public class GlobalTransactionJUnitTest extends TestCase {
@@ -54,9 +47,9 @@ public class GlobalTransactionJUnitTest extends TestCase {
   @Override
   protected void setUp() throws Exception {
     props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     String path = TestUtil.getResourcePath(GlobalTransactionJUnitTest.class, "/jta/cachejta.xml");
-    props.setProperty("cache-xml-file", path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);
     utx = new UserTransactionImpl();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/JtaIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/JtaIntegrationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/JtaIntegrationJUnitTest.java
index e3feb01..24b7aee 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/JtaIntegrationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/JtaIntegrationJUnitTest.java
@@ -16,27 +16,20 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-import javax.transaction.Transaction;
-import javax.transaction.TransactionManager;
-
+import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.apache.logging.log4j.Logger;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import javax.transaction.Transaction;
+import javax.transaction.TransactionManager;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.*;
 
 /**
  * Moved some non-DUnit tests over from com/gemstone/gemfire/internal/jta/dunit/JTADUnitTest
@@ -58,7 +51,7 @@ public class JtaIntegrationJUnitTest {
   @Test
   public void testBug43987() {
     //InternalDistributedSystem ds = getSystem(); // ties us in to the DS owned by DistributedTestCase.
-    CacheFactory cf = new CacheFactory().set("mcast-port", "0");//(ds.getProperties());
+    CacheFactory cf = new CacheFactory().set(MCAST_PORT, "0");//(ds.getProperties());
     Cache cache = cf.create(); // should just reuse the singleton DS owned by DistributedTestCase.
     RegionFactory<String, String> rf = cache.createRegionFactory(RegionShortcut.REPLICATE);
     Region<String, String> r = rf.create("JTA_reg");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionImplJUnitTest.java
index f5bb3b5..33df027 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionImplJUnitTest.java
@@ -20,23 +20,17 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import static org.junit.Assert.assertTrue;
-
-import java.util.Properties;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.*;
+import org.junit.experimental.categories.Category;
 
 import javax.transaction.Synchronization;
 import javax.transaction.UserTransaction;
+import java.util.Properties;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertTrue;
 
 /**
  */
@@ -51,7 +45,7 @@ public class TransactionImplJUnitTest {
   @BeforeClass
   public static void beforeClass() throws Exception {
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     ds = DistributedSystem.connect(props);
     tm = TransactionManagerImpl.getTransactionManager();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionManagerImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionManagerImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionManagerImplJUnitTest.java
index 5f7be9f..fa91ce9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionManagerImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionManagerImplJUnitTest.java
@@ -16,31 +16,19 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Properties;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.*;
+import org.junit.experimental.categories.Category;
 
-import javax.transaction.RollbackException;
-import javax.transaction.Status;
-import javax.transaction.Synchronization;
-import javax.transaction.SystemException;
-import javax.transaction.Transaction;
-import javax.transaction.UserTransaction;
+import javax.transaction.*;
 import javax.transaction.xa.XAException;
 import javax.transaction.xa.XAResource;
 import javax.transaction.xa.Xid;
+import java.util.Properties;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.*;
 
 /**
  * Test TransactionManagerImpl methods not tested by UserTransactionImplTest
@@ -57,7 +45,7 @@ public class TransactionManagerImplJUnitTest {
   @BeforeClass
   public static void beforeClass() throws Exception{
     Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
+    props.setProperty(MCAST_PORT, "0");
     ds = DistributedSystem.connect(props);
     tm = TransactionManagerImpl.getTransactionManager();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionTimeOutJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionTimeOutJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionTimeOutJUnitTest.java
index 6148bef..2bd6790 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionTimeOutJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/TransactionTimeOutJUnitTest.java
@@ -16,36 +16,31 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.datasource.GemFireTransactionDataSource;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import junit.framework.TestCase;
+import org.junit.FixMethodOrder;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.MethodSorters;
+
+import javax.naming.Context;
+import javax.sql.DataSource;
+import javax.transaction.UserTransaction;
+import java.io.*;
 import java.sql.Connection;
 import java.sql.ResultSet;
-//import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Properties;
 import java.util.Random;
 
-import javax.naming.Context;
-import javax.sql.DataSource;
-import javax.transaction.UserTransaction;
-
-import junit.framework.TestCase;
-
-import org.junit.FixMethodOrder;
-import org.junit.experimental.categories.Category;
-import org.junit.runners.MethodSorters;
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.internal.datasource.GemFireTransactionDataSource;
-import com.gemstone.gemfire.util.test.TestUtil;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+//import java.sql.SQLException;
 
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
 @Category(IntegrationTest.class)
@@ -58,7 +53,7 @@ public class TransactionTimeOutJUnitTest extends TestCase {
   protected void setUp() throws Exception {
     super.setUp();
     props = new Properties();
-    props.setProperty("mcast-port", "0");
+    props.setProperty(MCAST_PORT, "0");
     int pid = new Random().nextInt();
     File tmpFile = File.createTempFile("dunit-cachejta_", ".xml");
     tmpFile.deleteOnExit();
@@ -70,8 +65,8 @@ public class TransactionTimeOutJUnitTest extends TestCase {
     wr.write(modified_file_str);
     wr.flush();
     wr.close();
-    props.setProperty("cache-xml-file", path);
-    // props.setProperty("cache-xml-file","D:\\projects\\JTA\\cachejta.xml");
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
+    // props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME,"D:\\projects\\JTA\\cachejta.xml");
     ds1 = DistributedSystem.connect(props);
     cache = CacheFactory.create(ds1);
     cache.getLogger().fine("SWAP:running test:"+getName());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/UserTransactionImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/UserTransactionImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/UserTransactionImplJUnitTest.java
index cae89a3..6f78b0e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/UserTransactionImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/UserTransactionImplJUnitTest.java
@@ -16,26 +16,21 @@
  */
 package com.gemstone.gemfire.internal.jta;
 
-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.util.Properties;
-
-import javax.transaction.Status;
-import javax.transaction.Transaction;
-import javax.transaction.UserTransaction;
-
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import javax.transaction.Status;
+import javax.transaction.Transaction;
+import javax.transaction.UserTransaction;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.*;
 
 /**
  */
@@ -50,7 +45,7 @@ public class UserTransactionImplJUnitTest {
   @BeforeClass
   public static void beforeClass() throws Exception {
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(MCAST_PORT, "0");
     ds = DistributedSystem.connect(props);
     tm = TransactionManagerImpl.getTransactionManager();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/ExceptionsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/ExceptionsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/ExceptionsDUnitTest.java
index e94ddf6..874a04d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/ExceptionsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/ExceptionsDUnitTest.java
@@ -16,24 +16,10 @@
  */
 package com.gemstone.gemfire.internal.jta.dunit;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.sql.SQLException;
-import java.util.Properties;
-
-import javax.naming.Context;
-import javax.naming.NamingException;
-import javax.sql.DataSource;
-import javax.transaction.UserTransaction;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.jta.CacheUtils;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
@@ -42,6 +28,13 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.util.test.TestUtil;
 
+import javax.naming.Context;
+import javax.sql.DataSource;
+import javax.transaction.UserTransaction;
+import java.io.*;
+import java.sql.SQLException;
+import java.util.Properties;
+
 public class ExceptionsDUnitTest extends DistributedTestCase {
 
   static DistributedSystem ds;
@@ -132,9 +125,9 @@ public class ExceptionsDUnitTest extends DistributedTestCase {
     wr.write(modified_file_str);
     wr.flush();
     wr.close();
-    props.setProperty("cache-xml-file", path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
 //    String tableName = "";
-    //		  props.setProperty("mcast-port", "10339");
+    //		  props.setProperty(DistributionConfig.SystemConfigurationProperties.MCAST_PORT, "10339");
     try {
       //			   ds = DistributedSystem.connect(props);
       ds = (new ExceptionsDUnitTest("temp")).getSystem(props);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/IdleTimeOutDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/IdleTimeOutDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/IdleTimeOutDUnitTest.java
index c66f57a..9dd4dbe 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/IdleTimeOutDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/IdleTimeOutDUnitTest.java
@@ -16,36 +16,24 @@
  */
 package com.gemstone.gemfire.internal.jta.dunit;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Properties;
-
-import javax.naming.Context;
-import javax.naming.NamingException;
-import javax.sql.DataSource;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.jta.CacheUtils;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.ThreadUtils;
-import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.util.test.TestUtil;
 
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.sql.DataSource;
+import java.io.*;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
 public class IdleTimeOutDUnitTest extends DistributedTestCase {
 
   static DistributedSystem ds;
@@ -140,9 +128,9 @@ public class IdleTimeOutDUnitTest extends DistributedTestCase {
     wr.write(modified_file_str);
     wr.flush();
     wr.close();
-    props.setProperty("cache-xml-file", path);
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
     String tableName = "";
-    //	        props.setProperty("mcast-port", "10339");
+    //	        props.setProperty(DistributionConfig.SystemConfigurationProperties.MCAST_PORT, "10339");
     try {
       //	  	      ds = DistributedSystem.connect(props);
       ds = (new IdleTimeOutDUnitTest("temp")).getSystem(props);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/LoginTimeOutDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/LoginTimeOutDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/LoginTimeOutDUnitTest.java
index fb617d9..644809f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/LoginTimeOutDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jta/dunit/LoginTimeOutDUnitTest.java
@@ -16,40 +16,27 @@
  */
 package com.gemstone.gemfire.internal.jta.dunit;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Properties;
-
-import javax.naming.Context;
-import javax.naming.NamingException;
-import javax.sql.DataSource;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.jta.CacheUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.RMIException;
-import com.gemstone.gemfire.test.dunit.ThreadUtils;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.util.test.TestUtil;
+import org.apache.logging.log4j.Logger;
+
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.sql.DataSource;
+import java.io.*;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.distributed.SystemConfigurationProperties.MCAST_PORT;
 
 public class LoginTimeOutDUnitTest extends DistributedTestCase {
   private static final Logger logger = LogService.getLogger();
@@ -145,8 +132,8 @@ public class LoginTimeOutDUnitTest extends DistributedTestCase {
     wr.write(modified_file_str);
     wr.flush();
     wr.close();
-    props.setProperty("cache-xml-file", path);
-    props.setProperty("mcast-port", "0");
+    props.setProperty(DistributionConfig.CACHE_XML_FILE_NAME, path);
+    props.setProperty(MCAST_PORT, "0");
     String tableName = "";
     cache = new CacheFactory(props).create();
     if (className != null && !className.equals("")) {