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 2017/05/09 21:09:36 UTC

[48/50] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java b/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
index 57a1a46..f2cbfaf 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.cache;
 
 import java.io.File;
@@ -22,6 +21,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
+import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.geode.InternalGemFireError;
@@ -40,8 +40,8 @@ import org.apache.geode.internal.cache.DynamicRegionFactoryImpl;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.EnumListenerEvent;
 import org.apache.geode.internal.cache.EvictionAttributesImpl;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InitialImageOperation;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegionArguments;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.RegionEntry;
@@ -49,7 +49,6 @@ import org.apache.geode.internal.cache.RegionEventImpl;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.security.GemFireSecurityException;
 
-
 /**
  * DynamicRegionFactory provides a distributed region creation service. Any other member of the
  * GemFire DistributedSystem that has created an instance of this class will automatically
@@ -65,12 +64,16 @@ import org.apache.geode.security.GemFireSecurityException;
  * <li>Before you've created a GemFire Cache in your application, add a line of code as follows:<br>
  * 
  * <pre>
- * <code>  DynamicRegionFactory factory = DynamicRegionFactory.get();
- *  factory.open(config);</code>
+ * {
+ *   DynamicRegionFactory factory = DynamicRegionFactory.get();
+ *   factory.open(config);
+ * }
  * </pre>
  * 
  * <pre>
- * <code>  DynamicRegionFactory myFactoryHandle = DynamicRegionFactory.get().open(config);</code>
+ * {
+ *   DynamicRegionFactory myFactoryHandle = DynamicRegionFactory.get().open(config);
+ * }
  * </pre>
  * 
  * or just use a dynamic-region-factory element in the cache.xml.
@@ -130,26 +133,31 @@ import org.apache.geode.security.GemFireSecurityException;
  * @since GemFire 4.3
  * @deprecated This class is deprecated. Use {@link FunctionService} to create regions on other
  *             members instead.
- *
  */
 @SuppressWarnings("deprecation")
 @Deprecated
 public abstract class DynamicRegionFactory {
 
   public static final String dynamicRegionListName = "__DynamicRegions";
+
   private Region dynamicRegionList = null;
+
   /**
    * This controls the delay introduced to try and avoid any race conditions between propagation of
    * newly created Dynamic Regions and the Entries put into them.
    */
   private static final long regionCreateSleepMillis =
-      Long.getLong("DynamicRegionFactory.msDelay", 250).longValue();
-  private static DynamicRegionFactory singleInstance = new DynamicRegionFactoryImpl();
-  GemFireCacheImpl c = null;
-  Config config = null;
+      Long.getLong("DynamicRegionFactory.msDelay", 250);
+
+  private static final DynamicRegionFactory singleInstance = new DynamicRegionFactoryImpl();
+
+  InternalCache cache = null;
+
+  private Config config = null;
 
   /** The region listeners registered on this DynamicRegionFactory */
-  private static volatile List regionListeners = Collections.EMPTY_LIST;
+  private static volatile List regionListeners = Collections.emptyList();
+
   private static final Object regionListenerLock = new Object();
 
   /**
@@ -174,9 +182,9 @@ public abstract class DynamicRegionFactory {
    * Closes the dynamic region factory, disabling any further creation or destruction of dynamic
    * regions in this cache.
    */
-  protected void _close() {
+  protected void doClose() {
     this.config = null;
-    this.c = null;
+    this.cache = null;
   }
 
   /**
@@ -191,7 +199,7 @@ public abstract class DynamicRegionFactory {
    * after their cache has been created.
    */
   public boolean isActive() {
-    return isOpen() && this.c != null;
+    return isOpen() && this.cache != null;
   }
 
   /**
@@ -217,24 +225,21 @@ public abstract class DynamicRegionFactory {
 
   /**
    * The method is for internal use only. It is called implicitly during cache creation.
+   * <p>
+   * This method is called internally during cache initialization at the correct time. Initialize
+   * the factory with a GemFire Cache. We create the metadata Region which holds all our dynamically
+   * created regions.
    * 
-   * @param theCache The GemFire <code>Cache</code>
-   * @throws CacheException
+   * @param theCache The GemFire {@code Cache}
    */
-
-  protected void _internalInit(GemFireCacheImpl theCache) throws CacheException {
-
+  protected void doInternalInit(InternalCache theCache) throws CacheException {
     if (isClosed()) {
       // DynamicRegions are not enabled in this vm. Just return.
       return;
     }
-    /**
-     * This method is called internally during cache initialization at the correct time. Initialize
-     * the factory with a GemFire Cache. We create the metadata Region which holds all our
-     * dynamically created regions.
-     */
+
     try {
-      this.c = theCache;
+      this.cache = theCache;
       this.dynamicRegionList = theCache.getRegion(dynamicRegionListName);
       final boolean isClient = this.config.getPoolName() != null;
       if (this.dynamicRegionList == null) {
@@ -276,7 +281,7 @@ public abstract class DynamicRegionFactory {
             af.setDataPolicy(DataPolicy.REPLICATE); // setMirrorType(MirrorType.KEYS_VALUES);
           }
 
-          for (GatewaySender gs : c.getGatewaySenders()) {
+          for (GatewaySender gs : this.cache.getGatewaySenders()) {
             if (!gs.isParallel())
               af.addGatewaySenderId(gs.getId());
           }
@@ -284,36 +289,31 @@ public abstract class DynamicRegionFactory {
         }
 
         try {
-          dynamicRegionList = theCache.createVMRegion(dynamicRegionListName, af.create(), ira);
+          this.dynamicRegionList = theCache.createVMRegion(dynamicRegionListName, af.create(), ira);
         } catch (IOException e) {
           // only if loading snapshot, not here
-          InternalGemFireError assErr = new InternalGemFireError(
-              LocalizedStrings.DynamicRegionFactory_UNEXPECTED_EXCEPTION.toLocalizedString());
-          assErr.initCause(e);
-          throw assErr;
+          throw new InternalGemFireError(
+              LocalizedStrings.DynamicRegionFactory_UNEXPECTED_EXCEPTION.toLocalizedString(), e);
         } catch (ClassNotFoundException e) {
           // only if loading snapshot, not here
-          InternalGemFireError assErr = new InternalGemFireError(
-              LocalizedStrings.DynamicRegionFactory_UNEXPECTED_EXCEPTION.toLocalizedString());
-          assErr.initCause(e);
-          throw assErr;
+          throw new InternalGemFireError(
+              LocalizedStrings.DynamicRegionFactory_UNEXPECTED_EXCEPTION.toLocalizedString(), e);
         }
         if (isClient) {
-          dynamicRegionList.registerInterest("ALL_KEYS");
+          this.dynamicRegionList.registerInterest("ALL_KEYS");
         }
         if (theCache.getLoggerI18n().fineEnabled()) {
-          theCache.getLoggerI18n().fine("Created dynamic region: " + dynamicRegionList);
+          theCache.getLoggerI18n().fine("Created dynamic region: " + this.dynamicRegionList);
         }
       } else {
         if (theCache.getLoggerI18n().fineEnabled()) {
-          theCache.getLoggerI18n().fine("Retrieved dynamic region: " + dynamicRegionList);
+          theCache.getLoggerI18n().fine("Retrieved dynamic region: " + this.dynamicRegionList);
         }
       }
 
       createDefinedDynamicRegions();
 
     } catch (CacheException e) {
-      //
       theCache.getLoggerI18n().warning(
           LocalizedStrings.DynamicRegionFactory_ERROR_INITIALIZING_DYNAMICREGIONFACTORY, e);
       throw e;
@@ -322,47 +322,44 @@ public abstract class DynamicRegionFactory {
 
   /**
    * This creates Dynamic Regions that already exist in other publishing processes
-   *
    */
   private void createDefinedDynamicRegions() throws CacheException {
-    // TODO: perhaps add some logic here to avoid the possiblity of synchronization issues . . . .
-    Set s = dynamicRegionList.entrySet(false);
+    // TODO: perhaps add some logic here to avoid the possibility of synchronization issues
+    Set set = this.dynamicRegionList.entrySet(false);
 
-    Iterator i = s.iterator();
-    TreeMap sorted = new TreeMap();
+    Iterator iterator = set.iterator();
+    SortedMap sorted = new TreeMap();
 
     // sort by region name before creating (bug 35528)
-    while (i.hasNext()) {
-      Region.Entry e = (Region.Entry) i.next();
+    while (iterator.hasNext()) {
+      Region.Entry e = (Region.Entry) iterator.next();
       DynamicRegionAttributes dda = (DynamicRegionAttributes) e.getValue();
-      sorted.put(dda.rootRegionName + "/" + dda.name, dda);
+      sorted.put(dda.rootRegionName + '/' + dda.name, dda);
     }
-    i = sorted.values().iterator();
+    iterator = sorted.values().iterator();
 
-    while (i.hasNext()) {
-      DynamicRegionAttributes dda = (DynamicRegionAttributes) i.next();
+    while (iterator.hasNext()) {
+      DynamicRegionAttributes dda = (DynamicRegionAttributes) iterator.next();
 
-      doBeforeRegionCreated(dda.rootRegionName, dda.name, null);
+      doBeforeRegionCreated(dda.rootRegionName, dda.name);
       Region region = createDynamicRegionImpl(dda.rootRegionName, dda.name, false);
       doAfterRegionCreated(region, false, false, null);
-
     }
-
   }
 
   /**
-   * Returns the <code>DynamicRegionFactory</code> singleton instance.
+   * Returns the {@code DynamicRegionFactory} singleton instance.
    * 
-   * @return the <code>DynamicRegionFactory</code> singleton instance
+   * @return the {@code DynamicRegionFactory} singleton instance
    */
   public static DynamicRegionFactory get() {
     return singleInstance;
   }
 
   /**
-   * Registers a <code>DynamicRegionListener</code> for callbacks.
+   * Registers a {@code DynamicRegionListener} for callbacks.
    * 
-   * @param listener The <code>DynamicRegionListener</code> to be registered
+   * @param listener The {@code DynamicRegionListener} to be registered
    */
   public void registerDynamicRegionListener(DynamicRegionListener listener) {
     synchronized (regionListenerLock) {
@@ -376,9 +373,9 @@ public abstract class DynamicRegionFactory {
   }
 
   /**
-   * Unregisters a <code>DynamicRegionListener</code> for callbacks.
+   * Unregisters a {@code DynamicRegionListener} for callbacks.
    * 
-   * @param listener The <code>DynamicRegionListener</code> to be unregistered
+   * @param listener The {@code DynamicRegionListener} to be unregistered
    */
   public void unregisterDynamicRegionListener(DynamicRegionListener listener) {
     synchronized (regionListenerLock) {
@@ -392,10 +389,9 @@ public abstract class DynamicRegionFactory {
     }
   }
 
-  private void doBeforeRegionCreated(String parentRegion, String regionName,
-      DistributedMember mbr) {
-    for (Iterator i = regionListeners.iterator(); i.hasNext();) {
-      DynamicRegionListener listener = (DynamicRegionListener) i.next();
+  private void doBeforeRegionCreated(String parentRegion, String regionName) {
+    for (Object regionListener : regionListeners) {
+      DynamicRegionListener listener = (DynamicRegionListener) regionListener;
       try {
         listener.beforeRegionCreate(parentRegion, regionName);
       } catch (VirtualMachineError err) {
@@ -410,7 +406,7 @@ public abstract class DynamicRegionFactory {
         // error condition, so you also need to check to see if the JVM
         // is still usable:
         SystemFailure.checkFailure();
-        this.c.getLoggerI18n().warning(
+        this.cache.getLoggerI18n().warning(
             LocalizedStrings.DynamicRegionFactory_DYNAMICREGIONLISTENER__0__THREW_EXCEPTION_ON_BEFOREREGIONCREATED,
             listener, t);
       }
@@ -421,10 +417,10 @@ public abstract class DynamicRegionFactory {
       DistributedMember mbr) {
     RegionEvent event =
         new RegionEventImpl(region, Operation.REGION_CREATE, null, isOriginRemote, getMember(mbr));
-    for (Iterator i = regionListeners.iterator(); i.hasNext();) {
-      DynamicRegionListener listener = (DynamicRegionListener) i.next();
+    for (Object regionListener : regionListeners) {
+      DynamicRegionListener listener = (DynamicRegionListener) regionListener;
       try {
-        listener.afterRegionCreate(event /* region */ );
+        listener.afterRegionCreate(event /* region */);
       } catch (VirtualMachineError err) {
         SystemFailure.initiateFailure(err);
         // If this ever returns, rethrow the error. We're poisoned
@@ -437,7 +433,7 @@ public abstract class DynamicRegionFactory {
         // error condition, so you also need to check to see if the JVM
         // is still usable:
         SystemFailure.checkFailure();
-        this.c.getLoggerI18n().warning(
+        this.cache.getLoggerI18n().warning(
             LocalizedStrings.DynamicRegionFactory_DYNAMICREGIONLISTENER__0__THREW_EXCEPTION_ON_AFTERREGIONCREATED,
             listener, t);
       }
@@ -453,10 +449,10 @@ public abstract class DynamicRegionFactory {
       op = expiration ? Operation.REGION_EXPIRE_DESTROY : Operation.REGION_DESTROY;
     }
     RegionEvent event = new RegionEventImpl(region, op, null, isOriginRemote, getMember(mbr));
-    for (Iterator i = regionListeners.iterator(); i.hasNext();) {
-      DynamicRegionListener listener = (DynamicRegionListener) i.next();
+    for (Object regionListener : regionListeners) {
+      DynamicRegionListener listener = (DynamicRegionListener) regionListener;
       try {
-        listener.beforeRegionDestroy(event /* fullRegionName */ );
+        listener.beforeRegionDestroy(event /* fullRegionName */);
       } catch (VirtualMachineError err) {
         SystemFailure.initiateFailure(err);
         // If this ever returns, rethrow the error. We're poisoned
@@ -469,7 +465,7 @@ public abstract class DynamicRegionFactory {
         // error condition, so you also need to check to see if the JVM
         // is still usable:
         SystemFailure.checkFailure();
-        this.c.getLoggerI18n().warning(
+        this.cache.getLoggerI18n().warning(
             LocalizedStrings.DynamicRegionFactory_DYNAMICREGIONLISTENER__0__THREW_EXCEPTION_ON_BEFOREREGIONDESTROYED,
             listener, t);
       }
@@ -485,10 +481,10 @@ public abstract class DynamicRegionFactory {
       op = expiration ? Operation.REGION_EXPIRE_DESTROY : Operation.REGION_DESTROY;
     }
     RegionEvent event = new RegionEventImpl(region, op, null, isOriginRemote, getMember(mbr));
-    for (Iterator i = regionListeners.iterator(); i.hasNext();) {
-      DynamicRegionListener listener = (DynamicRegionListener) i.next();
+    for (Object regionListener : regionListeners) {
+      DynamicRegionListener listener = (DynamicRegionListener) regionListener;
       try {
-        listener.afterRegionDestroy(event /* fullRegionName */ );
+        listener.afterRegionDestroy(event /* fullRegionName */);
       } catch (VirtualMachineError err) {
         SystemFailure.initiateFailure(err);
         // If this ever returns, rethrow the error. We're poisoned
@@ -501,7 +497,7 @@ public abstract class DynamicRegionFactory {
         // error condition, so you also need to check to see if the JVM
         // is still usable:
         SystemFailure.checkFailure();
-        this.c.getLoggerI18n().warning(
+        this.cache.getLoggerI18n().warning(
             LocalizedStrings.DynamicRegionFactory_DYNAMICREGIONLISTENER__0__THREW_EXCEPTION_ON_AFTERREGIONDESTROYED,
             listener, t);
       }
@@ -511,27 +507,25 @@ public abstract class DynamicRegionFactory {
   /** return the argument, or if null the DistributedMember id of this vm */
   private DistributedMember getMember(DistributedMember mbr) {
     if (mbr == null) {
-      return InternalDistributedSystem.getAnyInstance().getDistributedMember();
+      return this.cache.getInternalDistributedSystem().getDistributedMember();
     } else {
       return null;
     }
   }
 
-
   /**
    * Creates the dynamic Region in the local cache and distributes the creation to other caches.
    * 
    * @param parentRegionName the new region is created as a subregion of the region having this path
    * @param regionName the name of the new subregion
-   * @return the <code>Region</code> created
-   * @throws CacheException
+   * @return the {@code Region} created
    */
   public Region createDynamicRegion(String parentRegionName, String regionName)
       throws CacheException {
     if (isClosed()) {
       throw new IllegalStateException("Dynamic region factory is closed");
     }
-    doBeforeRegionCreated(parentRegionName, regionName, null);
+    doBeforeRegionCreated(parentRegionName, regionName);
     Region region = createDynamicRegionImpl(parentRegionName, regionName, true);
     doAfterRegionCreated(region, false, false, null);
     return region;
@@ -540,13 +534,12 @@ public abstract class DynamicRegionFactory {
   /**
    * Destroys the dynamic Region in the local cache and distributes the destruction to other caches.
    * 
-   * @param fullRegionName The full path of the <code>Region</code> to be dynamically destroyed
-   * @throws CacheException
+   * @param fullRegionName The full path of the {@code Region} to be dynamically destroyed
    * @throws RegionDestroyedException if the dynamic region was never created or has already been
    *         destroyed
    */
   public void destroyDynamicRegion(String fullRegionName) throws CacheException {
-    if (!dynamicRegionList.containsKey(fullRegionName)) {
+    if (!this.dynamicRegionList.containsKey(fullRegionName)) {
       throw new RegionDestroyedException(
           LocalizedStrings.DynamicRegionFactory_DYNAMIC_REGION_0_HAS_NOT_BEEN_CREATED
               .toLocalizedString(fullRegionName),
@@ -557,7 +550,7 @@ public abstract class DynamicRegionFactory {
     }
 
     // Retrieve the region to destroy
-    Region region = c.getRegion(fullRegionName);
+    Region region = this.cache.getRegion(fullRegionName);
     if (region != null) {
       DistributedMember mbr = getMember(null);
       doBeforeRegionDestroyed(region, false, false, false, mbr);
@@ -575,15 +568,14 @@ public abstract class DynamicRegionFactory {
   private Region createDynamicRegionImpl(String parentRegionName, String newRegionName,
       boolean addEntry) throws CacheException {
 
-    Region parentRegion = c.getRegion(parentRegionName);
-    Region newRegion = null;
+    Region parentRegion = this.cache.getRegion(parentRegionName);
 
     if (parentRegion == null) {
       String errMsg =
           LocalizedStrings.DynamicRegionFactory_ERROR__COULD_NOT_FIND_A_REGION_NAMED___0_
               .toLocalizedString(parentRegionName);
       RegionDestroyedException e = new RegionDestroyedException(errMsg, parentRegionName);
-      c.getLoggerI18n().warning(
+      this.cache.getLoggerI18n().warning(
           LocalizedStrings.DynamicRegionFactory_ERROR__COULD_NOT_FIND_A_REGION_NAMED___0_,
           parentRegionName, e);
       throw e;
@@ -591,51 +583,46 @@ public abstract class DynamicRegionFactory {
 
     // Create RegionAttributes by inheriting from the parent
     RegionAttributes rra = parentRegion.getAttributes();
-    RegionAttributes newRegionAttributes = null;
 
     AttributesFactory af = new AttributesFactory(rra);
-    {
-      EvictionAttributes ev = rra.getEvictionAttributes();
-      if (ev != null && ev.getAlgorithm().isLRU()) {
-        EvictionAttributes rev = new EvictionAttributesImpl((EvictionAttributesImpl) ev);
-        af.setEvictionAttributes(rev);
-      }
+    EvictionAttributes ev = rra.getEvictionAttributes();
+    if (ev != null && ev.getAlgorithm().isLRU()) {
+      EvictionAttributes rev = new EvictionAttributesImpl((EvictionAttributesImpl) ev);
+      af.setEvictionAttributes(rev);
     }
 
     // for internal testing, until partitioned regions support subclasses or
     // DynamicRegion implementation is redone to not inherit attrs from parent
-    // regions [bruce]
+    // regions
     if (newRegionName.endsWith("_PRTEST_")) {
-      af.setPartitionAttributes((new PartitionAttributesFactory()).create());
+      af.setPartitionAttributes(new PartitionAttributesFactory().create());
     }
 
-    newRegionAttributes = af.create();
+    RegionAttributes newRegionAttributes = af.create();
 
+    Region newRegion;
     try {
       newRegion = parentRegion.createSubregion(newRegionName, newRegionAttributes);
-      c.getLoggerI18n().fine("Created dynamic region " + newRegion);
+      this.cache.getLoggerI18n().fine("Created dynamic region " + newRegion);
     } catch (RegionExistsException ex) {
       // a race condition exists that can cause this so just fine log it
-      c.getLoggerI18n().fine(
+      this.cache.getLoggerI18n().fine(
           "DynamicRegion " + newRegionName + " in parent " + parentRegionName + " already existed");
       newRegion = ex.getRegion();
-      // } catch ( CacheException e ) {
-      // c.getLoggerI18n().warning ( "Error creating new Dynamic Region '" + newRegionName, e );
-      // throw e;
     }
 
     if (addEntry) {
       DynamicRegionAttributes dra = new DynamicRegionAttributes();
       dra.name = newRegionName;
       dra.rootRegionName = parentRegion.getFullPath();
-      if (c.getLoggerI18n().fineEnabled()) {
-        c.getLoggerI18n()
+      if (this.cache.getLoggerI18n().fineEnabled()) {
+        this.cache.getLoggerI18n()
             .fine("Putting entry into dynamic region list at key: " + newRegion.getFullPath());
       }
-      dynamicRegionList.put(newRegion.getFullPath(), dra);
+      this.dynamicRegionList.put(newRegion.getFullPath(), dra);
     }
 
-    if (config.getRegisterInterest()) {
+    if (this.config.getRegisterInterest()) {
       ServerRegionProxy proxy = ((LocalRegion) newRegion).getServerProxy();
       if (proxy != null) {
         if (((Pool) proxy.getPool()).getSubscriptionEnabled()) {
@@ -643,7 +630,7 @@ public abstract class DynamicRegionFactory {
             newRegion.registerInterest("ALL_KEYS");
           } catch (GemFireSecurityException ex) {
             // Ignore security exceptions here
-            c.getSecurityLoggerI18n().warning(
+            this.cache.getSecurityLoggerI18n().warning(
                 LocalizedStrings.DynamicRegionFactory_EXCEPTION_WHEN_REGISTERING_INTEREST_FOR_ALL_KEYS_IN_DYNAMIC_REGION_0_1,
                 new Object[] {newRegion.getFullPath(), ex});
           }
@@ -654,13 +641,13 @@ public abstract class DynamicRegionFactory {
     if (regionCreateSleepMillis > 0) {
       try {
         Thread.sleep(regionCreateSleepMillis);
-      } catch (InterruptedException e) {
+      } catch (InterruptedException ignore) {
         Thread.currentThread().interrupt();
       }
     }
 
-    if (c.getLoggerI18n().fineEnabled()) {
-      c.getLoggerI18n().fine("Created Dynamic Region " + newRegion.getFullPath());
+    if (this.cache.getLoggerI18n().fineEnabled()) {
+      this.cache.getLoggerI18n().fine("Created Dynamic Region " + newRegion.getFullPath());
     }
     return newRegion;
   }
@@ -668,30 +655,30 @@ public abstract class DynamicRegionFactory {
   private void destroyDynamicRegionImpl(String fullRegionName) throws CacheException {
     // Destroy the entry in the dynamicRegionList
     try {
-      if (c.getLoggerI18n().fineEnabled()) {
-        c.getLoggerI18n()
+      if (this.cache.getLoggerI18n().fineEnabled()) {
+        this.cache.getLoggerI18n()
             .fine("Destroying entry from dynamic region list at key: " + fullRegionName);
       }
-      dynamicRegionList.destroy(fullRegionName);
+      this.dynamicRegionList.destroy(fullRegionName);
     } catch (CacheException e) {
-      c.getLoggerI18n().warning(
+      this.cache.getLoggerI18n().warning(
           LocalizedStrings.DynamicRegionFactory_ERROR_DESTROYING_DYNAMIC_REGION__0, fullRegionName,
           e);
       throw e;
     }
 
-    if (c.getLoggerI18n().fineEnabled()) {
-      c.getLoggerI18n().fine("Destroyed Dynamic Region " + fullRegionName);
+    if (this.cache.getLoggerI18n().fineEnabled()) {
+      this.cache.getLoggerI18n().fine("Destroyed Dynamic Region " + fullRegionName);
     }
   }
 
   /**
    * Configuration for dynamic region factory. The default attributes are:
    * <ul>
-   * <li>diskDir: <code>null</code>
-   * <li>poolName: <code>null</code>
-   * <li>persistBackup: <code>true</code>
-   * <li>registerInterest: <code>true</code>
+   * <li>diskDir: {@code null}
+   * <li>poolName: {@code null}
+   * <li>persistBackup: {@code true}
+   * <li>registerInterest: {@code true}
    * </ul>
    * 
    * @since GemFire 4.3
@@ -795,8 +782,8 @@ public abstract class DynamicRegionFactory {
 
     /**
      * Returns the disk directory that the dynamic region factory data will be written to. Returns
-     * null if no directory has been specified. The diskDir is only used if
-     * <code>persistBackup</code> is true.
+     * null if no directory has been specified. The diskDir is only used if {@code persistBackup} is
+     * true.
      */
     public File getDiskDir() {
       return this.diskDir;
@@ -834,11 +821,11 @@ public abstract class DynamicRegionFactory {
     String newRegionName = dra.name;
 
     try {
-      doBeforeRegionCreated(parentRegionName, newRegionName, event.getDistributedMember());
+      doBeforeRegionCreated(parentRegionName, newRegionName);
       Region region = createDynamicRegionImpl(parentRegionName, newRegionName, false);
       doAfterRegionCreated(region, true, true, event.getDistributedMember());
     } catch (Exception e) {
-      c.getLoggerI18n().warning(
+      cache.getLoggerI18n().warning(
           LocalizedStrings.DynamicRegionFactory_ERROR_ATTEMPTING_TO_LOCALLY_CREATE_DYNAMIC_REGION__0,
           newRegionName, e);
     }
@@ -852,12 +839,12 @@ public abstract class DynamicRegionFactory {
     // "local" events. If this is a true local then c.getRegion will return
     // null and this code will do nothing.
     // When bug 35644 fixed the following "if" can be uncommented.
-    // // Ignore the callback if it originated in this process (because the region
-    // // will already have been destroyed)
-    // if ( !event.isOriginRemote() && !(event instanceof BridgeEntryEventImpl)) return;
+
+    // Ignore the callback if it originated in this process (because the region
+    // will already have been destroyed)
 
     String fullRegionName = (String) event.getKey();
-    Region drRegion = c.getRegion(fullRegionName);
+    Region drRegion = cache.getRegion(fullRegionName);
     if (drRegion != null) {
       try {
         doBeforeRegionDestroyed(drRegion, true, event.getOperation().isDistributed(),
@@ -866,49 +853,19 @@ public abstract class DynamicRegionFactory {
         doAfterRegionDestroyed(drRegion, true, event.getOperation().isDistributed(),
             event.getOperation().isExpiration(), event.getDistributedMember());
       } catch (Exception e) {
-        c.getLoggerI18n().warning(
+        cache.getLoggerI18n().warning(
             LocalizedStrings.DynamicRegionFactory_ERROR_ATTEMPTING_TO_LOCALLY_DESTROY_DYNAMIC_REGION__0,
             fullRegionName, e);
       }
     }
   }
 
-  // private class DRListener implements CacheListener {
-  // public void afterCreate(EntryEvent arg0) {
-  // buildDynamicRegion(arg0);
-  // }
-  //
-  // public void afterDestroy(EntryEvent arg0) {
-  // razeDynamicRegion(arg0);
-  // }
-  //
-  // public void afterInvalidate(EntryEvent arg0) {
-  // // Stub, nothing to do.
-  // }
-  //
-  // public void afterRegionDestroy(RegionEvent arg0) {
-  // // Stub, nothing to do.
-  // }
-  //
-  // public void afterRegionInvalidate(RegionEvent arg0) {
-  // // Stub, nothing to do.
-  // }
-  //
-  // public void afterUpdate(EntryEvent arg0) {
-  // // Stub, nothing to do.
-  // }
-  //
-  // public void close() {
-  // // Stub, nothing to do.
-  // }
-  // }
-
   // Introduced to keep symmetry with DistributedMetaRegion and potentially provide improved control
   // of
   // the meta data
   private class LocalMetaRegion extends LocalRegion {
     protected LocalMetaRegion(RegionAttributes attrs, InternalRegionArguments ira) {
-      super(dynamicRegionListName, attrs, null, DynamicRegionFactory.this.c, ira);
+      super(dynamicRegionListName, attrs, null, DynamicRegionFactory.this.cache, ira);
       Assert.assertTrue(attrs.getScope().isLocal());
     }
 
@@ -1003,9 +960,9 @@ public abstract class DynamicRegionFactory {
           if (!event.isBulkOpInProgress()) {
             try {
               entry.dispatchListenerEvents(event);
-            } catch (InterruptedException ie) {
+            } catch (InterruptedException ignore) {
               Thread.currentThread().interrupt();
-              stopper.checkCancelInProgress(null);
+              this.stopper.checkCancelInProgress(null);
             }
           }
         }
@@ -1017,7 +974,7 @@ public abstract class DynamicRegionFactory {
   // distribution and notification order on the BridgeServer
   private class DistributedMetaRegion extends DistributedRegion {
     protected DistributedMetaRegion(RegionAttributes attrs) {
-      super(dynamicRegionListName, attrs, null, DynamicRegionFactory.this.c,
+      super(dynamicRegionListName, attrs, null, DynamicRegionFactory.this.cache,
           new InternalRegionArguments());
     }
 
@@ -1115,9 +1072,9 @@ public abstract class DynamicRegionFactory {
           if (!event.isBulkOpInProgress()) {
             try {
               entry.dispatchListenerEvents(event);
-            } catch (InterruptedException ie) {
+            } catch (InterruptedException ignore) {
               Thread.currentThread().interrupt();
-              stopper.checkCancelInProgress(null);
+              this.stopper.checkCancelInProgress(null);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/GemFireCache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/GemFireCache.java b/geode-core/src/main/java/org/apache/geode/cache/GemFireCache.java
index e60bc59..f2a75db 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/GemFireCache.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/GemFireCache.java
@@ -43,8 +43,8 @@ import org.apache.geode.pdx.PdxSerializer;
  */
 public interface GemFireCache extends RegionService {
   /**
-   * Returns the name of this cache. This method does not throw <code>CacheClosedException</code> if
-   * the cache is closed.
+   * Returns the name of this cache. This method does not throw {@code CacheClosedException} if the
+   * cache is closed.
    * 
    * @return the String name of this cache
    */
@@ -52,14 +52,14 @@ public interface GemFireCache extends RegionService {
 
   /**
    * Returns the distributed system used by this cache. This method does not throw
-   * <code>CacheClosedException</code> if the cache is closed.
+   * {@code CacheClosedException} if the cache is closed.
    */
   public DistributedSystem getDistributedSystem();
 
   /**
-   * Returns the <code>ResourceManager</code> for managing this cache's resources.
+   * Returns the {@code ResourceManager} for managing this cache's resources.
    * 
-   * @return <code>ResourceManager</code> for managing this cache's resources
+   * @return {@code ResourceManager} for managing this cache's resources
    * @since GemFire 6.0
    */
   public ResourceManager getResourceManager();
@@ -81,8 +81,8 @@ public interface GemFireCache extends RegionService {
   public boolean getCopyOnRead();
 
   /**
-   * Returns the <code>RegionAttributes</code> with the given <code>id</code> or <code>null</code>
-   * if no <code>RegionAttributes</code> with that id exists.
+   * Returns the {@code RegionAttributes} with the given {@code id} or {@code null} if no
+   * {@code RegionAttributes} with that id exists.
    *
    * @see #setRegionAttributes
    *
@@ -91,14 +91,13 @@ public interface GemFireCache extends RegionService {
   public <K, V> RegionAttributes<K, V> getRegionAttributes(String id);
 
   /**
-   * Sets the <code>id</code> of the given <code>RegionAttributes</code>. If a region attributes
-   * named <code>name</code> already exists, the mapping will be overwritten with
-   * <code>attrs</code>. However, changing the mapping will not effect existing regions.
+   * Sets the {@code id} of the given {@code RegionAttributes}. If a region attributes named
+   * {@code name} already exists, the mapping will be overwritten with {@code attrs}. However,
+   * changing the mapping will not effect existing regions.
    *
    * @param id The id of the region attributes
-   * @param attrs The attributes to associate with <code>id</code>. If <code>attrs</code> is
-   *        <code>null</code>, any existing <code>RegionAttributes</code> associated with
-   *        <code>id</code> will be removed.
+   * @param attrs The attributes to associate with {@code id}. If {@code attrs} is {@code null}, any
+   *        existing {@code RegionAttributes} associated with {@code id} will be removed.
    *
    * @see #getRegionAttributes
    *
@@ -122,13 +121,12 @@ public interface GemFireCache extends RegionService {
    *
    * <P>
    *
-   * Because this method may perform a {@link Region#put(Object, Object) put} on a
-   * <code>Region</code>, it declares that it throws a <code>TimeoutException</code>,
-   * <code>CacheWriterException</code>, <code>GatewayException</code>, or
-   * <code>RegionExistsException</code>.
+   * Because this method may perform a {@link Region#put(Object, Object) put} on a {@code Region},
+   * it declares that it throws a {@code TimeoutException}, {@code CacheWriterException},
+   * {@code GatewayException}, or {@code RegionExistsException}.
    *
-   * @throws CacheXmlException If the XML read from <code>is</code> does not conform to the dtd or
-   *         if an <code>IOException</code> occurs while reading the XML.
+   * @throws CacheXmlException If the XML read from {@code is} does not conform to the dtd or if an
+   *         {@code IOException} occurs while reading the XML.
    *
    * @since GemFire 4.1
    */
@@ -136,8 +134,8 @@ public interface GemFireCache extends RegionService {
       throws TimeoutException, CacheWriterException, GatewayException, RegionExistsException;
 
   /**
-   * Gets the logging object for GemFire. This method does not throw
-   * <code>CacheClosedException</code> if the cache is closed.
+   * Gets the logging object for GemFire. This method does not throw {@code CacheClosedException} if
+   * the cache is closed.
    * 
    * @return the logging object
    */
@@ -145,17 +143,17 @@ public interface GemFireCache extends RegionService {
 
   /**
    * Gets the security logging object for GemFire. This method does not throw
-   * <code>CacheClosedException</code> if the cache is closed.
+   * {@code CacheClosedException} if the cache is closed.
    * 
    * @return the security logging object
    */
   public LogWriter getSecurityLogger();
 
   /**
-   * Returns the DiskStore by name or <code>null</code> if no disk store is found.
+   * Returns the DiskStore by name or {@code null} if no disk store is found.
    * 
-   * @param name the name of the disk store to find. If <code>null</code> then the default disk
-   *        store, if it exists, is returned.
+   * @param name the name of the disk store to find. If {@code null} then the default disk store, if
+   *        it exists, is returned.
    * @since GemFire 6.5
    */
   public DiskStore findDiskStore(String name);
@@ -237,15 +235,15 @@ public interface GemFireCache extends RegionService {
   public Context getJNDIContext();
 
   /**
-   * Returns the Declarable used to initialize this cache or <code>null</code> if it does not have
-   * an initializer.
+   * Returns the Declarable used to initialize this cache or {@code null} if it does not have an
+   * initializer.
    * 
    * @since GemFire 6.6
    */
   public Declarable getInitializer();
 
   /**
-   * Returns the Properties used to initialize the cache initializer or <code>null</code> if no
+   * Returns the Properties used to initialize the cache initializer or {@code null} if no
    * initializer properties exist.
    * 
    * @since GemFire 6.6

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/Region.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/Region.java b/geode-core/src/main/java/org/apache/geode/cache/Region.java
index 4418d89..0fa29c6 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/Region.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/Region.java
@@ -1026,6 +1026,7 @@ public interface Region<K, V> extends ConcurrentMap<K, V> {
    * @return the Cache
    * @deprecated as of 6.5 use {@link #getRegionService()} instead.
    */
+  @Deprecated
   public Cache getCache();
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/RegionFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/RegionFactory.java b/geode-core/src/main/java/org/apache/geode/cache/RegionFactory.java
index 3a2e9f6..90e182f 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/RegionFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/RegionFactory.java
@@ -20,12 +20,12 @@ import java.util.Properties;
 import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.compression.Compressor;
 import org.apache.geode.distributed.LeaseExpiredException;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
 /**
- * <code>RegionFactory</code> is used to create {@link Region regions} in a {@link Cache cache}.
+ * {@code RegionFactory} is used to create {@link Region regions} in a {@link Cache cache}.
  * Instances of this interface can be created:
  * <ul>
  * <li>using a {@link RegionShortcut shortcut} by calling
@@ -63,17 +63,16 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  *
  * @since GemFire 5.0
  */
-
 public class RegionFactory<K, V> {
   private final AttributesFactory<K, V> attrsFactory;
-  private final GemFireCacheImpl cache;
+  private final InternalCache cache;
 
   /**
    * For internal use only.
    * 
    * @since GemFire 6.5
    */
-  protected RegionFactory(GemFireCacheImpl cache) {
+  protected RegionFactory(InternalCache cache) {
     this.cache = cache;
     this.attrsFactory = new AttributesFactory<K, V>();
   }
@@ -83,7 +82,7 @@ public class RegionFactory<K, V> {
    * 
    * @since GemFire 6.5
    */
-  protected RegionFactory(GemFireCacheImpl cache, RegionShortcut pra) {
+  protected RegionFactory(InternalCache cache, RegionShortcut pra) {
     this.cache = cache;
     RegionAttributes ra = cache.getRegionAttributes(pra.toString());
     if (ra == null) {
@@ -97,7 +96,7 @@ public class RegionFactory<K, V> {
    * 
    * @since GemFire 6.5
    */
-  protected RegionFactory(GemFireCacheImpl cache, RegionAttributes ra) {
+  protected RegionFactory(InternalCache cache, RegionAttributes ra) {
     this.cache = cache;
     this.attrsFactory = new AttributesFactory<K, V>(ra);
   }
@@ -107,7 +106,7 @@ public class RegionFactory<K, V> {
    * 
    * @since GemFire 6.5
    */
-  protected RegionFactory(GemFireCacheImpl cache, String regionAttributesId) {
+  protected RegionFactory(InternalCache cache, String regionAttributesId) {
     this.cache = cache;
     RegionAttributes<K, V> ra = getCache().getRegionAttributes(regionAttributesId);
     if (ra == null) {
@@ -127,7 +126,7 @@ public class RegionFactory<K, V> {
    */
   @Deprecated
   public RegionFactory() throws CacheWriterException, RegionExistsException, TimeoutException {
-    this((GemFireCacheImpl) new CacheFactory().create());
+    this((InternalCache) new CacheFactory().create());
   }
 
   /**
@@ -142,7 +141,7 @@ public class RegionFactory<K, V> {
   @Deprecated
   public RegionFactory(RegionAttributes<K, V> regionAttributes)
       throws CacheWriterException, RegionExistsException, TimeoutException {
-    this((GemFireCacheImpl) new CacheFactory().create(), regionAttributes);
+    this((InternalCache) new CacheFactory().create(), regionAttributes);
   }
 
   /**
@@ -160,7 +159,7 @@ public class RegionFactory<K, V> {
   @Deprecated
   public RegionFactory(String regionAttributesId)
       throws CacheWriterException, RegionExistsException, TimeoutException {
-    this((GemFireCacheImpl) new CacheFactory().create(), regionAttributesId);
+    this((InternalCache) new CacheFactory().create(), regionAttributesId);
   }
 
   /**
@@ -170,7 +169,7 @@ public class RegionFactory<K, V> {
    * used.
    *
    * @param distributedSystemProperties an instance of Properties containing
-   *        <code>DistributedSystem</code configuration
+   *        {@code DistributedSystem} configuration
    * @throws CacheException if unable to connect the DistributedSystem or create a Cache
    * @deprecated as of 6.5 use {@link CacheFactory#CacheFactory(Properties)} and
    *             {@link Cache#createRegionFactory()} instead.
@@ -178,7 +177,7 @@ public class RegionFactory<K, V> {
   @Deprecated
   public RegionFactory(Properties distributedSystemProperties)
       throws CacheWriterException, RegionExistsException, TimeoutException {
-    this((GemFireCacheImpl) new CacheFactory(distributedSystemProperties).create());
+    this((InternalCache) new CacheFactory(distributedSystemProperties).create());
   }
 
   /**
@@ -198,8 +197,7 @@ public class RegionFactory<K, V> {
   public RegionFactory(Properties distributedSystemProperties,
       RegionAttributes<K, V> regionAttributes)
       throws CacheWriterException, RegionExistsException, TimeoutException {
-    this((GemFireCacheImpl) new CacheFactory(distributedSystemProperties).create(),
-        regionAttributes);
+    this((InternalCache) new CacheFactory(distributedSystemProperties).create(), regionAttributes);
   }
 
   /**
@@ -222,19 +220,19 @@ public class RegionFactory<K, V> {
   @Deprecated
   public RegionFactory(Properties distributedSystemProperties, String regionAttributesId)
       throws CacheWriterException, RegionExistsException, TimeoutException {
-    this((GemFireCacheImpl) new CacheFactory(distributedSystemProperties).create(),
+    this((InternalCache) new CacheFactory(distributedSystemProperties).create(),
         regionAttributesId);
   }
 
   /**
    * Returns the cache used by this factory.
    */
-  private synchronized GemFireCacheImpl getCache() {
+  private synchronized InternalCache getCache() {
     return this.cache;
   }
 
   /**
-   * Sets the cache loader for the next <code>RegionAttributes</code> created.
+   * Sets the cache loader for the next {@code RegionAttributes} created.
    *
    * @param cacheLoader the cache loader or null if no loader
    * @return a reference to this RegionFactory object
@@ -247,7 +245,7 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the cache writer for the next <code>RegionAttributes</code> created.
+   * Sets the cache writer for the next {@code RegionAttributes} created.
    *
    * @param cacheWriter the cache writer or null if no cache writer
    * @return a reference to this RegionFactory object
@@ -263,7 +261,7 @@ public class RegionFactory<K, V> {
    * 
    * @param aListener the cache listener to add
    * @return a reference to this RegionFactory object
-   * @throws IllegalArgumentException if <code>aListener</code> is null
+   * @throws IllegalArgumentException if {@code aListener} is null
    * @see AttributesFactory#addCacheListener
    */
   public RegionFactory<K, V> addCacheListener(CacheListener<K, V> aListener) {
@@ -273,11 +271,11 @@ public class RegionFactory<K, V> {
 
   /**
    * Removes all cache listeners and then adds each listener in the specified array. for the next
-   * <code>RegionAttributes</code> created.
+   * {@code RegionAttributes} created.
    * 
    * @param newListeners a possibly null or empty array of listeners to add to this factory.
    * @return a reference to this RegionFactory object
-   * @throws IllegalArgumentException if the <code>newListeners</code> array has a null element
+   * @throws IllegalArgumentException if the {@code newListeners} array has a null element
    * @see AttributesFactory#initCacheListeners
    */
   public RegionFactory<K, V> initCacheListeners(CacheListener<K, V>[] newListeners) {
@@ -298,8 +296,8 @@ public class RegionFactory<K, V> {
 
   /**
    * Sets the idleTimeout expiration attributes for region entries for the next
-   * <code>RegionAttributes</code> created. Note that the XML element that corresponds to this
-   * method "entry-idle-time", does not include "out" in its name.
+   * {@code RegionAttributes} created. Note that the XML element that corresponds to this method
+   * "entry-idle-time", does not include "out" in its name.
    *
    * @param idleTimeout the idleTimeout ExpirationAttributes for entries in this region
    * @return a reference to this RegionFactory object
@@ -312,7 +310,7 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the custom idleTimeout for the next <code>RegionAttributes</code> created.
+   * Sets the custom idleTimeout for the next {@code RegionAttributes} created.
    * 
    * @param custom the custom method
    * @return the receiver
@@ -325,7 +323,7 @@ public class RegionFactory<K, V> {
 
   /**
    * Sets the timeToLive expiration attributes for region entries for the next
-   * <code>RegionAttributes</code> created.
+   * {@code RegionAttributes} created.
    *
    * @param timeToLive the timeToLive ExpirationAttributes for entries in this region
    * @return a reference to this RegionFactory object
@@ -338,8 +336,7 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the custom timeToLive expiration method for the next <code>RegionAttributes</code>
-   * created.
+   * Sets the custom timeToLive expiration method for the next {@code RegionAttributes} created.
    * 
    * @param custom the custom method
    * @return the receiver
@@ -352,8 +349,8 @@ public class RegionFactory<K, V> {
 
   /**
    * Sets the idleTimeout expiration attributes for the region itself for the next
-   * <code>RegionAttributes</code> created. Note that the XML element that corresponds to this
-   * method "region-idle-time", does not include "out" in its name.
+   * {@code RegionAttributes} created. Note that the XML element that corresponds to this method
+   * "region-idle-time", does not include "out" in its name.
    *
    * @param idleTimeout the ExpirationAttributes for this region idleTimeout
    * @return a reference to this RegionFactory object
@@ -367,7 +364,7 @@ public class RegionFactory<K, V> {
 
   /**
    * Sets the timeToLive expiration attributes for the region itself for the next
-   * <code>RegionAttributes</code> created.
+   * {@code RegionAttributes} created.
    *
    * @param timeToLive the ExpirationAttributes for this region timeToLive
    * @return a reference to this RegionFactory object
@@ -380,7 +377,7 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the scope for the next <code>RegionAttributes</code> created.
+   * Sets the scope for the next {@code RegionAttributes} created.
    *
    * @param scopeType the type of Scope to use for the region
    * @return a reference to this RegionFactory object
@@ -393,7 +390,7 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the data policy for the next <code>RegionAttributes</code> created.
+   * Sets the data policy for the next {@code RegionAttributes} created.
    *
    * @param dataPolicy The type of mirroring to use for the region
    * @return a reference to this RegionFactory object
@@ -436,15 +433,14 @@ public class RegionFactory<K, V> {
    * Sets the pool name attribute. This causes regions that use these attributes to be a client
    * region which communicates with the servers that the connection pool communicates with.
    * <p>
-   * If this attribute is set to <code>null</code> or <code>""</code> then the connection pool is
-   * disabled causing regions that use these attributes to be communicate with peers instead of
-   * servers.
+   * If this attribute is set to {@code null} or {@code ""} then the connection pool is disabled
+   * causing regions that use these attributes to be communicate with peers instead of servers.
    * <p>
    * The named connection pool must exist on the cache at the time these attributes are used to
    * create a region. See {@link PoolManager#createFactory} for how to create a connection pool.
    * 
-   * @param poolName the name of the connection pool to use; if <code>null</code> or <code>""</code>
-   *        then the connection pool attribute is disabled for regions using these attributes.
+   * @param poolName the name of the connection pool to use; if {@code null} or {@code ""} then the
+   *        connection pool attribute is disabled for regions using these attributes.
    * @return a reference to this RegionFactory object
    * @throws IllegalStateException if a cache loader or cache writer has already been set.
    * @since GemFire 5.7
@@ -491,14 +487,13 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the key constraint for the next <code>RegionAttributes</code> created. Keys in the region
-   * will be constrained to this class (or subclass). Any attempt to store a key of an incompatible
-   * type in the region will cause a <code>ClassCastException</code> to be thrown.
+   * Sets the key constraint for the next {@code RegionAttributes} created. Keys in the region will
+   * be constrained to this class (or subclass). Any attempt to store a key of an incompatible type
+   * in the region will cause a {@code ClassCastException} to be thrown.
    *
    * @param keyConstraint The Class to constrain the keys to, or null if no constraint
    * @return a reference to this RegionFactory object
-   * @throws IllegalArgumentException if <code>keyConstraint</code> is a class denoting a primitive
-   *         type
+   * @throws IllegalArgumentException if {@code keyConstraint} is a class denoting a primitive type
    * @see AttributesFactory#setKeyConstraint
    */
   public RegionFactory<K, V> setKeyConstraint(Class<K> keyConstraint) {
@@ -507,14 +502,14 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the value constraint for the next <code>RegionAttributes</code> created. Values in the
-   * region will be constrained to this class (or subclass). Any attempt to store a value of an
-   * incompatible type in the region will cause a <code>ClassCastException</code> to be thrown.
+   * Sets the value constraint for the next {@code RegionAttributes} created. Values in the region
+   * will be constrained to this class (or subclass). Any attempt to store a value of an
+   * incompatible type in the region will cause a {@code ClassCastException} to be thrown.
    *
    * @param valueConstraint The Class to constrain the values to, or null if no constraint
    * @return a reference to this RegionFactory object
-   * @throws IllegalArgumentException if <code>valueConstraint</code> is a class denoting a
-   *         primitive type
+   * @throws IllegalArgumentException if {@code valueConstraint} is a class denoting a primitive
+   *         type
    * @see AttributesFactory#setValueConstraint
    */
   public RegionFactory<K, V> setValueConstraint(Class<V> valueConstraint) {
@@ -523,8 +518,8 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the entry initial capacity for the next <code>RegionAttributes</code> created. This value
-   * is used in initializing the map that holds the entries.
+   * Sets the entry initial capacity for the next {@code RegionAttributes} created. This value is
+   * used in initializing the map that holds the entries.
    *
    * @param initialCapacity the initial capacity of the entry map
    * @return a reference to this RegionFactory object
@@ -538,8 +533,8 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the entry load factor for the next <code>RegionAttributes</code> created. This value is
-   * used in initializing the map that holds the entries.
+   * Sets the entry load factor for the next {@code RegionAttributes} created. This value is used in
+   * initializing the map that holds the entries.
    *
    * @param loadFactor the load factor of the entry map
    * @return a reference to this RegionFactory object
@@ -553,8 +548,8 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the concurrency level tof the next <code>RegionAttributes</code> created. This value is
-   * used in initializing the map that holds the entries.
+   * Sets the concurrency level tof the next {@code RegionAttributes} created. This value is used in
+   * initializing the map that holds the entries.
    *
    * @param concurrencyLevel the concurrency level of the entry map
    * @return a reference to this RegionFactory object
@@ -655,8 +650,8 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the <code>PartitionAttributes</code> that describe how the region is partitioned among
-   * members of the distributed system.
+   * Sets the {@code PartitionAttributes} that describe how the region is partitioned among members
+   * of the distributed system.
    *
    * @return a reference to this RegionFactory object
    * @see AttributesFactory#setPartitionAttributes
@@ -667,14 +662,15 @@ public class RegionFactory<K, V> {
   }
 
   /**
-   * Sets the <code>MembershipAttributes</code> that describe the membership roles required for
-   * reliable access to the region.
+   * Sets the {@code MembershipAttributes} that describe the membership roles required for reliable
+   * access to the region.
    *
    * @param ra the MembershipAttributes to use
    * @return a reference to this RegionFactory object
    * @see AttributesFactory#setMembershipAttributes
    * @deprecated this API is scheduled to be removed
    */
+  @Deprecated
   public RegionFactory<K, V> setMembershipAttributes(MembershipAttributes ra) {
     this.attrsFactory.setMembershipAttributes(ra);
     return this;
@@ -787,7 +783,6 @@ public class RegionFactory<K, V> {
   /**
    * Sets cloning on region Note: off-heap regions always behave as if cloning is enabled.
    * 
-   * @param cloningEnable
    * @return a reference to this RegionFactory object
    * @since GemFire 6.1
    * @see AttributesFactory#setCloningEnabled
@@ -800,7 +795,6 @@ public class RegionFactory<K, V> {
   /**
    * Adds a gatewaySenderId to the RegionAttributes
    * 
-   * @param gatewaySenderId
    * @return a reference to this RegionFactory object
    * @since GemFire 7.0
    * @see AttributesFactory#addGatewaySenderId(String)

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/TransactionDataRebalancedException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/TransactionDataRebalancedException.java b/geode-core/src/main/java/org/apache/geode/cache/TransactionDataRebalancedException.java
index fded49f..5119628 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/TransactionDataRebalancedException.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/TransactionDataRebalancedException.java
@@ -32,4 +32,12 @@ public class TransactionDataRebalancedException extends TransactionException {
   public TransactionDataRebalancedException(String s) {
     super(s);
   }
+
+  public TransactionDataRebalancedException(Throwable cause) {
+    super(cause);
+  }
+
+  public TransactionDataRebalancedException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
index 1a4052b..7549a26 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
@@ -152,44 +152,12 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
         logger.debug("Creating GatewaySender that underlies the AsyncEventQueue");
       }
 
-      // TODO: Suranjan .separate asynceventqueue from gatewaysender
-      // GatewaySenderFactory senderFactory = this.cache.createGatewaySenderFactory();
-      // senderFactory.setMaximumQueueMemory(attrs.getMaximumQueueMemory());
-      // senderFactory.setBatchSize(attrs.getBatchSize());
-      // senderFactory.setBatchTimeInterval(attrs.getBatchTimeInterval());
-      // if (attrs.isPersistenceEnabled()) {
-      // senderFactory.setPersistenceEnabled(true);
-      // }
-      // senderFactory.setDiskStoreName(attrs.getDiskStoreName());
-      // senderFactory.setDiskSynchronous(attrs.isDiskSynchronous());
-      // senderFactory.setBatchConflationEnabled(attrs.isBatchConflationEnabled());
-      // senderFactory.setParallel(attrs.isParallel());
-      // senderFactory.setDispatcherThreads(attrs.getDispatcherThreads());
-      // if OrderPolicy is not null, set it, otherwise, let the default OrderPolicy take the charge
-      // if (attrs.getOrderPolicy() != null) {
-      // senderFactory.setOrderPolicy(attrs.getOrderPolicy());
-      // }
-      // for (GatewayEventFilter filter : attrs.eventFilters) {
-      // senderFactory.addGatewayEventFilter(filter);
-      // }
-      // senderFactory.setGatewayEventSubstitutionFilter(attrs.getGatewayEventSubstitutionFilter());
-      // Type cast to GatewaySenderFactory implementation impl to add the async event listener
-      // and set the isForInternalUse to true. These methods are not exposed on GatewaySenderFactory
-      // GatewaySenderFactory factoryImpl = (GatewaySenderFactoryImpl) senderFactory;
-      // senderFactory.setForInternalUse(true);
-      // senderFactory.addAsyncEventListener(listener);
-      // senderFactory.setBucketSorted(attrs.isBucketSorted());
-      // add member id to differentiate between this region and the redundant bucket
-      // region created for this queue.
-      // GatewaySender sender =
-      // senderFactory.create(
-      // AsyncEventQueueImpl.getSenderIdFromAsyncEventQueueId(asyncQueueId));
       addAsyncEventListener(listener);
       GatewaySender sender =
           create(AsyncEventQueueImpl.getSenderIdFromAsyncEventQueueId(asyncQueueId));
       AsyncEventQueueImpl queue = new AsyncEventQueueImpl(sender, listener);
       asyncEventQueue = queue;
-      ((GemFireCacheImpl) cache).addAsyncEventQueue(queue);
+      this.cache.addAsyncEventQueue(queue);
     } else if (this.cache instanceof CacheCreation) {
       asyncEventQueue = new AsyncEventQueueCreation(asyncQueueId, attrs, listener);
       ((CacheCreation) cache).addAsyncEventQueue(asyncEventQueue);
@@ -220,7 +188,7 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
 
       if (this.cache instanceof GemFireCacheImpl) {
         sender = new ParallelAsyncEventQueueImpl(this.cache, this.attrs);
-        ((GemFireCacheImpl) this.cache).addGatewaySender(sender);
+        this.cache.addGatewaySender(sender);
         if (!this.attrs.isManualStart()) {
           sender.start();
         }
@@ -229,19 +197,12 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
         ((CacheCreation) this.cache).addGatewaySender(sender);
       }
     } else {
-      // if (this.attrs.getOrderPolicy() != null) {
-      // if (this.attrs.getDispatcherThreads() == GatewaySender.DEFAULT_DISPATCHER_THREADS) {
-      // throw new AsyncEventQueueConfigurationException(
-      // LocalizedStrings.AsyncEventQueue_INVALID_ORDER_POLICY_CONCURRENCY_0
-      // .toLocalizedString(id));
-      // }
-      // }
       if (this.attrs.getOrderPolicy() == null && this.attrs.getDispatcherThreads() > 1) {
         this.attrs.policy = GatewaySender.DEFAULT_ORDER_POLICY;
       }
       if (this.cache instanceof GemFireCacheImpl) {
         sender = new SerialAsyncEventQueueImpl(this.cache, this.attrs);
-        ((GemFireCacheImpl) this.cache).addGatewaySender(sender);
+        this.cache.addGatewaySender(sender);
         if (!this.attrs.isManualStart()) {
           sender.start();
         }

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
index 0def5d2..bf7e874 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
@@ -25,7 +25,7 @@ import org.apache.geode.cache.wan.GatewayEventFilter;
 import org.apache.geode.cache.wan.GatewayEventSubstitutionFilter;
 import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.cache.wan.GatewaySender.OrderPolicy;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.RegionQueue;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor;
@@ -150,7 +150,7 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
     if (this == obj) {
       return true;
     }
-    if (!(obj instanceof AsyncEventQueue)) {
+    if (!(obj instanceof AsyncEventQueueImpl)) {
       return false;
     }
     AsyncEventQueueImpl asyncEventQueue = (AsyncEventQueueImpl) obj;
@@ -203,9 +203,9 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
   }
 
   public void destroy(boolean initiator) {
-    GemFireCacheImpl gfci = (GemFireCacheImpl) ((AbstractGatewaySender) this.sender).getCache();
+    InternalCache cache = ((AbstractGatewaySender) this.sender).getCache();
     ((AbstractGatewaySender) this.sender).destroy(initiator);
-    gfci.removeAsyncEventQueue(this);
+    cache.removeAsyncEventQueue(this);
   }
 
   public boolean isBucketSorted() {

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
index e799880..9fa15f7 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
@@ -19,23 +19,22 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.cache.EntryOperation;
 import org.apache.geode.cache.asyncqueue.AsyncEventListener;
 import org.apache.geode.cache.wan.GatewayTransportFilter;
+import org.apache.geode.distributed.internal.DistributionAdvisor.Profile;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ResourceEvent;
-import org.apache.geode.distributed.internal.DistributionAdvisor.Profile;
 import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.PartitionedRegionHelper;
 import org.apache.geode.internal.cache.UpdateAttributesProcessor;
 import org.apache.geode.internal.cache.ha.ThreadIdentifier;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor;
+import org.apache.geode.internal.cache.wan.GatewaySenderAdvisor.GatewaySenderProfile;
 import org.apache.geode.internal.cache.wan.GatewaySenderAttributes;
 import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderEventProcessor;
 import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderQueue;
-import org.apache.geode.internal.cache.wan.GatewaySenderAdvisor.GatewaySenderProfile;
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -86,7 +85,7 @@ public class ParallelAsyncEventQueueImpl extends AbstractGatewaySender {
 
       // Only notify the type registry if this is a WAN gateway queue
       if (!isAsyncEventQueue()) {
-        ((GemFireCacheImpl) getCache()).getPdxRegistry().gatewaySenderStarted(this);
+        getCache().getPdxRegistry().gatewaySenderStarted(this);
       }
       new UpdateAttributesProcessor(this).distribute(false);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
index 9252dc7..a1d933f 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
@@ -25,7 +25,6 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ResourceEvent;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.RegionQueue;
 import org.apache.geode.internal.cache.UpdateAttributesProcessor;
@@ -99,7 +98,7 @@ public class SerialAsyncEventQueueImpl extends AbstractGatewaySender {
 
       // Only notify the type registry if this is a WAN gateway queue
       if (!isAsyncEventQueue()) {
-        ((GemFireCacheImpl) getCache()).getPdxRegistry().gatewaySenderStarted(this);
+        getCache().getPdxRegistry().gatewaySenderStarted(this);
       }
       new UpdateAttributesProcessor(this).distribute(false);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java b/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
index e72cbff..0d678ca 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/ClientCacheFactory.java
@@ -12,11 +12,19 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.cache.client;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+
+import java.util.Properties;
+
 import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.CacheWriterException;
+import org.apache.geode.cache.CacheXmlException;
 import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionExistsException;
+import org.apache.geode.cache.TimeoutException;
+import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.GemFireVersion;
@@ -25,11 +33,8 @@ import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.pdx.PdxInstance;
 import org.apache.geode.pdx.PdxSerializer;
-
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.AuthenticationRequiredException;
 
 /**
  * Factory class used to create the singleton {@link ClientCache client cache} and connect to one or
@@ -209,7 +214,7 @@ public class ClientCacheFactory {
 
   private ClientCache basicCreate() {
     synchronized (ClientCacheFactory.class) {
-      GemFireCacheImpl instance = GemFireCacheImpl.getInstance();
+      InternalClientCache instance = GemFireCacheImpl.getInstance();
 
       {
         String propValue = this.dsProps.getProperty(MCAST_PORT);
@@ -224,7 +229,7 @@ public class ClientCacheFactory {
       }
       {
         String propValue = this.dsProps.getProperty(LOCATORS);
-        if (propValue != null && !propValue.equals("")) {
+        if (propValue != null && !propValue.isEmpty()) {
           throw new IllegalStateException(
               "On a client cache the locators property must be set to an empty string or not set. It was set to \""
                   + propValue + "\".");
@@ -254,8 +259,7 @@ public class ClientCacheFactory {
 
         return instance;
       } else {
-        GemFireCacheImpl gfc = GemFireCacheImpl.createClient(system, this.pf, cacheConfig);
-        return gfc;
+        return GemFireCacheImpl.createClient(system, this.pf, cacheConfig);
       }
     }
   }
@@ -606,7 +610,7 @@ public class ClientCacheFactory {
    *         ClientCacheFactory
    */
   public static synchronized ClientCache getAnyInstance() {
-    GemFireCacheImpl instance = GemFireCacheImpl.getInstance();
+    InternalClientCache instance = GemFireCacheImpl.getInstance();
     if (instance == null) {
       throw new CacheClosedException(
           LocalizedStrings.CacheFactory_A_CACHE_HAS_NOT_YET_BEEN_CREATED.toLocalizedString());

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientRegionFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientRegionFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientRegionFactoryImpl.java
index ef67667..07806d9 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientRegionFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientRegionFactoryImpl.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.cache.client.internal;
 
+import static org.apache.commons.lang.StringUtils.isEmpty;
+
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.CacheListener;
 import org.apache.geode.cache.CustomExpiry;
@@ -29,7 +31,6 @@ import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.client.Pool;
 import org.apache.geode.compression.Compressor;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.UserSpecifiedRegionAttributes;
 
@@ -41,7 +42,7 @@ import org.apache.geode.internal.cache.UserSpecifiedRegionAttributes;
 
 public class ClientRegionFactoryImpl<K, V> implements ClientRegionFactory<K, V> {
   private final AttributesFactory<K, V> attrsFactory;
-  private final GemFireCacheImpl cache;
+  private final InternalClientCache cache;
 
   /**
    * Constructs a ClientRegionFactory by creating a DistributedSystem and a Cache. If no
@@ -51,7 +52,7 @@ public class ClientRegionFactoryImpl<K, V> implements ClientRegionFactory<K, V>
    *
    * @param pra the region shortcut to use
    */
-  public ClientRegionFactoryImpl(GemFireCacheImpl cache, ClientRegionShortcut pra) {
+  public ClientRegionFactoryImpl(InternalClientCache cache, ClientRegionShortcut pra) {
     this.cache = cache;
     RegionAttributes ra = cache.getRegionAttributes(pra.toString());
     if (ra == null) {
@@ -70,7 +71,7 @@ public class ClientRegionFactoryImpl<K, V> implements ClientRegionFactory<K, V>
    *
    * @param refid the name of the region attributes to use
    */
-  public ClientRegionFactoryImpl(GemFireCacheImpl cache, String refid) {
+  public ClientRegionFactoryImpl(InternalClientCache cache, String refid) {
     this.cache = cache;
     RegionAttributes ra = cache.getRegionAttributes(refid);
     if (ra == null) {
@@ -90,7 +91,7 @@ public class ClientRegionFactoryImpl<K, V> implements ClientRegionFactory<K, V>
   /**
    * Returns the cache used by this factory.
    */
-  private GemFireCacheImpl getCache() {
+  private InternalClientCache getCache() {
     return this.cache;
   }
 
@@ -219,7 +220,7 @@ public class ClientRegionFactoryImpl<K, V> implements ClientRegionFactory<K, V>
   @SuppressWarnings("deprecation")
   private RegionAttributes<K, V> createRegionAttributes() {
     RegionAttributes<K, V> ra = this.attrsFactory.create();
-    if (ra.getPoolName() == null || "".equals(ra.getPoolName())) {
+    if (isEmpty(ra.getPoolName())) {
       UserSpecifiedRegionAttributes<K, V> ura = (UserSpecifiedRegionAttributes<K, V>) ra;
       if (ura.requiresPoolName) {
         Pool dp = getDefaultPool();
@@ -233,17 +234,4 @@ public class ClientRegionFactoryImpl<K, V> implements ClientRegionFactory<K, V>
     }
     return ra;
   }
-
-  // public ClientRegionFactory<K, V> addParallelGatewaySenderId(
-  // String parallelGatewaySenderId) {
-  // this.attrsFactory.addParallelGatewaySenderId(parallelGatewaySenderId);
-  // return this;
-  // }
-  //
-  // public ClientRegionFactory<K, V> addSerialGatewaySenderId(
-  // String serialGatewaySenderId) {
-  // this.attrsFactory.addSerialGatewaySenderId(serialGatewaySenderId);
-  // return this;
-  // }
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
index a494138..f71b79b 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
@@ -31,16 +31,16 @@ import org.apache.geode.cache.client.internal.ExecuteFunctionOp.ExecuteFunctionO
 import org.apache.geode.cache.client.internal.ExecuteRegionFunctionOp.ExecuteRegionFunctionOpImpl;
 import org.apache.geode.cache.client.internal.ExecuteRegionFunctionSingleHopOp.ExecuteRegionFunctionSingleHopOpImpl;
 import org.apache.geode.cache.wan.GatewaySender;
+import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.internal.net.SocketCreator;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.tier.sockets.HandShake;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.cache.tier.sockets.ServerQueueStatus;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+import org.apache.geode.internal.net.SocketCreator;
 
 /**
  * A single client to server connection.
@@ -49,10 +49,11 @@ import org.apache.geode.internal.logging.log4j.LocalizedMessage;
  * server connection at the same time.
  * 
  * @since GemFire 5.7
- *
  */
 public class ConnectionImpl implements Connection {
 
+  // TODO: DEFAULT_CLIENT_FUNCTION_TIMEOUT should be private
+  public static final int DEFAULT_CLIENT_FUNCTION_TIMEOUT = 0;
   private static Logger logger = LogService.getLogger();
 
   /**
@@ -61,17 +62,21 @@ public class ConnectionImpl implements Connection {
    */
   private static boolean TEST_DURABLE_CLIENT_CRASH = false;
 
+  // TODO: clientFunctionTimeout is not thread-safe and should be non-static
+  private static int clientFunctionTimeout;
+
   private Socket theSocket;
   private ByteBuffer commBuffer;
   private ByteBuffer commBufferForAsyncRead;
-  // private int handShakeTimeout = AcceptorImpl.DEFAULT_HANDSHAKE_TIMEOUT_MS;
   private ServerQueueStatus status;
   private volatile boolean connectFinished;
   private final AtomicBoolean destroyed = new AtomicBoolean();
   private Endpoint endpoint;
-  private short wanSiteVersion = -1;// In Gateway communication version of connected wan site
-                                    // will be stored after successful handshake
-  // private final CancelCriterion cancelCriterion;
+
+  // In Gateway communication version of connected wan site will be stored after successful
+  // handshake
+  private short wanSiteVersion = -1;
+
   private final InternalDistributedSystem ds;
 
   private OutputStream out;
@@ -82,8 +87,14 @@ public class ConnectionImpl implements Connection {
   private HandShake handShake;
 
   public ConnectionImpl(InternalDistributedSystem ds, CancelCriterion cancelCriterion) {
-    // this.cancelCriterion = cancelCriterion;
     this.ds = ds;
+    int time = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "CLIENT_FUNCTION_TIMEOUT",
+        DEFAULT_CLIENT_FUNCTION_TIMEOUT);
+    clientFunctionTimeout = time >= 0 ? time : DEFAULT_CLIENT_FUNCTION_TIMEOUT;
+  }
+
+  public static int getClientFunctionTimeout() {
+    return clientFunctionTimeout;
   }
 
   public ServerQueueStatus connect(EndpointManager endpointManager, ServerLocation location,
@@ -149,9 +160,7 @@ public class ConnectionImpl implements Connection {
     commBuffer = null;
     try {
       theSocket.close();
-    } catch (IOException e) {
-      // ignore
-    } catch (RuntimeException e) {
+    } catch (IOException | RuntimeException ignore) {
       // ignore
     }
   }
@@ -256,7 +265,7 @@ public class ConnectionImpl implements Connection {
       if (op instanceof ExecuteFunctionOpImpl || op instanceof ExecuteRegionFunctionOpImpl
           || op instanceof ExecuteRegionFunctionSingleHopOpImpl) {
         int earliertimeout = this.getSocket().getSoTimeout();
-        this.getSocket().setSoTimeout(GemFireCacheImpl.getClientFunctionTimeout());
+        this.getSocket().setSoTimeout(getClientFunctionTimeout());
         try {
           result = op.attempt(this);
         } finally {

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
index 54521d5..3da550a 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecutablePool.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.cache.client.internal;
 
-
 import org.apache.geode.cache.NoSubscriptionServersAvailableException;
 import org.apache.geode.cache.client.SubscriptionNotEnabledException;
 import org.apache.geode.distributed.internal.ServerLocation;

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
index d32e0f4..5e5a4e9 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
@@ -23,7 +23,6 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.cache.execute.FunctionStats;
 import org.apache.geode.internal.cache.execute.InternalFunctionException;
@@ -500,8 +499,8 @@ public class ExecuteFunctionOp {
     }
 
     private void addBytes(byte isReexecute, byte fnStateOrHasResult) {
-      if (GemFireCacheImpl
-          .getClientFunctionTimeout() == GemFireCacheImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT) {
+      if (ConnectionImpl
+          .getClientFunctionTimeout() == ConnectionImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT) {
         if (isReexecute == 1) {
           getMessage().addBytesPart(
               new byte[] {AbstractExecution.getReexecuteFunctionState(fnStateOrHasResult)});
@@ -515,7 +514,7 @@ public class ExecuteFunctionOp {
         } else {
           bytes[0] = fnStateOrHasResult;
         }
-        Part.encodeInt(GemFireCacheImpl.getClientFunctionTimeout(), bytes, 1);
+        Part.encodeInt(ConnectionImpl.getClientFunctionTimeout(), bytes, 1);
         getMessage().addBytesPart(bytes);
       }
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/d888c75e/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionOp.java
index 01f9081..70f3bbf 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionOp.java
@@ -32,7 +32,6 @@ import org.apache.geode.cache.execute.FunctionInvocationTargetException;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.cache.execute.BucketMovedException;
 import org.apache.geode.internal.cache.execute.FunctionStats;
@@ -479,13 +478,13 @@ public class ExecuteRegionFunctionOp {
     }
 
     private void addBytes(byte functionStateOrHasResult) {
-      if (GemFireCacheImpl
-          .getClientFunctionTimeout() == GemFireCacheImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT) {
+      if (ConnectionImpl
+          .getClientFunctionTimeout() == ConnectionImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT) {
         getMessage().addBytesPart(new byte[] {functionStateOrHasResult});
       } else {
         byte[] bytes = new byte[5];
         bytes[0] = functionStateOrHasResult;
-        Part.encodeInt(GemFireCacheImpl.getClientFunctionTimeout(), bytes, 1);
+        Part.encodeInt(ConnectionImpl.getClientFunctionTimeout(), bytes, 1);
         getMessage().addBytesPart(bytes);
       }
     }