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:05:07 UTC

[30/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/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
index a4cf7e9..c5746ed 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
@@ -16,80 +16,23 @@
  */
 package com.gemstone.gemfire.internal.cache.ha;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelCriterion;
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.InternalGemFireException;
-import com.gemstone.gemfire.StatisticsFactory;
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.CacheListener;
-import com.gemstone.gemfire.cache.CacheWriterException;
-import com.gemstone.gemfire.cache.CustomExpiry;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.EntryNotFoundException;
-import com.gemstone.gemfire.cache.ExpirationAction;
-import com.gemstone.gemfire.cache.ExpirationAttributes;
-import com.gemstone.gemfire.cache.MirrorType;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.TimeoutException;
-import com.gemstone.gemfire.cache.query.CqQuery;
+import com.gemstone.gemfire.*;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.query.internal.CqQueryVsdStats;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.CacheServerImpl;
-import com.gemstone.gemfire.internal.cache.Conflatable;
-import com.gemstone.gemfire.internal.cache.EventID;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.HARegion;
-import com.gemstone.gemfire.internal.cache.RegionQueue;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientMarkerMessageImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientUpdateMessage;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientUpdateMessageImpl;
+import com.gemstone.gemfire.internal.cache.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.*;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientUpdateMessageImpl.CqNameToOp;
-import com.gemstone.gemfire.internal.cache.tier.sockets.HAEventWrapper;
-import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
@@ -98,7 +41,18 @@ import com.gemstone.gemfire.internal.util.concurrent.StoppableCondition;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantReadWriteLock;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantReadWriteLock.StoppableWriteLock;
-import com.gemstone.gemfire.i18n.StringId;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * An implementation of Queue using Gemfire Region as the underlying
@@ -268,7 +222,7 @@ public class HARegionQueue implements RegionQueue
   /**
    * Constant used to set region entry expiry time using system property.
    */
-  public static final String REGION_ENTRY_EXPIRY_TIME = "gemfire.MessageTimeToLive";
+  public static final String REGION_ENTRY_EXPIRY_TIME = DistributionConfig.GEMFIRE_PREFIX + "MessageTimeToLive";
   
   /**
    * The default frequency (in seconds) at which a message will be sent by the

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueAttributes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueAttributes.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueAttributes.java
index c0af8e2..c2ff1a0 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueAttributes.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueAttributes.java
@@ -19,6 +19,8 @@
  */
 package com.gemstone.gemfire.internal.cache.ha;
 
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+
 /**
  * 
  * This class defines the user specified attributes of the HARegion which are
@@ -38,7 +40,7 @@ public class HARegionQueueAttributes
   /**
    * String storing the System Property key representing the blocking queue capacity 
    */
-  private static final String BLOCKING_QUEUE_CAPACITY = "gemfire.Capacity";
+  private static final String BLOCKING_QUEUE_CAPACITY = DistributionConfig.GEMFIRE_PREFIX + "Capacity";
 
   /**
    * expiry time for region entries in seconds

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapEvictor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapEvictor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapEvictor.java
index 6aac4a9..b4bba96 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapEvictor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapEvictor.java
@@ -16,41 +16,25 @@
  */
 package com.gemstone.gemfire.internal.cache.lru;
 
-import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.OverflowQueueWithDMStats;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.RegionEvictorTask;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.control.HeapMemoryMonitor;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager.ResourceType;
 import com.gemstone.gemfire.internal.cache.control.MemoryEvent;
 import com.gemstone.gemfire.internal.cache.control.ResourceListener;
-import com.gemstone.gemfire.internal.lang.ThreadUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
+import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
+import org.apache.logging.log4j.Logger;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Triggers centralized eviction(asynchronously) when the ResourceManager sends
@@ -65,23 +49,23 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
 
   // Add 1 for the management task that's putting more eviction tasks on the queue
   public static final int MAX_EVICTOR_THREADS = Integer.getInteger(
-      "gemfire.HeapLRUCapacityController.MAX_EVICTOR_THREADS", (Runtime.getRuntime().availableProcessors()*4)) + 1;
+      DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.MAX_EVICTOR_THREADS", (Runtime.getRuntime().availableProcessors() * 4)) + 1;
 
   public static final boolean DISABLE_HEAP_EVICTIOR_THREAD_POOL = Boolean
-      .getBoolean("gemfire.HeapLRUCapacityController.DISABLE_HEAP_EVICTIOR_THREAD_POOL");
+      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.DISABLE_HEAP_EVICTIOR_THREAD_POOL");
 
   public static final boolean EVICT_HIGH_ENTRY_COUNT_BUCKETS_FIRST = Boolean.valueOf(
       System.getProperty(
-          "gemfire.HeapLRUCapacityController.evictHighEntryCountBucketsFirst",
+          DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.evictHighEntryCountBucketsFirst",
           "true")).booleanValue(); 
 
   public static final int MINIMUM_ENTRIES_PER_BUCKET = Integer
-  .getInteger("gemfire.HeapLRUCapacityController.inlineEvictionThreshold",0);
+      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.inlineEvictionThreshold", 0);
   
   public static final long TOTAL_BYTES_TO_EVICT_FROM_HEAP; 
   
   public static final int BUCKET_SORTING_INTERVAL = Integer.getInteger(
-      "gemfire.HeapLRUCapacityController.higherEntryCountBucketCalculationInterval",
+      DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.higherEntryCountBucketCalculationInterval",
       100).intValue();
   
   private static final String EVICTOR_THREAD_GROUP_NAME = "EvictorThreadGroup";
@@ -90,7 +74,7 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
 
   static {
     float evictionBurstPercentage = Float.parseFloat(System.getProperty(
-        "gemfire.HeapLRUCapacityController.evictionBurstPercentage", "0.4"));
+        DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.evictionBurstPercentage", "0.4"));
     long maxTenuredBytes = HeapMemoryMonitor.getTenuredPoolMaxMemory();
     TOTAL_BYTES_TO_EVICT_FROM_HEAP = (long)(maxTenuredBytes * 0.01 * evictionBurstPercentage);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapLRUCapacityController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapLRUCapacityController.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapLRUCapacityController.java
index 948a9d7..7bf2d1f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapLRUCapacityController.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/HeapLRUCapacityController.java
@@ -16,8 +16,6 @@
  */
 package com.gemstone.gemfire.internal.cache.lru;
 
-import java.util.Properties;
-
 import com.gemstone.gemfire.StatisticDescriptor;
 import com.gemstone.gemfire.StatisticsFactory;
 import com.gemstone.gemfire.StatisticsType;
@@ -26,15 +24,14 @@ import com.gemstone.gemfire.cache.EvictionAction;
 import com.gemstone.gemfire.cache.EvictionAlgorithm;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.util.ObjectSizer;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
-import com.gemstone.gemfire.internal.cache.AbstractRegion;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.Token;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
+import java.util.Properties;
+
 /**
  * A <code>HeapLRUCapacityController</code> controls the contents of
  * {@link Region} based on the percentage of memory that is
@@ -65,7 +62,7 @@ public class HeapLRUCapacityController extends LRUAlgorithm {
   /**
    * The default percentage of VM heap usage over which LRU eviction occurs
    */
-  public static final String TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY = "gemfire.topUpHeapEvictionPercentage";
+  public static final String TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY = DistributionConfig.GEMFIRE_PREFIX + "topUpHeapEvictionPercentage";
   
   public static final float DEFAULT_TOP_UP_HEAP_EVICTION_PERCENTAGE = 4.0f;
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
index 59245f8..7d2591c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
@@ -17,21 +17,16 @@
 
 package com.gemstone.gemfire.internal.cache.lru;
 
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.StatisticsFactory;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.cache.*;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.PlaceHolderDiskRegion;
-import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
+import org.apache.logging.log4j.Logger;
 
 /**
  *  AbstractLRUClockHand holds the lrulist, and the behavior for
@@ -56,13 +51,13 @@ public class NewLRUClockHand  {
   final private LRUStatistics stats;
   /** Counter for the size of the LRU list */
   protected int size = 0;
-  
-public static final boolean debug = Boolean.getBoolean("gemfire.verbose-lru-clock");
+
+  public static final boolean debug = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "verbose-lru-clock");
 
 static private final int maxEntries;
 
 static {
-  String squelch = System.getProperty("gemfire.lru.maxSearchEntries");
+  String squelch = System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "lru.maxSearchEntries");
   if (squelch == null)
     maxEntries = -1;
   else

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/OffHeapEvictor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/OffHeapEvictor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/OffHeapEvictor.java
index 4b1270b..d147c57 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/OffHeapEvictor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/OffHeapEvictor.java
@@ -17,7 +17,7 @@
 package com.gemstone.gemfire.internal.cache.lru;
 
 import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.control.ResourceManager;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
@@ -44,7 +44,8 @@ public class OffHeapEvictor extends HeapEvictor {
   }
 
   private void calculateEvictionBurst() {
-    float evictionBurstPercentage = Float.parseFloat(System.getProperty("gemfire.HeapLRUCapacityController.evictionBurstPercentage", "0.4"));
+    float evictionBurstPercentage = Float
+        .parseFloat(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.evictionBurstPercentage", "0.4"));
     
     MemoryAllocator allocator = ((GemFireCacheImpl) this.cache).getOffHeapStore();
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
index 76cdeb3..e903def 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
@@ -17,48 +17,18 @@
 
 package com.gemstone.gemfire.internal.cache.partitioned;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.TransactionDataNotColocatedException;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DirectReplyProcessor;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.DistributionMessage;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.ReplyException;
-import com.gemstone.gemfire.distributed.internal.ReplyMessage;
-import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
-import com.gemstone.gemfire.distributed.internal.ReplySender;
+import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.BucketRegion.RawValue;
-import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
-import com.gemstone.gemfire.internal.cache.DataLocationException;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyInfo;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
-import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
-import com.gemstone.gemfire.internal.cache.TXManagerImpl;
-import com.gemstone.gemfire.internal.cache.TXStateProxy;
-import com.gemstone.gemfire.internal.cache.Token;
-import com.gemstone.gemfire.internal.cache.VersionTagHolder;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -66,6 +36,13 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.util.BlobHelper;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
 
 /**
  * This message is used as the request for a
@@ -114,7 +91,7 @@ public final class GetMessage extends PartitionMessageWithDirectReply
     this.returnTombstones = returnTombstones;
   }
 
-  private static final boolean ORDER_PR_GETS = Boolean.getBoolean("gemfire.order-pr-gets");
+    private static final boolean ORDER_PR_GETS = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "order-pr-gets");
 
   @Override
   final public int getProcessorType()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRSanityCheckMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRSanityCheckMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRSanityCheckMessage.java
index 47b73ba..9e69e7a 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRSanityCheckMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRSanityCheckMessage.java
@@ -16,16 +16,12 @@
  */
 package com.gemstone.gemfire.internal.cache.partitioned;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Set;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
 import com.gemstone.gemfire.internal.SystemTimer;
@@ -34,6 +30,11 @@ import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionHelper;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Set;
+
 /**
  * PRSanityCheckMessage is used to assert correctness of prID assignments
  * across the distributed system.
@@ -97,7 +98,7 @@ public final class PRSanityCheckMessage extends PartitionMessage
    * be enabled with gemfire.PRSanityCheckEnabled=true. 
    */
   public static void schedule(final PartitionedRegion pr) {
-    if (Boolean.getBoolean("gemfire.PRSanityCheckEnabled")) {
+    if (Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "PRSanityCheckEnabled")) {
       final DM dm = pr.getDistributionManager();
 //      RegionAdvisor ra = pr.getRegionAdvisor();
 //      final Set recipients = ra.adviseAllPRNodes();
@@ -114,7 +115,7 @@ public final class PRSanityCheckMessage extends PartitionMessage
       PRSanityCheckMessage instance = new PRSanityCheckMessage(recipients,
           pr.getPRId(), null, pr.getRegionIdentifier());
       dm.putOutgoing(instance);
-      int sanityCheckInterval = Integer.getInteger("gemfire.PRSanityCheckInterval",
+      int sanityCheckInterval = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "PRSanityCheckInterval",
                                                    5000).intValue();
       if (sanityCheckInterval != 0) {
         final SystemTimer tm = new SystemTimer(dm.getSystem(), true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
index 641d43d..d683f03 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
@@ -16,22 +16,11 @@
  */
 package com.gemstone.gemfire.internal.cache.partitioned;
 
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.cache.partition.PartitionMemberInfo;
 import com.gemstone.gemfire.cache.partition.PartitionRebalanceInfo;
 import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.MembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
@@ -46,17 +35,15 @@ import com.gemstone.gemfire.internal.cache.control.ResourceManagerStats;
 import com.gemstone.gemfire.internal.cache.partitioned.BecomePrimaryBucketMessage.BecomePrimaryBucketResponse;
 import com.gemstone.gemfire.internal.cache.partitioned.MoveBucketMessage.MoveBucketResponse;
 import com.gemstone.gemfire.internal.cache.partitioned.RemoveBucketMessage.RemoveBucketResponse;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.BucketOperator;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.BucketOperatorImpl;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.BucketOperatorWrapper;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.ParallelBucketOperator;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.PartitionedRegionLoadModel;
+import com.gemstone.gemfire.internal.cache.partitioned.rebalance.*;
 import com.gemstone.gemfire.internal.cache.partitioned.rebalance.PartitionedRegionLoadModel.AddressComparor;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.RebalanceDirector;
-import com.gemstone.gemfire.internal.cache.partitioned.rebalance.SimulatedBucketOperator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import org.apache.logging.log4j.Logger;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * This class performs a rebalance on a single partitioned region. 
@@ -85,9 +72,9 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 @SuppressWarnings("synthetic-access")
 public class PartitionedRegionRebalanceOp {
   private static final Logger logger = LogService.getLogger();
-  
-  private static final int MAX_PARALLEL_OPERATIONS = Integer.getInteger("gemfire.MAX_PARALLEL_BUCKET_RECOVERIES", 8);
-  private final boolean DEBUG = Boolean.getBoolean("gemfire.LOG_REBALANCE");
+
+  private static final int MAX_PARALLEL_OPERATIONS = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "MAX_PARALLEL_BUCKET_RECOVERIES", 8);
+  private final boolean DEBUG = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "LOG_REBALANCE");
   
   private final boolean simulate;
   private final boolean replaceOfflineData;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
index bdb3c25..176a41a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
@@ -17,28 +17,6 @@
 
 package com.gemstone.gemfire.internal.cache.partitioned;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.AbstractSet;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireException;
@@ -46,26 +24,15 @@ import com.gemstone.gemfire.cache.InterestPolicy;
 import com.gemstone.gemfire.cache.LowMemoryException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ProfileListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.cache.BucketAdvisor;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor.BucketProfile;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor.ServerBucketProfile;
-import com.gemstone.gemfire.internal.cache.BucketPersistenceAdvisor;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.BucketServerLocation66;
-import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.FixedPartitionAttributesImpl;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
-import com.gemstone.gemfire.internal.cache.Node;
 import com.gemstone.gemfire.internal.cache.PRHARedundancyProvider.DataStoreBuckets;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegionStats;
-import com.gemstone.gemfire.internal.cache.ProxyBucketRegion;
 import com.gemstone.gemfire.internal.cache.control.MemoryThresholds;
 import com.gemstone.gemfire.internal.cache.control.ResourceAdvisor;
 import com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisor;
@@ -74,6 +41,16 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicLong;
 
 public class RegionAdvisor extends CacheDistributionAdvisor
 {
@@ -83,7 +60,7 @@ public class RegionAdvisor extends CacheDistributionAdvisor
    * Number of threads allowed to concurrently volunteer for bucket primary.
    */
   public static final short VOLUNTEERING_THREAD_COUNT = Integer.getInteger(
-      "gemfire.RegionAdvisor.volunteeringThreadCount", 1).shortValue();
+      DistributionConfig.GEMFIRE_PREFIX + "RegionAdvisor.volunteeringThreadCount", 1).shortValue();
   
   /**
    * Non-thread safe queue for volunteering for primary bucket. Each
@@ -1232,7 +1209,7 @@ public class RegionAdvisor extends CacheDistributionAdvisor
 
   /**
    * Get the most recent primary node for the bucketId. Returns null if no 
-   * primary can be found within {@link com.gemstone.gemfire.distributed.internal.DistributionConfig#getMemberTimeout}.
+   * primary can be found within {@link DistributionConfig#getMemberTimeout}.
    * @param bucketId
    * @return the Node managing the primary copy of the bucket 
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/SizedBasedLoadProbe.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/SizedBasedLoadProbe.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/SizedBasedLoadProbe.java
index 068c6a4..df4d894 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/SizedBasedLoadProbe.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/SizedBasedLoadProbe.java
@@ -16,11 +16,13 @@
  */
 package com.gemstone.gemfire.internal.cache.partitioned;
 
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -34,7 +36,7 @@ import java.io.IOException;
 public class SizedBasedLoadProbe implements LoadProbe, DataSerializableFixedID {
   private static final long serialVersionUID = 7040814060882774875L;
   //TODO rebalancing come up with a better threshold for minumum bucket size?
-  public static final int MIN_BUCKET_SIZE = Integer.getInteger("gemfire.MIN_BUCKET_SIZE", 1).intValue();
+  public static final int MIN_BUCKET_SIZE = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "MIN_BUCKET_SIZE", 1).intValue();
 
   public PRLoad getLoad(PartitionedRegion pr) {
     PartitionedRegionDataStore ds = pr.getDataStore();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
index 89856ad..7c4cc55 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
@@ -16,18 +16,6 @@
  */
 package com.gemstone.gemfire.internal.cache.persistence;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-
 import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.cache.persistence.PersistentID;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -42,6 +30,13 @@ import com.gemstone.gemfire.internal.cache.DiskStoreImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URL;
+import java.util.*;
+import java.util.concurrent.CountDownLatch;
+
 /**
  * This class manages the state an logic to backup a single
  * cache.
@@ -243,7 +238,7 @@ public class BackupManager implements MembershipListener {
     
     URL propertyURL = DistributedSystem.getPropertyFileURL();
     if(propertyURL != null) {
-      File propertyBackup = new File(configBackupDir, "gemfire.properties");
+      File propertyBackup = new File(configBackupDir, DistributionConfig.GEMFIRE_PREFIX + "properties");
       FileUtil.copy(propertyURL, propertyBackup);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
index 3c09612..d865828 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
@@ -16,29 +16,13 @@
  */
 package com.gemstone.gemfire.internal.cache.persistence;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.cache.DiskAccessException;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.persistence.ConflictingPersistentDataException;
 import com.gemstone.gemfire.cache.persistence.RevokedPersistentDataException;
 import com.gemstone.gemfire.distributed.DistributedLockService;
-import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.Profile;
-import com.gemstone.gemfire.distributed.internal.MembershipListener;
-import com.gemstone.gemfire.distributed.internal.ProfileListener;
-import com.gemstone.gemfire.distributed.internal.ReplyException;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor.CacheProfile;
@@ -51,6 +35,11 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.process.StartupStatus;
 import com.gemstone.gemfire.internal.util.TransformUtils;
+import org.apache.logging.log4j.Logger;
+
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 /**
  *
@@ -79,8 +68,8 @@ public class PersistenceAdvisorImpl implements PersistenceAdvisor {
   private volatile Set<PersistentMemberID> allMembersWaitingFor;
   private volatile Set<PersistentMemberID> offlineMembersWaitingFor;
   protected final Object lock;
-  
-  private static final int PERSISTENT_VIEW_RETRY = Integer.getInteger("gemfire.PERSISTENT_VIEW_RETRY", 5);
+
+  private static final int PERSISTENT_VIEW_RETRY = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "PERSISTENT_VIEW_RETRY", 5);
   
   public PersistenceAdvisorImpl(CacheDistributionAdvisor advisor, DistributedLockService dl, PersistentMemberView storage, String regionPath, DiskRegionStats diskStats, PersistentMemberManager memberManager) {
     this.advisor = advisor;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/RegionSnapshotServiceImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/RegionSnapshotServiceImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/RegionSnapshotServiceImpl.java
index 78f6b0e..804aa93 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/RegionSnapshotServiceImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/RegionSnapshotServiceImpl.java
@@ -16,51 +16,36 @@
  */
 package com.gemstone.gemfire.internal.cache.snapshot;
 
-import static com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.getLoggerI18n;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.ProxyRegion;
-import com.gemstone.gemfire.cache.execute.Function;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionException;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
-import com.gemstone.gemfire.cache.execute.ResultCollector;
-import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.execute.*;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.cache.snapshot.RegionSnapshotService;
 import com.gemstone.gemfire.cache.snapshot.SnapshotOptions;
 import com.gemstone.gemfire.cache.snapshot.SnapshotOptions.SnapshotFormat;
 import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.DSCODE;
-import com.gemstone.gemfire.internal.cache.CachePerfStats;
-import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalDataSet;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.Token;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.snapshot.GFSnapshot.GFSnapshotImporter;
 import com.gemstone.gemfire.internal.cache.snapshot.GFSnapshot.SnapshotWriter;
 import com.gemstone.gemfire.internal.cache.snapshot.SnapshotPacket.SnapshotRecord;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
+import java.io.*;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import static com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.getLoggerI18n;
+
 /**
  * Provides an implementation for region snapshots.
  * 
@@ -70,10 +55,10 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
  */
 public class RegionSnapshotServiceImpl<K, V> implements RegionSnapshotService<K, V> {
   // controls number of concurrent putAll ops during an import
-  private static final int IMPORT_CONCURRENCY = Integer.getInteger("gemfire.RegionSnapshotServiceImpl.IMPORT_CONCURRENCY", 10);
+  private static final int IMPORT_CONCURRENCY = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "RegionSnapshotServiceImpl.IMPORT_CONCURRENCY", 10);
   
   // controls the size (in bytes) of the r/w buffer during imoprt and export
-  static final int BUFFER_SIZE = Integer.getInteger("gemfire.RegionSnapshotServiceImpl.BUFFER_SIZE", 1024 * 1024);
+  static final int BUFFER_SIZE = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "RegionSnapshotServiceImpl.BUFFER_SIZE", 1024 * 1024);
   
   static final SnapshotFileMapper LOCAL_MAPPER = new SnapshotFileMapper() {
     private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/WindowedExporter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/WindowedExporter.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/WindowedExporter.java
index a097568..2860719 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/WindowedExporter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/snapshot/WindowedExporter.java
@@ -16,7 +16,21 @@
  */
 package com.gemstone.gemfire.internal.cache.snapshot;
 
-import static com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.getLoggerI18n;
+import com.gemstone.gemfire.cache.EntryDestroyedException;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.*;
+import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
+import com.gemstone.gemfire.cache.snapshot.SnapshotOptions;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.execute.InternalExecution;
+import com.gemstone.gemfire.internal.cache.execute.LocalResultCollector;
+import com.gemstone.gemfire.internal.cache.snapshot.FlowController.Window;
+import com.gemstone.gemfire.internal.cache.snapshot.RegionSnapshotServiceImpl.ExportSink;
+import com.gemstone.gemfire.internal.cache.snapshot.RegionSnapshotServiceImpl.Exporter;
+import com.gemstone.gemfire.internal.cache.snapshot.SnapshotPacket.SnapshotRecord;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -32,26 +46,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import com.gemstone.gemfire.cache.EntryDestroyedException;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.execute.Function;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionException;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
-import com.gemstone.gemfire.cache.execute.ResultCollector;
-import com.gemstone.gemfire.cache.execute.ResultSender;
-import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
-import com.gemstone.gemfire.cache.snapshot.SnapshotOptions;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.execute.InternalExecution;
-import com.gemstone.gemfire.internal.cache.execute.LocalResultCollector;
-import com.gemstone.gemfire.internal.cache.snapshot.FlowController.Window;
-import com.gemstone.gemfire.internal.cache.snapshot.RegionSnapshotServiceImpl.ExportSink;
-import com.gemstone.gemfire.internal.cache.snapshot.RegionSnapshotServiceImpl.Exporter;
-import com.gemstone.gemfire.internal.cache.snapshot.SnapshotPacket.SnapshotRecord;
+import static com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.getLoggerI18n;
 
 /**
  * Exports snapshot data using a sliding window to prevent the nodes in a 
@@ -65,7 +60,7 @@ import com.gemstone.gemfire.internal.cache.snapshot.SnapshotPacket.SnapshotRecor
  * @param <V> the value type
  */
 public class WindowedExporter<K, V> implements Exporter<K, V> {
-  private static final int WINDOW_SIZE = Integer.getInteger("gemfire.WindowedExporter.WINDOW_SIZE", 10);
+  private static final int WINDOW_SIZE = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "WindowedExporter.WINDOW_SIZE", 10);
 
   @Override
   public long export(Region<K, V> region, ExportSink sink, SnapshotOptions<K, V> options) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index e1c22b7..2a7c144 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -17,48 +17,6 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.net.BindException;
-import java.net.Inet6Address;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.nio.channels.CancelledKeyException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.ClosedSelectorException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.net.ssl.SSLException;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.ToDataException;
@@ -67,12 +25,7 @@ import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.wan.GatewayTransportFilter;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.LonerDistributionManager;
-import com.gemstone.gemfire.distributed.internal.PooledExecutorWithDMStats;
-import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
+import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor;
@@ -90,6 +43,18 @@ import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
+import org.apache.logging.log4j.Logger;
+
+import javax.net.ssl.SSLException;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.*;
+import java.nio.ByteBuffer;
+import java.nio.channels.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Implements the acceptor thread on the bridge server. Accepts connections from

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
index ca7818a..bdebb6b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
@@ -19,57 +19,18 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Semaphore;
-import java.util.regex.Pattern;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.CopyException;
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.SerializationException;
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.CacheLoaderException;
-import com.gemstone.gemfire.cache.CacheWriterException;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.TransactionException;
+import com.gemstone.gemfire.*;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.persistence.PartitionOfflineException;
 import com.gemstone.gemfire.cache.query.types.CollectionType;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.cache.DistributedRegion;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.EntrySnapshot;
-import com.gemstone.gemfire.internal.cache.EventID;
-import com.gemstone.gemfire.internal.cache.FindVersionTagOperation;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.LocalRegion.NonTXEntry;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegionHelper;
-import com.gemstone.gemfire.internal.cache.TXManagerImpl;
-import com.gemstone.gemfire.internal.cache.TXStateProxy;
-import com.gemstone.gemfire.internal.cache.Token;
-import com.gemstone.gemfire.internal.cache.VersionTagHolder;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.InterestType;
@@ -82,6 +43,12 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import org.apache.logging.log4j.Logger;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.Semaphore;
+import java.util.regex.Pattern;
 
 /**
  *
@@ -97,7 +64,7 @@ public abstract class BaseCommand implements Command {
   protected static final boolean zipValues = false;
 
   protected static final boolean APPLY_RETRIES = Boolean
-      .getBoolean("gemfire.gateway.ApplyRetries");
+      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "gateway.ApplyRetries");
 
   public static final byte[] OK_BYTES = new byte[]{0};  
 
@@ -108,7 +75,7 @@ public abstract class BaseCommand implements Command {
 
   /** Whether to suppress logging of IOExceptions */
   private static boolean suppressIOExceptionLogging = Boolean
-      .getBoolean("gemfire.bridge.suppressIOExceptionLogging");
+      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "bridge.suppressIOExceptionLogging");
 
   /**
    * Maximum number of concurrent incoming client message bytes that a bridge

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
index b8cd2f5..517d497 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
@@ -352,7 +352,10 @@ public class CacheClientNotifier {
             .getProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_PP_NAME);
         if (postAuthzFactoryName != null && postAuthzFactoryName.length() > 0) {
           if (principal == null) {
-            securityLogWriter.warning(LocalizedStrings.CacheClientNotifier_CACHECLIENTNOTIFIER_POST_PROCESS_AUTHORIZATION_CALLBACK_ENABLED_BUT_AUTHENTICATION_CALLBACK_0_RETURNED_WITH_NULL_CREDENTIALS_FOR_PROXYID_1, new Object[] {DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, proxyID});
+            securityLogWriter.warning(
+                LocalizedStrings.CacheClientNotifier_CACHECLIENTNOTIFIER_POST_PROCESS_AUTHORIZATION_CALLBACK_ENABLED_BUT_AUTHENTICATION_CALLBACK_0_RETURNED_WITH_NULL_CREDENTIALS_FOR_PROXYID_1,
+                new Object[] {
+                    DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, proxyID });
           }
           Method authzMethod = ClassLoadUtil
               .methodFromName(postAuthzFactoryName);
@@ -2575,11 +2578,11 @@ public class CacheClientNotifier {
    * System property name for indicating how much frequently the "Queue full"
    * message should be logged.
    */
-  public static final String MAX_QUEUE_LOG_FREQUENCY = "gemfire.logFrequency.clientQueueReachedMaxLimit";
+  public static final String MAX_QUEUE_LOG_FREQUENCY = DistributionConfig.GEMFIRE_PREFIX + "logFrequency.clientQueueReachedMaxLimit";
 
   public static final long DEFAULT_LOG_FREQUENCY = 1000;
 
-  public static final String EVENT_ENQUEUE_WAIT_TIME_NAME = "gemfire.subscription.EVENT_ENQUEUE_WAIT_TIME";
+  public static final String EVENT_ENQUEUE_WAIT_TIME_NAME = DistributionConfig.GEMFIRE_PREFIX + "subscription.EVENT_ENQUEUE_WAIT_TIME";
 
   public static final int DEFAULT_EVENT_ENQUEUE_WAIT_TIME = 100;
 
@@ -2607,10 +2610,10 @@ public class CacheClientNotifier {
   private final SocketCloser socketCloser;
   
   private static final long CLIENT_PING_TASK_PERIOD =
-    Long.getLong("gemfire.serverToClientPingPeriod", 60000);
+      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "serverToClientPingPeriod", 60000);
 
   private static final long CLIENT_PING_TASK_COUNTER =
-    Long.getLong("gemfire.serverToClientPingCounter", 3);
+      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "serverToClientPingCounter", 3);
 
   public long getLogFrequency() {
     return this.logFrequency;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
index d9a71d5..c4b48f4 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
@@ -17,77 +17,25 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Pattern;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.StatisticsFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheClosedException;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.ClientSession;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.InterestRegistrationEvent;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.RegionExistsException;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
-import com.gemstone.gemfire.cache.operations.DestroyOperationContext;
-import com.gemstone.gemfire.cache.operations.InvalidateOperationContext;
-import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.cache.operations.PutOperationContext;
-import com.gemstone.gemfire.cache.operations.RegionClearOperationContext;
-import com.gemstone.gemfire.cache.operations.RegionCreateOperationContext;
-import com.gemstone.gemfire.cache.operations.RegionDestroyOperationContext;
+import com.gemstone.gemfire.cache.operations.*;
 import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.CqQuery;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
 import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.SystemTimer.SystemTimerTask;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.ClientServerObserver;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
-import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisee;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor.InitialImageAdvice;
-import com.gemstone.gemfire.internal.cache.Conflatable;
-import com.gemstone.gemfire.internal.cache.DistributedRegion;
-import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
-import com.gemstone.gemfire.internal.cache.EventID;
-import com.gemstone.gemfire.internal.cache.FilterProfile;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InterestRegistrationEventImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.StateFlushOperation;
 import com.gemstone.gemfire.internal.cache.ha.HAContainerWrapper;
 import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
 import com.gemstone.gemfire.internal.cache.ha.HARegionQueueAttributes;
@@ -104,7 +52,23 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
 import com.gemstone.gemfire.security.AccessControl;
-import com.gemstone.gemfire.i18n.StringId;
+import org.apache.logging.log4j.Logger;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Pattern;
 
 /**
  * Class <code>CacheClientProxy</code> represents the server side of the
@@ -198,7 +162,7 @@ public class CacheClientProxy implements ClientSession {
   /**
    * The number of times to peek on shutdown before giving up and shutting down
    */
-  protected static final int MAXIMUM_SHUTDOWN_PEEKS = Integer.getInteger("gemfire.MAXIMUM_SHUTDOWN_PEEKS",50).intValue();
+  protected static final int MAXIMUM_SHUTDOWN_PEEKS = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "MAXIMUM_SHUTDOWN_PEEKS", 50).intValue();
 
   /**
    * The number of milliseconds to wait for an offering to the message queue
@@ -230,7 +194,7 @@ public class CacheClientProxy implements ClientSession {
    * be logged.
    */
   protected static final boolean LOG_DROPPED_MSGS = !Boolean
-      .getBoolean("gemfire.disableNotificationWarnings");
+      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "disableNotificationWarnings");
 
   /**
    * for testing purposes, delays the start of the dispatcher thread
@@ -294,7 +258,7 @@ public class CacheClientProxy implements ClientSession {
    * currently.
    */
   protected static final boolean NOTIFY_REGION_ON_INTEREST = Boolean
-      .getBoolean("gemfire.updateAccessTimeOnClientInterest");   
+      .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "updateAccessTimeOnClientInterest");
   
   /**
    * The AcceptorImpl identifier to which the proxy is connected.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
index 6e45d1d..28b789a 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
@@ -17,68 +17,21 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.net.NoRouteToHostException;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import javax.net.ssl.SSLException;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.InvalidDeltaException;
-import com.gemstone.gemfire.StatisticDescriptor;
-import com.gemstone.gemfire.Statistics;
-import com.gemstone.gemfire.StatisticsType;
-import com.gemstone.gemfire.StatisticsTypeFactory;
-import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.*;
 import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.client.ServerRefusedConnectionException;
-import com.gemstone.gemfire.cache.client.internal.ClientUpdater;
-import com.gemstone.gemfire.cache.client.internal.Endpoint;
-import com.gemstone.gemfire.cache.client.internal.EndpointManager;
-import com.gemstone.gemfire.cache.client.internal.GetEventValueOp;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.QueueManager;
+import com.gemstone.gemfire.cache.client.internal.*;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.DisconnectListener;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
-import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.InternalInstantiator;
-import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.ClientServerObserver;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.EventID;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.*;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.versions.ConcurrentCacheModificationException;
@@ -95,6 +48,19 @@ import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.AuthenticationRequiredException;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import org.apache.logging.log4j.Logger;
+
+import javax.net.ssl.SSLException;
+import java.io.*;
+import java.net.ConnectException;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * <code>CacheClientUpdater</code> is a thread that processes update messages
@@ -214,8 +180,8 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
   private EndpointManager eManager = null;
   private Endpoint endpoint = null;
 
-  static private final long MAX_CACHE_WAIT = 
-    Long.getLong("gemfire.CacheClientUpdater.MAX_WAIT", 120).longValue(); // seconds
+  static private final long MAX_CACHE_WAIT =
+      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "CacheClientUpdater.MAX_WAIT", 120).longValue(); // seconds
   /**
    * Return true if cache appears
    * @return true if cache appears

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
index ed38bd6..eb701fb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
@@ -17,36 +17,25 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicIntegerArray;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.SystemTimer.SystemTimerTask;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.CacheClientStatus;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.IncomingGatewayStatus;
-import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.cache.TXManagerImpl;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.concurrent.ConcurrentHashSet;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import org.apache.logging.log4j.Logger;
+
+import java.net.InetAddress;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicIntegerArray;
 
 /**
  * Class <code>ClientHealthMonitor</code> is a server-side singleton that
@@ -279,8 +268,8 @@ public class ClientHealthMonitor {
       }
     }
   }
-  
-  private final Set<TXId> scheduledToBeRemovedTx = Boolean.getBoolean("gemfire.trackScheduledToBeRemovedTx")? 
+
+  private final Set<TXId> scheduledToBeRemovedTx = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "trackScheduledToBeRemovedTx") ?
       new ConcurrentHashSet<TXId>() : null;
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
index 61ea62a..d4c291f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
@@ -16,35 +16,21 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.Serializable;
-import java.util.Arrays;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.DurableClientAttributes;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.DataSerializableFixedID;
-import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.VersionedDataInputStream;
+import com.gemstone.gemfire.internal.*;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import org.apache.logging.log4j.Logger;
+
+import java.io.*;
+import java.util.Arrays;
 
 /**
  * This class represents a ConnectionProxy of the CacheClient
@@ -215,8 +201,8 @@ public final class ClientProxyMembershipID
   }
 
   private ClientProxyMembershipID(int id, byte[] clientSideIdentity) {
-    Boolean specialCase=Boolean.getBoolean("gemfire.SPECIAL_DURABLE");
-    String durableID = this.system.getProperties().getProperty("durable-client-id"); 
+    Boolean specialCase = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "SPECIAL_DURABLE");
+    String durableID = this.system.getProperties().getProperty(DistributionConfig.DURABLE_CLIENT_ID_NAME);
     if (specialCase.booleanValue() && durableID != null && (!durableID.equals(""))) {
         this.uniqueId = durable_synch_counter;
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
index 3e07cfe..4ea34f5 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
@@ -17,50 +17,6 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataOutput;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.Method;
-import java.math.BigInteger;
-import java.net.Socket;
-import java.security.Key;
-import java.security.KeyFactory;
-import java.security.KeyPair;
-import java.security.KeyPairGenerator;
-import java.security.KeyStore;
-import java.security.Principal;
-import java.security.PrivateKey;
-import java.security.PublicKey;
-import java.security.SecureRandom;
-import java.security.Signature;
-import java.security.cert.Certificate;
-import java.security.cert.X509Certificate;
-import java.security.spec.X509EncodedKeySpec;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import javax.crypto.Cipher;
-import javax.crypto.KeyAgreement;
-import javax.crypto.SecretKey;
-import javax.crypto.spec.DHParameterSpec;
-import javax.crypto.spec.IvParameterSpec;
-import javax.crypto.spec.SecretKeySpec;
-import javax.net.ssl.SSLSocket;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireException;
@@ -70,20 +26,9 @@ import com.gemstone.gemfire.cache.client.ServerRefusedConnectionException;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.LonerDistributionManager;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.internal.ClassLoadUtil;
-import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.InternalInstantiator;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.VersionedDataInputStream;
-import com.gemstone.gemfire.internal.VersionedDataOutputStream;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.*;
 import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.cache.tier.ClientHandShake;
 import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
@@ -91,11 +36,25 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.pdx.internal.PeerTypeRegistration;
-import com.gemstone.gemfire.security.AuthInitialize;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-import com.gemstone.gemfire.security.AuthenticationRequiredException;
-import com.gemstone.gemfire.security.Authenticator;
-import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.*;
+import org.apache.logging.log4j.Logger;
+
+import javax.crypto.Cipher;
+import javax.crypto.KeyAgreement;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.DHParameterSpec;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import javax.net.ssl.SSLSocket;
+import java.io.*;
+import java.lang.reflect.Method;
+import java.math.BigInteger;
+import java.net.Socket;
+import java.security.*;
+import java.security.cert.Certificate;
+import java.security.cert.X509Certificate;
+import java.security.spec.X509EncodedKeySpec;
+import java.util.*;
 
 public class HandShake implements ClientHandShake
 {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70612010/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
index aef2935..139ccde 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
@@ -16,20 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SocketChannel;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.SerializationException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
@@ -41,6 +29,18 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.util.BlobHelper;
+import org.apache.logging.log4j.Logger;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
 
 /**
  * This class encapsulates the wire protocol. It provides accessors to
@@ -85,7 +85,7 @@ public class Message  {
   /**
    * maximum size of an outgoing message.  See GEODE-478
    */
-  public static int MAX_MESSAGE_SIZE = Integer.getInteger("gemfire.client.max-message-size", DEFAULT_MAX_MESSAGE_SIZE).intValue();
+  public static int MAX_MESSAGE_SIZE = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "client.max-message-size", DEFAULT_MAX_MESSAGE_SIZE).intValue();
 
   private static final Logger logger = LogService.getLogger();