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/08 23:06:24 UTC

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

http://git-wip-us.apache.org/repos/asf/geode/blob/8c2210db/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java
index ff6dd9d..b3c0df6 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java
@@ -15,31 +15,32 @@
 package org.apache.geode.admin.internal;
 
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.CancelException;
-import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.cache.DiskStore;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
 import org.apache.geode.internal.admin.remote.AdminResponse;
 import org.apache.geode.internal.admin.remote.CliLegacyMessage;
-import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
 
 /**
  * A request to from an admin VM to all non admin members to start a backup. In the prepare phase of
  * the backup, the members will suspend bucket destroys to make sure buckets aren't missed during
  * the backup.
- * 
- *
  */
 public class FlushToDiskRequest extends CliLegacyMessage {
+  private static final Logger logger = LogService.getLogger();
 
   public FlushToDiskRequest() {
-
+    // nothing
   }
 
   public static void send(DM dm, Set recipients) {
@@ -56,7 +57,7 @@ public class FlushToDiskRequest extends CliLegacyMessage {
         throw e;
       }
     } catch (InterruptedException e) {
-      e.printStackTrace();
+      logger.debug(e);
     }
     AdminResponse response = request.createResponse((DistributionManager) dm);
     response.setSender(dm.getDistributionManagerId());
@@ -65,18 +66,15 @@ public class FlushToDiskRequest extends CliLegacyMessage {
 
   @Override
   protected AdminResponse createResponse(DistributionManager dm) {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-    HashSet<PersistentID> persistentIds;
+    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache != null) {
-      Collection<DiskStoreImpl> diskStores = cache.listDiskStoresIncludingRegionOwned();
-      for (DiskStoreImpl store : diskStores) {
-        store.flush();
-      }
+      cache.listDiskStoresIncludingRegionOwned().forEach(DiskStore::flush);
     }
 
     return new FlushToDiskResponse(this.getSender());
   }
 
+  @Override
   public int getDSFID() {
     return FLUSH_TO_DISK_REQUEST;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/8c2210db/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
index 951b364..b257a17 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
@@ -14,55 +14,45 @@
  */
 package org.apache.geode.admin.internal;
 
+import java.util.List;
+
 import org.apache.geode.CancelException;
-import org.apache.geode.admin.*;
+import org.apache.geode.admin.GemFireHealthConfig;
+import org.apache.geode.admin.MemberHealthConfig;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.*;
-import org.apache.geode.internal.*;
-import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DMStats;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.CachePerfStats;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 
-import java.util.*;
-
 /**
  * Contains the logic for evaluating the health of a GemFire distributed system member according to
  * the thresholds provided in a {@link MemberHealthConfig}.
  *
- * @see VMStats
- * @see ProcessStats
- * @see DMStats
- *
- *
  * @since GemFire 3.5
  */
-/**
- *
- */
 class MemberHealthEvaluator extends AbstractHealthEvaluator {
 
   /** The config from which we get the evaluation criteria */
-  private MemberHealthConfig config;
+  private final MemberHealthConfig config;
 
   /** The description of the member being evaluated */
-  private String description;
-
-  // /** Statistics about this VM (may be null) */
-  // private VMStatsContract vmStats;
+  private final String description;
 
   /** Statistics about this process (may be null) */
   private ProcessStats processStats;
 
   /** Statistics about the distribution manager */
-  private DMStats dmStats;
+  private final DMStats dmStats;
 
   /** The previous value of the reply timeouts stat */
   private long prevReplyTimeouts;
 
-  ////////////////////// Constructors //////////////////////
-
   /**
    * Creates a new <code>MemberHealthEvaluator</code>
    */
@@ -81,7 +71,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
 
     this.dmStats = dm.getStats();
 
-    StringBuffer sb = new StringBuffer();
+    StringBuilder sb = new StringBuilder();
     sb.append("Application VM member ");
     sb.append(dm.getId());
     int pid = OSProcess.getId();
@@ -92,8 +82,6 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
     this.description = sb.toString();
   }
 
-  //////////////////// Instance Methods ////////////////////
-
   @Override
   protected String getDescription() {
     return this.description;
@@ -115,7 +103,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
     if (vmSize > threshold) {
       String s =
           LocalizedStrings.MemberHealthEvaluator_THE_SIZE_OF_THIS_VM_0_MEGABYTES_EXCEEDS_THE_THRESHOLD_1_MEGABYTES
-              .toLocalizedString(new Object[] {Long.valueOf(vmSize), Long.valueOf(threshold)});
+              .toLocalizedString(vmSize, threshold);
       status.add(okayHealth(s));
     }
   }
@@ -126,14 +114,13 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
    * {@linkplain MemberHealthConfig#getMaxMessageQueueSize threshold}. If not, the status is "okay"
    * health.
    */
-  void checkMessageQueueSize(List status) {
+  private void checkMessageQueueSize(List status) {
     long threshold = this.config.getMaxMessageQueueSize();
     long overflowSize = this.dmStats.getOverflowQueueSize();
     if (overflowSize > threshold) {
       String s =
           LocalizedStrings.MemberHealthEvaluator_THE_SIZE_OF_THE_OVERFLOW_QUEUE_0_EXCEEDS_THE_THRESHOLD_1
-              .toLocalizedString(
-                  new Object[] {Long.valueOf(overflowSize), Long.valueOf(threshold)});
+              .toLocalizedString(overflowSize, threshold);
       status.add(okayHealth(s));
     }
   }
@@ -143,7 +130,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
    * does not exceed the {@linkplain MemberHealthConfig#getMaxReplyTimeouts threshold}. If not, the
    * status is "okay" health.
    */
-  void checkReplyTimeouts(List status) {
+  private void checkReplyTimeouts(List status) {
     if (isFirstEvaluation()) {
       return;
     }
@@ -153,74 +140,45 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
     if (deltaReplyTimeouts > threshold) {
       String s =
           LocalizedStrings.MemberHealthEvaluator_THE_NUMBER_OF_MESSAGE_REPLY_TIMEOUTS_0_EXCEEDS_THE_THRESHOLD_1
-              .toLocalizedString(
-                  new Object[] {Long.valueOf(deltaReplyTimeouts), Long.valueOf(threshold)});
+              .toLocalizedString(deltaReplyTimeouts, threshold);
       status.add(okayHealth(s));
     }
   }
 
   /**
-   * See if the multicast retransmission ratio is okay
-   */
-  void checkRetransmissionRatio(List status) {
-    double threshold = this.config.getMaxRetransmissionRatio();
-    int mcastMessages = this.dmStats.getMcastWrites();
-    if (mcastMessages > 100000) { // avoid initial state & int overflow
-      // the ratio we actually use here is (retransmit requests) / (mcast datagram writes)
-      // a single retransmit request may include multiple missed messages
-      double ratio =
-          (this.dmStats.getMcastRetransmits() * 1.0) / (this.dmStats.getMcastWrites() * 1.0);
-      if (ratio > threshold) {
-        String s = "The number of message retransmissions (" + ratio + ") exceeds the threshold ("
-            + threshold + ")";
-        status.add(okayHealth(s));
-      }
-    }
-  }
-
-  /**
    * The function keeps updating the health of the cache based on roles required by the regions and
-   * their reliablity policies.
-   * 
+   * their reliability policies.
    */
+  private void checkCacheRequiredRolesMeet(List status) {
+    // will have to call here okayHealth() or poorHealth()
 
-  void checkCacheRequiredRolesMeet(List status) {
-    // will have to call here okeyHealth() or poorHealth()
-    // GemFireCache cache = (GemFireCache)CacheFactory.getAnyInstance();
-
-    // CachePerfStats cPStats= null;
     try {
-      GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
-      CachePerfStats cPStats = null;
-      cPStats = cache.getCachePerfStats();
+      InternalCache cache = (InternalCache) CacheFactory.getAnyInstance();
+      CachePerfStats cPStats = cache.getCachePerfStats();
 
       if (cPStats.getReliableRegionsMissingFullAccess() > 0) {
         // health is okay.
         int numRegions = cPStats.getReliableRegionsMissingFullAccess();
         status.add(okayHealth(
             LocalizedStrings.MemberHealthEvaluator_THERE_ARE_0_REGIONS_MISSING_REQUIRED_ROLES_BUT_ARE_CONFIGURED_FOR_FULL_ACCESS
-                .toLocalizedString(Integer.valueOf(numRegions))));
+                .toLocalizedString(numRegions)));
       } else if (cPStats.getReliableRegionsMissingLimitedAccess() > 0) {
         // health is poor
         int numRegions = cPStats.getReliableRegionsMissingLimitedAccess();
         status.add(poorHealth(
             LocalizedStrings.MemberHealthEvaluator_THERE_ARE_0_REGIONS_MISSING_REQUIRED_ROLES_AND_CONFIGURED_WITH_LIMITED_ACCESS
-                .toLocalizedString(Integer.valueOf(numRegions))));
+                .toLocalizedString(numRegions)));
       } else if (cPStats.getReliableRegionsMissingNoAccess() > 0) {
         // health is poor
         int numRegions = cPStats.getReliableRegionsMissingNoAccess();
         status.add(poorHealth(
             LocalizedStrings.MemberHealthEvaluator_THERE_ARE_0_REGIONS_MISSING_REQUIRED_ROLES_AND_CONFIGURED_WITHOUT_ACCESS
-                .toLocalizedString(Integer.valueOf(numRegions))));
-      } // else{
-        // health is good/okay
-        // status.add(okayHealth("All regions have there required roles meet"));
-        // }
+                .toLocalizedString(numRegions)));
+      }
     } catch (CancelException ignore) {
     }
   }
 
-
   /**
    * Updates the previous values of statistics
    */
@@ -234,7 +192,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
     checkMessageQueueSize(status);
     checkReplyTimeouts(status);
     // will have to add another call to check for roles
-    // missing and reliablity attributed.
+    // missing and reliability attributed.
     checkCacheRequiredRolesMeet(status);
 
     updatePrevious();
@@ -242,6 +200,6 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
 
   @Override
   void close() {
-
+    // nothing
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/8c2210db/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
index 7025721..0c096f9 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
@@ -36,6 +36,7 @@ import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
 import org.apache.geode.internal.admin.remote.AdminResponse;
 import org.apache.geode.internal.admin.remote.CliLegacyMessage;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -79,10 +80,10 @@ public class PrepareBackupRequest extends CliLegacyMessage {
 
   @Override
   protected AdminResponse createResponse(DistributionManager dm) {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    InternalCache cache = GemFireCacheImpl.getInstance();
     HashSet<PersistentID> persistentIds;
     if (cache == null) {
-      persistentIds = new HashSet<PersistentID>();
+      persistentIds = new HashSet<>();
     } else {
       try {
         BackupManager manager = cache.startBackup(getSender());

http://git-wip-us.apache.org/repos/asf/geode/blob/8c2210db/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java b/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java
index 1a46f24..69f1087 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/AttributesFactory.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;
@@ -32,7 +31,6 @@ import org.apache.geode.internal.cache.AbstractRegion;
 import org.apache.geode.internal.cache.DiskStoreFactoryImpl;
 import org.apache.geode.internal.cache.DiskWriteAttributesImpl;
 import org.apache.geode.internal.cache.EvictionAttributesImpl;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.PartitionAttributesImpl;
 import org.apache.geode.internal.cache.PartitionedRegionHelper;
 import org.apache.geode.internal.cache.UserSpecifiedRegionAttributes;
@@ -40,34 +38,34 @@ import org.apache.geode.internal.cache.xmlcache.RegionAttributesCreation;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
 /**
- * Creates instances of {@link RegionAttributes}. An <code>AttributesFactory</code> instance
- * maintains state for creating <code>RegionAttributes</code> instances. The setter methods are used
- * to change the settings that will be used for creating the next attributes instance with the
- * {@link #create} method. If you create a factory with the default constructor, then the factory is
- * set up to create attributes with all default settings. You can also create a factory by providing
- * a <code>RegionAttributes</code>, which will set up the new factory with the settings provided in
- * that attributes instance.
+ * Creates instances of {@link RegionAttributes}. An {@code AttributesFactory} instance maintains
+ * state for creating {@code RegionAttributes} instances. The setter methods are used to change the
+ * settings that will be used for creating the next attributes instance with the {@link #create}
+ * method. If you create a factory with the default constructor, then the factory is set up to
+ * create attributes with all default settings. You can also create a factory by providing a
+ * {@code RegionAttributes}, which will set up the new factory with the settings provided in that
+ * attributes instance.
  *
  * <p>
- * Once a <code>RegionAttributes</code> is created, it can only be modified after it has been used
- * to create a <code>Region</code>, and then only by using an {@link AttributesMutator} obtained
- * from the region.
+ * Once a {@code RegionAttributes} is created, it can only be modified after it has been used to
+ * create a {@code Region}, and then only by using an {@link AttributesMutator} obtained from the
+ * region.
  *
  * <h3>Attributes</h3>
  * <h4>Callbacks</h4>
  * <dl>
- * <dt>{@link CacheLoader} [<em>default:</em> <code>null</code>, meaning no loader]</dt>
+ * <dt>{@link CacheLoader} [<em>default:</em> {@code null}, meaning no loader]</dt>
  * <dd>User-implemented plug-in for loading data on cache misses.<br>
  * {@link #setCacheLoader} {@link RegionAttributes#getCacheLoader}
  * {@link AttributesMutator#setCacheLoader}</dd>
  *
- * <dt>{@link CacheWriter} [<em>default:</em> <code>null</code>, meaning no writer]</dt>
+ * <dt>{@link CacheWriter} [<em>default:</em> {@code null}, meaning no writer]</dt>
  * <dd>User-implemented plug-in for intercepting cache modifications, e.g. for writing to an
  * external data source.<br>
  * {@link #setCacheWriter} {@link RegionAttributes#getCacheWriter}
  * {@link AttributesMutator#setCacheWriter}</dd>
  *
- * <dt>{@link CacheListener} [<em>default:</em> <code>null</code>, meaning no listener ]</dt>
+ * <dt>{@link CacheListener} [<em>default:</em> {@code null}, meaning no listener ]</dt>
  * <dd>User-implemented plug-in for receiving and handling cache related events.<br>
  * {@link #addCacheListener} {@link #initCacheListeners} {@link #initCacheListeners}
  * {@link RegionAttributes#getCacheListeners} {@link AttributesMutator#initCacheListeners}
@@ -106,12 +104,12 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * whether acknowledgements are required, and whether distributed synchronization is required. <br>
  * {@link #setScope} {@link RegionAttributes#getScope}</dd>
  *
- * <dt>EarlyAck [<em>default:</em> <code>false</code>]</dt>
- * <dd>Whether or not acks required by <code>Scope.DISTRIBUTED_ACK</code> are sent after an
- * operation is processed. If <code>true</code> then remote caches will ACK before processing an
- * operation sent by the cache that has set earlyAck to <code>true</code>. Note that this attribute
- * is only meaningful on the cache that is initiating an operation; it does not matter what it is
- * set to on the cache that receives the operation.<br>
+ * <dt>EarlyAck [<em>default:</em> {@code false}]</dt>
+ * <dd>Whether or not acks required by {@code Scope.DISTRIBUTED_ACK} are sent after an operation is
+ * processed. If {@code true} then remote caches will ACK before processing an operation sent by the
+ * cache that has set earlyAck to {@code true}. Note that this attribute is only meaningful on the
+ * cache that is initiating an operation; it does not matter what it is set to on the cache that
+ * receives the operation.<br>
  * {@link #setEarlyAck} {@link RegionAttributes#getEarlyAck}</dd>
  * 
  * <dt>{@link SubscriptionAttributes} [<em>default:</em> {@link InterestPolicy#DEFAULT}]</dt>
@@ -119,44 +117,42 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * <br>
  * {@link #setSubscriptionAttributes} {@link RegionAttributes#getSubscriptionAttributes}</dd>
  *
- * <dt>EnableAsyncConflation [<em>default:</em> <code>false</code>]</dt>
+ * <dt>EnableAsyncConflation [<em>default:</em> {@code false}]</dt>
  * <dd>Whether or not conflation is enabled for sending messages to async peers. Async peers are
- * those whose <code>async-distribution-timeout</code> gemfire.property is greater than zero.
- * AsyncConflation is ignored if the scope is <code>DISTRIBUTED_ACK</code> or <code>GLOBAL</code>.
- * Conflation is only done on entry update operations. It is done by dropping the earlier update
- * from the message queue. {@link #setEnableAsyncConflation}
+ * those whose {@code async-distribution-timeout} gemfire.property is greater than zero.
+ * AsyncConflation is ignored if the scope is {@code DISTRIBUTED_ACK} or {@code GLOBAL}. Conflation
+ * is only done on entry update operations. It is done by dropping the earlier update from the
+ * message queue. {@link #setEnableAsyncConflation}
  * {@link RegionAttributes#getEnableAsyncConflation}</dd>
- * <dt>poolName [<em>default:</em> <code>null</code>, meaning no pool]</dt>
+ * <dt>poolName [<em>default:</em> {@code null}, meaning no pool]</dt>
  * <dd>Whether or not this region is a client that is to use connections from the named pool to
- * communicate with servers. If <code>null</code>, then it is not a client. If
- * <code>non-null</code>, then the named pool will be used. {@link #setPoolName}
- * {@link RegionAttributes#getPoolName}</dd>
+ * communicate with servers. If {@code null}, then it is not a client. If {@code non-null}, then the
+ * named pool will be used. {@link #setPoolName} {@link RegionAttributes#getPoolName}</dd>
  * 
  *
- * <dt>EnableSubscriptionConflation [<em>default:</em> <code>false</code>]</dt>
+ * <dt>EnableSubscriptionConflation [<em>default:</em> {@code false}]</dt>
  * <dd>Whether or not conflation is enabled for sending messages from a cache server to its clients.
  * Note: This parameter is only valid for cache server to client communication. It has no effect in
- * peer to peer communication. If <code>true</code>, messages will be conflated before they are sent
- * from a cache server to its clients. Only the latest value will be sent. Note that this attribute
- * is only meaningful in a client server topology. {@link #setEnableSubscriptionConflation}
+ * peer to peer communication. If {@code true}, messages will be conflated before they are sent from
+ * a cache server to its clients. Only the latest value will be sent. Note that this attribute is
+ * only meaningful in a client server topology. {@link #setEnableSubscriptionConflation}
  * {@link RegionAttributes#getEnableSubscriptionConflation}</dd>
- * <dt>Publisher [<em>default:</em> <code>false</code>]</dt>
+ * <dt>Publisher [<em>default:</em> {@code false}]</dt>
  * <dd>Whether or not a region is a publisher. Publishers are regions that will have distributed
  * write operations done on them. If a publisher is also a replicate then it will be used as the
  * preferred source for initializing other replicates. {@link #setPublisher}
  * {@link RegionAttributes#getPublisher}</dd>
- * <dt>isCloningEnabled [<em>default:</em> <code>false</code>]</dt>
- * <dd>Whether or not value is cloned before appling <code>Delta</code>s If <code>false</code>,
- * value will not be cloned {@link #setCloningEnabled}
- * {@link RegionAttributes#getCloningEnabled()}</dd></dt>
+ * <dt>isCloningEnabled [<em>default:</em> {@code false}]</dt>
+ * <dd>Whether or not value is cloned before appling {@code Delta}s If {@code false}, value will not
+ * be cloned {@link #setCloningEnabled} {@link RegionAttributes#getCloningEnabled()}</dd></dt>
  * </dl>
  * <h4>Storage (see also <a href="package-summary.html#storage">package summary </a>)</h4>
  * <dl>
- * <dt>{@link DataPolicy} [<em>default:</em> <code>DataPolicy.NORMAL</code>]</dt>
+ * <dt>{@link DataPolicy} [<em>default:</em> {@code DataPolicy.NORMAL}]</dt>
  * <dd>Specifies the data storage policy.<br>
  * {@link #setDataPolicy} {@link RegionAttributes#getDataPolicy}</dd>
  *
- * <dt>{@link MirrorType} [<em>default:</em> <code>MirrorType.NONE</code>]</dt>
+ * <dt>{@link MirrorType} [<em>default:</em> {@code MirrorType.NONE}]</dt>
  * <dd><em>Deprecated</em>, use DataPolicy instead.</dd>
  *
  * <dt>{@link #setEvictionAttributes(EvictionAttributes) EvictionAttributes}</dt>
@@ -164,28 +160,28 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * CapacityController interface. EvictionAttributes describe the {@link EvictionAlgorithm} and the
  * {@link EvictionAction} as well as the various conditions under which the algorithm perform the
  * action e.g. when the maximum number of entries has been reached or the maximum percentage of JVM
- * heap has been consumed. Setting <code>EvictionAttributes</code> installs an eviction controller
- * on the Region instantiated with the associated RegionAttributes</dd>
+ * heap has been consumed. Setting {@code EvictionAttributes} installs an eviction controller on the
+ * Region instantiated with the associated RegionAttributes</dd>
  *
- * <dt>KeyConstraint [<em>default:</em> <code>null</code>, meaning no constraint]</dt>
+ * <dt>KeyConstraint [<em>default:</em> {@code null}, meaning no constraint]</dt>
  * <dd>The Class to constrain the keys to in the region.<br>
  * {@link #setKeyConstraint} {@link RegionAttributes#getKeyConstraint}</dd>
  *
- * <dt>ValueConstraint [<em>default:</em> <code>null</code>, meaning no constraint]</dt>
+ * <dt>ValueConstraint [<em>default:</em> {@code null}, meaning no constraint]</dt>
  * <dd>The Class to constrain the values to in the region. In addition to the utility of this for
- * applications in general, a <code>valueConstraint</code> is helpful for compiling queries.<br>
+ * applications in general, a {@code valueConstraint} is helpful for compiling queries.<br>
  * {@link #setValueConstraint} {@link RegionAttributes#getValueConstraint}</dd>
  *
- * <dt>InitialCapacity [<em>default:</em> <code>16</code>]</dt>
+ * <dt>InitialCapacity [<em>default:</em> {@code 16}]</dt>
  * <dd>The initial capacity of the map used for storing the entries. <br>
  * {@link java.util.HashMap} {@link #setInitialCapacity}
  * {@link RegionAttributes#getInitialCapacity}</dd>
  *
- * <dt>LoadFactor [<em>default:</em> <code>0.75</code>]</dt>
+ * <dt>LoadFactor [<em>default:</em> {@code 0.75}]</dt>
  * <dd>The load factor of the map used for storing the entries. <br>
  * {@link java.util.HashMap} {@link #setLoadFactor} {@link RegionAttributes#getLoadFactor}</dd>
  *
- * <dt>ConcurrencyLevel [<em>default:</em> <code>16</code>]</dt>
+ * <dt>ConcurrencyLevel [<em>default:</em> {@code 16}]</dt>
  * <dd>The allowed concurrency among updates to values in the region is guided by the
  * <tt>concurrencyLevel</tt>, which is used as a hint for internal sizing. The actual concurrency
  * will vary. Ideally, you should choose a value to accommodate as many threads as will ever
@@ -196,46 +192,45 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * others will only read. <br>
  * {@link #setConcurrencyLevel} {@link RegionAttributes#getConcurrencyLevel}</dd>
  * 
- * <dt>ConcurrencyChecksEnabled [<em>default:</em> <code>false</code>]</dt>
+ * <dt>ConcurrencyChecksEnabled [<em>default:</em> {@code false}]</dt>
  * <dd>Enables a distributed versioning algorithm that detects concurrency conflicts in regions and
  * ensures that changes to an entry are not applied in a different order in other members. This can
  * cause operations to be conflated, so that some cache listeners may see an event while others do
  * not, but it guarantees that the system will be consistent.</dd>
  *
- * <dt>StatisticsEnabled [<em>default:</em> <code>false</code>]</dt>
+ * <dt>StatisticsEnabled [<em>default:</em> {@code false}]</dt>
  * <dd>Whether statistics are enabled for this region. The default is disabled, which conserves on
  * memory. <br>
  * {@link #setStatisticsEnabled} {@link RegionAttributes#getStatisticsEnabled}</dd>
  *
- * <dt>IgnoreJTA [<em>default:</em> <code>false</code>]</dt>
+ * <dt>IgnoreJTA [<em>default:</em> {@code false}]</dt>
  * <dd>Whether JTA transactions are ignored for this region. The default is to look for and join JTA
  * transactions for operations performed on a region.
  *
- * <dt>DiskStoreName [<em>default:</em> <code>null</code>, meaning no disk store]</dt>
- * <dd>If not <code>null</code> then this region will write its data to the named
- * {@link DiskStore}.<br>
+ * <dt>DiskStoreName [<em>default:</em> {@code null}, meaning no disk store]</dt>
+ * <dd>If not {@code null} then this region will write its data to the named {@link DiskStore}.<br>
  * {@link #setDiskStoreName} {@link RegionAttributes#getDiskStoreName}</dd>
  *
- * <dt>DiskSynchronous [<em>default:</em> <code>true</code>]</dt>
- * <dd>If <code>true</code> then any writes to disk done for this region will be done synchronously.
- * This means that they will be in the file system buffer before the operation doing the write
+ * <dt>DiskSynchronous [<em>default:</em> {@code true}]</dt>
+ * <dd>If {@code true} then any writes to disk done for this region will be done synchronously. This
+ * means that they will be in the file system buffer before the operation doing the write
  * returns.<br>
- * If <code>false</code> then any writes to disk done for this region will be done asynchronously.
- * This means that they are queued up to be written and when they are actually written to the file
- * system buffer is determined by the region's {@link DiskStore} configuration. Asynchronous writes
- * will be conflated if the same entry is written while a previous operation for the same entry is
- * still in the queue.<br>
+ * If {@code false} then any writes to disk done for this region will be done asynchronously. This
+ * means that they are queued up to be written and when they are actually written to the file system
+ * buffer is determined by the region's {@link DiskStore} configuration. Asynchronous writes will be
+ * conflated if the same entry is written while a previous operation for the same entry is still in
+ * the queue.<br>
  * {@link #setDiskSynchronous} {@link RegionAttributes#isDiskSynchronous}</dd>
  * 
- * <dt>PersistBackup [<em>default:</em> <code>false</code>]</dt>
+ * <dt>PersistBackup [<em>default:</em> {@code false}]</dt>
  * <dd>Whether or not a persistent backup should be made of the region.<br>
  * {@link #setPersistBackup} {@link RegionAttributes#getPersistBackup}</dd>
  * <dd><em>Deprecated</em>, use {@link DataPolicy#PERSISTENT_REPLICATE} or
  * {@link DataPolicy#PERSISTENT_PARTITION} instead.</dd>
  *
  * <dt>DiskWriteAttributes [<em>default:</em> Asynchronously write to disk every second (a
- * <code>timeInterval</code> of 1000 and a <code>byteThreshold</codE> of 0). <code>rollOplogs</code>
- * is set to true and <code>maxOplogSize</code> is set to 1024 MB]</dt>
+ * {@code timeInterval} of 1000 and a {@code byteThreshold} of 0). {@code rollOplogs} is set to true
+ * and {@code maxOplogSize} is set to 1024 MB]</dt>
  * <dd>How region data should be written to disk. Determines whether data should be written
  * synchronously or asynchronously. Data that is written asynchronously can be written at a certain
  * {@linkplain DiskWriteAttributes#getTimeInterval time interval} or once a certain number of
@@ -245,7 +240,7 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * <dd><em>Deprecated</em>, use {@link #setDiskStoreName} and {@link #setDiskSynchronous}
  * instead.</dd>
  *
- * <dt>DiskDirs [<em>default:</em> Current working directory (<code>user.dir</code>
+ * <dt>DiskDirs [<em>default:</em> Current working directory ({@code user.dir}
  * {@linkplain System#getProperties system property})]</dt>
  * <dd>The directories to which the region's data are written. If multiple directories are used,
  * GemFire will attempt to distribute the data evenly among them. <br>
@@ -258,7 +253,7 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * <dd><em>Deprecated</em>, use {@link #setDiskStoreName} instead.</dd>
  *
  *
- * <dt>{@link PartitionAttributes} [<em>default:</em> <code>null</code>, meaning no region
+ * <dt>{@link PartitionAttributes} [<em>default:</em> {@code null}, meaning no region
  * partitioning]</dt>
  * <dd>How region data is partitioned among the members of the distributed system. <br>
  * {@link #setPartitionAttributes} {@link RegionAttributes#getPartitionAttributes}</dd>
@@ -272,14 +267,14 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  *
  * <h4>Locking</h4>
  * <dl>
- * <dt>LockGrantor [<em>default:</em> <code>false</code>]</dt>
+ * <dt>LockGrantor [<em>default:</em> {@code false}]</dt>
  * <dd>Should this process become lock grantor for the region?</dd><br>
  * {@link #setLockGrantor} {@link RegionAttributes#isLockGrantor} {@link Region#becomeLockGrantor}
  * </dl>
  *
  * <h4>Querying</h4>
  * <dl>
- * <dt>IndexMaintenanceSynchronous [<em>default:</em> <code>false</code>]</dt>
+ * <dt>IndexMaintenanceSynchronous [<em>default:</em> {@code false}]</dt>
  * <dd>Are indexes built over in this region updated synchronously when the underlying data is
  * modified?</dd><br>
  * {@link #setIndexMaintenanceSynchronous} {@link RegionAttributes#getIndexMaintenanceSynchronous}
@@ -291,29 +286,26 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * <a name="compatibility">
  * <h3>Compatibility Rules</h3>
  * <h4>RegionAttributes Creation Constraints</h4> If any of the following compatibility rules are
- * violated when {@link #create}</code> is called then an {@link IllegalStateException} is thrown.
- * See {@link #validateAttributes}.
+ * violated when {@link #create}} is called then an {@link IllegalStateException} is thrown. See
+ * {@link #validateAttributes}.
  *
  * <a name="creationConstraints">
  * <h3>Creation Constraints</h3>
  * <h4>Region Creation Constraints on RegionAttributes</h4>
  *
  * If any of the following rules are violated when {@link Region#createSubregion createSubregion} or
- * {@link Cache#createRegion createRegion} are called, then an <code>IllegalStateException</code> is
+ * {@link Cache#createRegion createRegion} are called, then an {@code IllegalStateException} is
  * thrown.
  *
  * <ul>
- * <li>A region with <code>Scope.LOCAL</code> can only have subregions with
- * <code>Scope.LOCAL</code>.</li>
- * <li><code>Scope.GLOBAL</code> is illegal if there is any other cache in the distributed system
- * that has the same region with <code>Scope.DISTRIBUTED_NO_ACK</code> or
- * <code>Scope.DISTRIBUTED_ACK</code>.</li>
- * <li><code>Scope.DISTRIBUTED_ACK</code> is illegal if there is any other cache in the distributed
- * system that has the same region with <code>Scope.DISTRIBUTED_NO_ACK</code> or
- * <code>Scope.GLOBAL</code>.</li>
- * <li><code>Scope.DISTRIBUTED_NO_ACK</code> is illegal if there is any other cache in the
- * distributed system that has the same region with <code>Scope.DISTRIBUTED_ACK</code> or
- * <code>Scope.GLOBAL</code>.</li>
+ * <li>A region with {@code Scope.LOCAL} can only have subregions with {@code Scope.LOCAL}.</li>
+ * <li>{@code Scope.GLOBAL} is illegal if there is any other cache in the distributed system that
+ * has the same region with {@code Scope.DISTRIBUTED_NO_ACK} or {@code Scope.DISTRIBUTED_ACK}.</li>
+ * <li>{@code Scope.DISTRIBUTED_ACK} is illegal if there is any other cache in the distributed
+ * system that has the same region with {@code Scope.DISTRIBUTED_NO_ACK} or
+ * {@code Scope.GLOBAL}.</li>
+ * <li>{@code Scope.DISTRIBUTED_NO_ACK} is illegal if there is any other cache in the distributed
+ * system that has the same region with {@code Scope.DISTRIBUTED_ACK} or {@code Scope.GLOBAL}.</li>
  * </ul>
  *
  * @see RegionAttributes
@@ -324,6 +316,7 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  * @deprecated as of 6.5 use {@link Cache#createRegionFactory(RegionShortcut)} or
  *             {@link ClientCache#createClientRegionFactory(ClientRegionShortcut)} instead.
  */
+@Deprecated
 @SuppressWarnings("synthetic-access")
 public class AttributesFactory<K, V> {
   private final RegionAttributesImpl<K, V> regionAttributes = new RegionAttributesImpl<K, V>();
@@ -331,24 +324,23 @@ public class AttributesFactory<K, V> {
   /**
    * The default disk synchronous write setting
    * <p>
-   * Current value: <code>true</code> each.
+   * Current value: {@code true} each.
    * 
    * @since GemFire 6.5
    */
   public static final boolean DEFAULT_DISK_SYNCHRONOUS = true;
 
   /**
-   * Creates a new instance of AttributesFactory ready to create a <code>RegionAttributes</code>
-   * with default settings.
+   * Creates a new instance of AttributesFactory ready to create a {@code RegionAttributes} with
+   * default settings.
    */
   public AttributesFactory() {}
 
   /**
-   * Creates a new instance of AttributesFactory ready to create a <code>RegionAttributes</code>
-   * with the same settings as those in the specified <code>RegionAttributes</code>.
+   * Creates a new instance of AttributesFactory ready to create a {@code RegionAttributes} with the
+   * same settings as those in the specified {@code RegionAttributes}.
    *
-   * @param regionAttributes the <code>RegionAttributes</code> used to initialize this
-   *        AttributesFactory
+   * @param regionAttributes the {@code RegionAttributes} used to initialize this AttributesFactory
    */
   @SuppressWarnings("deprecation")
   public AttributesFactory(RegionAttributes<K, V> regionAttributes) {
@@ -448,7 +440,7 @@ public class AttributesFactory<K, V> {
   // CALLBACKS
 
   /**
-   * 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
    * @throws IllegalStateException if this region has a {@link #setPoolName pool name set}
@@ -459,7 +451,7 @@ public class AttributesFactory<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
    * @throws IllegalStateException if this region has a {@link #setPoolName pool name set}
@@ -470,7 +462,7 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the CacheListener for the next <code>RegionAttributes</code> created. Any existing cache
+   * Sets the CacheListener for the next {@code RegionAttributes} created. Any existing cache
    * listeners on this factory are removed.
    * 
    * @param aListener a user defined CacheListener, null if no listener
@@ -495,7 +487,7 @@ public class AttributesFactory<K, V> {
    * Adds a cache listener to the end of the list of cache listeners on this factory.
    * 
    * @param aListener the cache listener to add to the factory.
-   * @throws IllegalArgumentException if <code>aListener</code> is null
+   * @throws IllegalArgumentException if {@code aListener} is null
    * @since GemFire 5.0
    */
   public void addCacheListener(CacheListener<K, V> aListener) {
@@ -513,7 +505,7 @@ public class AttributesFactory<K, V> {
    * Removes all cache listeners and then adds each listener in the specified array.
    * 
    * @param newListeners a possibly null or empty array of listeners to add to this factory.
-   * @throws IllegalArgumentException if the <code>newListeners</code> array has a null element
+   * @throws IllegalArgumentException if the {@code newListeners} array has a null element
    * @since GemFire 5.0
    */
   public void initCacheListeners(CacheListener<K, V>[] newListeners) {
@@ -538,7 +530,7 @@ public class AttributesFactory<K, V> {
 
   /**
    * Sets the idleTimeout expiration attributes for region entries for the next
-   * <code>RegionAttributes</code> created. Default is 0 which indicates no expiration of this type.
+   * {@code RegionAttributes} created. Default is 0 which indicates no expiration of this type.
    *
    * @param idleTimeout the idleTimeout ExpirationAttributes for entries in this region
    * @throws IllegalArgumentException if idleTimeout is null
@@ -554,7 +546,7 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the idleTimeout CustomExpiry for the next <code>RegionAttributes</code> created.
+   * Sets the idleTimeout CustomExpiry for the next {@code RegionAttributes} created.
    * 
    * @param custom the CustomExpiry to use; null means none will be used.
    */
@@ -565,7 +557,7 @@ public class AttributesFactory<K, V> {
 
   /**
    * Sets the timeToLive expiration attributes for region entries for the next
-   * <code>RegionAttributes</code> created. Default is 0 which indicates no expiration of this type.
+   * {@code RegionAttributes} created. Default is 0 which indicates no expiration of this type.
    *
    * @param timeToLive the timeToLive ExpirationAttributes for entries in this region
    * @throws IllegalArgumentException if timeToLive is null
@@ -581,7 +573,7 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the custom timeToLive for the next <code>RegionAttributes</code> created.
+   * Sets the custom timeToLive for the next {@code RegionAttributes} created.
    * 
    * @param custom the CustomExpiry to use, none if the default for the region is to be used.
    */
@@ -592,8 +584,8 @@ public class AttributesFactory<K, V> {
 
   /**
    * Sets the idleTimeout expiration attributes for the region itself for the next
-   * <code>RegionAttributes</code> created. Default is 0 which indicates no expiration of this type
-   * is set.
+   * {@code RegionAttributes} created. Default is 0 which indicates no expiration of this type is
+   * set.
    *
    * @param idleTimeout the ExpirationAttributes for this region idleTimeout
    * @throws IllegalArgumentException if idleTimeout is null
@@ -611,7 +603,7 @@ public class AttributesFactory<K, V> {
 
   /**
    * Sets the timeToLive expiration attributes for the region itself for the next
-   * <code>RegionAttributes</code> created. Default is 0 i.e. no expiration of this type.
+   * {@code RegionAttributes} created. Default is 0 i.e. no expiration of this type.
    *
    * @param timeToLive the ExpirationAttributes for this region timeToLive
    * @throws IllegalArgumentException if timeToLive is null
@@ -629,7 +621,7 @@ public class AttributesFactory<K, V> {
   // DISTRIBUTION ATTRIBUTES
 
   /**
-   * Sets the scope for the next <code>RegionAttributes</code> created. Default scope is
+   * Sets the scope for the next {@code RegionAttributes} created. Default scope is
    * DISTRIBUTED_NO_ACK. Refer gemfire documentation for more details on this.
    * 
    * @param scopeType the type of Scope to use for the region
@@ -646,10 +638,10 @@ public class AttributesFactory<K, V> {
   // STORAGE ATTRIBUTES
 
   /**
-   * Sets the EvictionController for the next <code>RegionAttributes</code> created. Use one of the
+   * Sets the EvictionController for the next {@code RegionAttributes} created. Use one of the
    * creation methods on {@link EvictionAttributes} e.g.
    * {@link EvictionAttributes#createLRUHeapAttributes()} to create the desired instance for this
-   * <code>AttributesFactory</code>
+   * {@code AttributesFactory}
    *
    * @param evictAttrs Explains how and when eviction occurs in the Region.
    */
@@ -663,7 +655,7 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the mirror type for the next <code>RegionAttributes</code> created.
+   * Sets the mirror type for the next {@code RegionAttributes} created.
    * 
    * @param mirrorType The type of mirroring to use for the region
    * @throws IllegalArgumentException if mirrorType is null
@@ -692,7 +684,7 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the data policy for the next <code>RegionAttributes</code> created. Default data policy is
+   * Sets the data policy for the next {@code RegionAttributes} created. Default data policy is
    * 'Normal'. Please refer gemfire documentation for more details on this.
    * 
    * @param dataPolicy The data policy to use for the region
@@ -716,13 +708,12 @@ public class AttributesFactory<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
-   * @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
    */
   public void setKeyConstraint(Class<K> keyConstraint) {
     if (keyConstraint != null && keyConstraint.isPrimitive())
@@ -734,13 +725,13 @@ public class AttributesFactory<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
-   * @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
    */
   public void setValueConstraint(Class<V> valueConstraint) {
     if (valueConstraint != null && valueConstraint.isPrimitive())
@@ -755,8 +746,8 @@ public class AttributesFactory<K, V> {
 
   // MAP ATTRIBUTES
   /**
-   * Sets the entry initial capacity for the next <code>RegionAttributes</code> created. This value
-   * is used in initializing the map that holds the entries. Default is 16.
+   * Sets the entry initial capacity for the next {@code RegionAttributes} created. This value is
+   * used in initializing the map that holds the entries. Default is 16.
    * 
    * @param initialCapacity the initial capacity of the entry map
    * @throws IllegalArgumentException if initialCapacity is negative.
@@ -771,8 +762,8 @@ public class AttributesFactory<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. Default is 0.75.
+   * Sets the entry load factor for the next {@code RegionAttributes} created. This value is used in
+   * initializing the map that holds the entries. Default is 0.75.
    * 
    * @param loadFactor the load factor of the entry map
    * @throws IllegalArgumentException if loadFactor is nonpositive
@@ -788,8 +779,8 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the concurrency level of the next <code>RegionAttributes</code> created. This value is
-   * used in initializing the map that holds the entries. Default is 16.
+   * Sets the concurrency level of the next {@code RegionAttributes} created. This value is used in
+   * initializing the map that holds the entries. Default is 16.
    * 
    * @param concurrencyLevel the concurrency level of the entry map
    * @throws IllegalArgumentException if concurrencyLevel is nonpositive
@@ -892,7 +883,7 @@ public class AttributesFactory<K, V> {
    * adds a gateway sender to the end of list of gateway senders on this factory
    * 
    * @param gatewaySenderId
-   * @throws IllegalArgumentException if <code>gatewaySender</code> is null
+   * @throws IllegalArgumentException if {@code gatewaySender} is null
    * @since GemFire 7.0
    */
   public void addGatewaySenderId(String gatewaySenderId) {
@@ -909,7 +900,7 @@ public class AttributesFactory<K, V> {
    * Adds a AsyncEventQueue to the end of list of async event queues on this factory
    * 
    * @param asyncEventQueueId
-   * @throws IllegalArgumentException if <code>gatewaySender</code> is null
+   * @throws IllegalArgumentException if {@code gatewaySender} is null
    * @since GemFire 7.0
    */
   public void addAsyncEventQueueId(String asyncEventQueueId) {
@@ -1078,9 +1069,9 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Sets the <code>PartitionAttributes</code> that describe how the region is partitioned among
-   * members of the distributed system. This also establishes a data policy of
-   * {@link DataPolicy#PARTITION PARTITION}, if the data policy has not already been set.
+   * Sets the {@code PartitionAttributes} that describe how the region is partitioned among members
+   * of the distributed system. This also establishes a data policy of {@link DataPolicy#PARTITION
+   * PARTITION}, if the data policy has not already been set.
    *
    * @since GemFire 5.0
    */
@@ -1121,19 +1112,20 @@ public class AttributesFactory<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.
    *
    * @deprecated this API is scheduled to be removed
    */
+  @Deprecated
   public void setMembershipAttributes(MembershipAttributes membership) {
     this.regionAttributes.membershipAttributes = membership;
     this.regionAttributes.setHasMembershipAttributes(true);
   }
 
   /**
-   * Sets the <code>SubscriptionAttributes</code> that describe how the region will subscribe to
-   * other distributed cache instances of the region.
+   * Sets the {@code SubscriptionAttributes} that describe how the region will subscribe to other
+   * distributed cache instances of the region.
    *
    * @since GemFire 5.0
    */
@@ -1199,7 +1191,6 @@ public class AttributesFactory<K, V> {
    * Sets cloning on region. Default is false. Note: off-heap regions always behave as if cloning is
    * enabled.
    * 
-   * @param cloningEnable
    * @since GemFire 6.1
    * @see RegionAttributes#getCloningEnabled()
    */
@@ -1213,15 +1204,14 @@ public class AttributesFactory<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 name the name of the connection pool to use; if <code>null</code> or <code>""</code>
-   *        then the connection pool is disabled for regions using these attributes.
+   * @param name the name of the connection pool to use; if {@code null} or {@code ""} then the
+   *        connection pool is disabled for regions using these attributes.
    * @since GemFire 5.7
    */
   public void setPoolName(String name) {
@@ -1268,9 +1258,9 @@ public class AttributesFactory<K, V> {
   // FACTORY METHOD
 
   /**
-   * Creates a <code>RegionAttributes</code> with the current settings.
+   * Creates a {@code RegionAttributes} with the current settings.
    * 
-   * @return the newly created <code>RegionAttributes</code>
+   * @return the newly created {@code RegionAttributes}
    * @throws IllegalStateException if the current settings violate the
    *         <a href="#compatibility">compatibility rules</a>
    * @deprecated as of GemFire 5.0, use {@link #create} instead
@@ -1281,9 +1271,9 @@ public class AttributesFactory<K, V> {
   }
 
   /**
-   * Creates a <code>RegionAttributes</code> with the current settings.
+   * Creates a {@code RegionAttributes} with the current settings.
    * 
-   * @return the newly created <code>RegionAttributes</code>
+   * @return the newly created {@code RegionAttributes}
    * @throws IllegalStateException if the current settings violate the
    *         <a href="#compatibility">compatibility rules</a>
    * @since GemFire 5.0
@@ -1612,7 +1602,7 @@ public class AttributesFactory<K, V> {
     boolean offHeap = false;
 
     /**
-     * Constructs an instance of <code>RegionAttributes</code> with default settings.
+     * Constructs an instance of {@code RegionAttributes} with default settings.
      * 
      * @see AttributesFactory
      */
@@ -1956,6 +1946,7 @@ public class AttributesFactory<K, V> {
     /**
      * @deprecated this API is scheduled to be removed
      */
+    @Deprecated
     public MembershipAttributes getMembershipAttributes() {
       return this.membershipAttributes;
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/8c2210db/geode-core/src/main/java/org/apache/geode/cache/CacheClosedException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/CacheClosedException.java b/geode-core/src/main/java/org/apache/geode/cache/CacheClosedException.java
index b24bc2f..d149462 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/CacheClosedException.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/CacheClosedException.java
@@ -16,12 +16,11 @@ package org.apache.geode.cache;
 
 import org.apache.geode.CancelException;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 
 /**
  * Indicates that the caching system has been closed. Can be thrown from almost any method related
- * to regions or the <code>Cache</code> after the cache has been closed.
- *
- *
+ * to regions or the {@code Cache} after the cache has been closed.
  *
  * @see Cache
  * @since GemFire 3.0
@@ -30,28 +29,28 @@ public class CacheClosedException extends CancelException {
   private static final long serialVersionUID = -6479561694497811262L;
 
   /**
-   * Constructs a new <code>CacheClosedException</code>.
+   * Constructs a new {@code CacheClosedException}.
    */
   public CacheClosedException() {
     super();
   }
 
   /**
-   * Constructs a new <code>CacheClosedException</code> with a message string.
+   * Constructs a new {@code CacheClosedException} with a message string.
    *
    * @param msg a message string
    */
   public CacheClosedException(String msg) {
     super(msg);
-    // bug #43108 - CacheClosedException should include cause of closure
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    // bug #43108 - CacheClosedException should include cause of closure TODO: but not this way!
+    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache != null) {
       initCause(cache.getDisconnectCause());
     }
   }
 
   /**
-   * Constructs a new <code>CacheClosedException</code> with a message string and a cause.
+   * Constructs a new {@code CacheClosedException} with a message string and a cause.
    *
    * @param msg the message string
    * @param cause a causal Throwable
@@ -61,7 +60,7 @@ public class CacheClosedException extends CancelException {
   }
 
   /**
-   * Constructs a new <code>CacheClosedException</code> with a cause.
+   * Constructs a new {@code CacheClosedException} with a cause.
    *
    * @param cause a causal Throwable
    */
@@ -69,4 +68,3 @@ public class CacheClosedException extends CancelException {
     super(cause);
   }
 }
-

http://git-wip-us.apache.org/repos/asf/geode/blob/8c2210db/geode-core/src/main/java/org/apache/geode/cache/CacheFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/CacheFactory.java b/geode-core/src/main/java/org/apache/geode/cache/CacheFactory.java
index 0772dcf..38fdac6 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/CacheFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/CacheFactory.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.util.Properties;
@@ -23,15 +22,16 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.cache.CacheConfig;
 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;
-import org.apache.geode.internal.jndi.JNDIInvoker;
 import org.apache.geode.pdx.PdxInstance;
 import org.apache.geode.pdx.PdxSerializer;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.AuthenticationRequiredException;
 import org.apache.geode.security.PostProcessor;
 import org.apache.geode.security.SecurityManager;
 
-
 /**
  * Factory class used to create the singleton {@link Cache cache} and connect to the GemFire
  * singleton {@link DistributedSystem distributed system}. If the application wants to connect to
@@ -87,8 +87,6 @@ import org.apache.geode.security.SecurityManager;
  * explicitly control the individual region attributes can do this declaratively in XML or using
  * APIs.
  *
- *
- *
  * @since GemFire 3.0
  */
 public class CacheFactory {
@@ -135,32 +133,29 @@ public class CacheFactory {
   }
 
   /**
-   * Creates a new cache that uses the specified <code>system</code>.
-   *
+   * Creates a new cache that uses the specified {@code system}.
    * <p>
-   *
-   * The <code>system</code> can specify a
+   * The {@code system} can specify a
    * <A href="../distributed/DistributedSystem.html#cache-xml-file">"cache-xml-file"</a> property
    * which will cause this creation to also create the regions, objects, and attributes declared in
-   * the file. The contents of the file must comply with the <code>"doc-files/cache8_0.dtd"></code>
-   * file. Note that when parsing the XML file {@link Declarable} classes are loaded using the
-   * current thread's {@linkplain Thread#getContextClassLoader context class loader}.
+   * the file. The contents of the file must comply with the {@code "doc-files/cache8_0.dtd">} file.
+   * Note that when parsing the XML file {@link Declarable} classes are loaded using the current
+   * thread's {@linkplain Thread#getContextClassLoader context class loader}.
    *
-   * @param system a <code>DistributedSystem</code> obtained by calling
+   * @param system a {@code DistributedSystem} obtained by calling
    *        {@link DistributedSystem#connect}.
    *
-   * @return a <code>Cache</code> that uses the specified <code>system</code> for distribution.
+   * @return a {@code Cache} that uses the specified {@code system} for distribution.
    *
-   * @throws IllegalArgumentException If <code>system</code> is not
-   *         {@link DistributedSystem#isConnected connected}.
+   * @throws IllegalArgumentException If {@code system} is not {@link DistributedSystem#isConnected
+   *         connected}.
    * @throws CacheExistsException If an open cache already exists.
    * @throws CacheXmlException If a problem occurs while parsing the declarative caching XML file.
    * @throws TimeoutException If a {@link Region#put(Object, Object)} times out while initializing
    *         the cache.
-   * @throws CacheWriterException If a <code>CacheWriterException</code> is thrown while
-   *         initializing the cache.
-   * @throws GatewayException If a <code>GatewayException</code> is thrown while initializing the
+   * @throws CacheWriterException If a {@code CacheWriterException} is thrown while initializing the
    *         cache.
+   * @throws GatewayException If a {@code GatewayException} is thrown while initializing the cache.
    * @throws RegionExistsException If the declarative caching XML file describes a region that
    *         already exists (including the root region).
    * @deprecated as of 6.5 use {@link #CacheFactory(Properties)} instead.
@@ -191,10 +186,9 @@ public class CacheFactory {
    * @throws CacheXmlException If a problem occurs while parsing the declarative caching XML file.
    * @throws TimeoutException If a {@link Region#put(Object, Object)} times out while initializing
    *         the cache.
-   * @throws CacheWriterException If a <code>CacheWriterException</code> is thrown while
-   *         initializing the cache.
-   * @throws GatewayException If a <code>GatewayException</code> is thrown while initializing the
+   * @throws CacheWriterException If a {@code CacheWriterException} is thrown while initializing the
    *         cache.
+   * @throws GatewayException If a {@code GatewayException} is thrown while initializing the cache.
    * @throws RegionExistsException If the declarative caching XML file describes a region that
    *         already exists (including the root region).
    * @throws IllegalStateException if cache already exists and is not compatible with the new
@@ -222,7 +216,7 @@ public class CacheFactory {
   /**
    * Gets the instance of {@link Cache} produced by an earlier call to {@link #create()}.
    * 
-   * @param system the <code>DistributedSystem</code> the cache was created with.
+   * @param system the {@code DistributedSystem} the cache was created with.
    * @return the {@link Cache} associated with the specified system.
    * @throws CacheClosedException if a cache has not been created or the created one is
    *         {@link Cache#isClosed closed}
@@ -235,7 +229,7 @@ public class CacheFactory {
    * Gets the instance of {@link Cache} produced by an earlier call to {@link #create()} even if it
    * has been closed.
    * 
-   * @param system the <code>DistributedSystem</code> the cache was created with.
+   * @param system the {@code DistributedSystem} the cache was created with.
    * @return the {@link Cache} associated with the specified system.
    * @throws CacheClosedException if a cache has not been created
    * @since GemFire 3.5
@@ -245,7 +239,6 @@ public class CacheFactory {
   }
 
   private static Cache basicGetInstance(DistributedSystem system, boolean closeOk) {
-
     // Avoid synchronization if this is an initialization thread to avoid
     // deadlock when messaging returns to this VM
     final int initReq = LocalRegion.threadInitLevelRequirement();
@@ -260,7 +253,7 @@ public class CacheFactory {
   }
 
   private static Cache basicGetInstancePart2(DistributedSystem system, boolean closeOk) {
-    GemFireCacheImpl instance = GemFireCacheImpl.getInstance();
+    InternalCache instance = GemFireCacheImpl.getInstance();
     if (instance == null) {
       throw new CacheClosedException(
           LocalizedStrings.CacheFactory_A_CACHE_HAS_NOT_YET_BEEN_CREATED.toLocalizedString());
@@ -286,7 +279,7 @@ public class CacheFactory {
    *         {@link Cache#isClosed closed}
    */
   public static synchronized Cache getAnyInstance() {
-    GemFireCacheImpl instance = GemFireCacheImpl.getInstance();
+    InternalCache instance = GemFireCacheImpl.getInstance();
     if (instance == null) {
       throw new CacheClosedException(
           LocalizedStrings.CacheFactory_A_CACHE_HAS_NOT_YET_BEEN_CREATED.toLocalizedString());
@@ -299,7 +292,7 @@ public class CacheFactory {
   /**
    * Returns the version of the cache implementation.
    * 
-   * @return the version of the cache implementation as a <code>String</code>
+   * @return the version of the cache implementation as a {@code String}
    */
   public static String getVersion() {
     return GemFireVersion.getGemFireVersion();
@@ -335,7 +328,6 @@ public class CacheFactory {
    * object provided this way is expected to be initialized already. We are not calling the init
    * method on this object
    *
-   * @param securityManager
    * @return this CacheFactory
    */
   public CacheFactory setSecurityManager(SecurityManager securityManager) {
@@ -351,7 +343,6 @@ public class CacheFactory {
    * object provided this way is expected to be initialized already. We are not calling the init
    * method on this object
    * 
-   * @param postProcessor
    * @return this CacheFactory
    */
   public CacheFactory setPostProcessor(PostProcessor postProcessor) {
@@ -393,8 +384,8 @@ public class CacheFactory {
   /**
    * Control whether the type metadata for PDX objects is persisted to disk. The default for this
    * setting is false. If you are using persistent regions with PDX then you must set this to true.
-   * If you are using a <code>GatewaySender</code> or <code>AsyncEventQueue</code> with PDX then you
-   * should set this to true.
+   * If you are using a {@code GatewaySender} or {@code AsyncEventQueue} with PDX then you should
+   * set this to true.
    * 
    * @param isPersistent true if the metadata should be persistent
    * @return this CacheFactory
@@ -410,12 +401,12 @@ public class CacheFactory {
    * preserve unread fields be including their data during serialization. But if you configure the
    * cache to ignore unread fields then their data will be lost during serialization.
    * <P>
-   * You should only set this attribute to <code>true</code> if you know this member will only be
-   * reading cache data. In this use case you do not need to pay the cost of preserving the unread
-   * fields since you will never be reserializing pdx data.
+   * You should only set this attribute to {@code true} if you know this member will only be reading
+   * cache data. In this use case you do not need to pay the cost of preserving the unread fields
+   * since you will never be reserializing pdx data.
    * 
-   * @param ignore <code>true</code> if fields not read during pdx deserialization should be
-   *        ignored; <code>false</code>, the default, if they should be preserved.
+   * @param ignore {@code true} if fields not read during pdx deserialization should be ignored;
+   *        {@code false}, the default, if they should be preserved.
    * @return this CacheFactory
    * @since GemFire 6.6
    */
@@ -424,4 +415,3 @@ public class CacheFactory {
     return this;
   }
 }
-