Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 84FAB200C78 for ; Thu, 4 May 2017 02:07:30 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 83B2F160BB5; Thu, 4 May 2017 00:07:30 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id CA365160BD1 for ; Thu, 4 May 2017 02:07:26 +0200 (CEST) Received: (qmail 54253 invoked by uid 500); 4 May 2017 00:07:17 -0000 Mailing-List: contact commits-help@geode.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@geode.apache.org Delivered-To: mailing list commits@geode.apache.org Received: (qmail 52932 invoked by uid 99); 4 May 2017 00:07:16 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 04 May 2017 00:07:16 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9AB41E0885; Thu, 4 May 2017 00:07:16 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: klund@apache.org To: commits@geode.apache.org Date: Thu, 04 May 2017 00:07:51 -0000 Message-Id: In-Reply-To: <92c1dd6bde8a4e7dba84c3fee7bbd6f9@git.apache.org> References: <92c1dd6bde8a4e7dba84c3fee7bbd6f9@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [37/54] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache archived-at: Thu, 04 May 2017 00:07:30 -0000 http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCALocalTransaction.java ---------------------------------------------------------------------- diff --git a/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCALocalTransaction.java b/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCALocalTransaction.java index 112f2fa..7ee1f4e 100644 --- a/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCALocalTransaction.java +++ b/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCALocalTransaction.java @@ -23,62 +23,43 @@ import javax.transaction.TransactionManager; import org.apache.geode.LogWriter; import org.apache.geode.cache.CacheFactory; import org.apache.geode.cache.TransactionId; -import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.TXStateProxy; - import org.apache.geode.internal.cache.TXManagerImpl; import org.apache.geode.internal.i18n.LocalizedStrings; -/** - * - * - */ public class JCALocalTransaction implements LocalTransaction { - private volatile GemFireCacheImpl cache; + private volatile InternalCache cache; private volatile TXManagerImpl gfTxMgr; private volatile TransactionId tid; - private static final boolean DEBUG = false; - private volatile boolean initDone = false; - JCALocalTransaction(GemFireCacheImpl cache, TXManagerImpl tm) { + JCALocalTransaction(InternalCache cache, TXManagerImpl tm) { this.cache = cache; this.gfTxMgr = tm; this.initDone = true; - // System.out.println("Asif:JCALocalTransaction:Param contrcutr for tx ="+ - // this ); } JCALocalTransaction() { this.cache = null; this.gfTxMgr = null; this.initDone = false; - // System.out.println("Asif:JCALocalTransaction:Empty constructor for tx ="+ - // this ); } + @Override public void begin() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCALocalTransaction:begin"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } try { - if (!initDone || this.cache.isClosed()) { + if (!this.initDone || this.cache.isClosed()) { this.init(); } - // System.out.println("JCALocalTransaction:Asif: cache is ="+cache + - // " for tx ="+this); - LogWriter logger = cache.getLogger(); + LogWriter logger = this.cache.getLogger(); if (logger.fineEnabled()) { logger.fine("JCALocalTransaction::begin:"); } - TransactionManager tm = cache.getJTATransactionManager(); + TransactionManager tm = this.cache.getJTATransactionManager(); if (this.tid != null) { throw new LocalTransactionException(" A transaction is already in progress"); } @@ -106,28 +87,13 @@ public class JCALocalTransaction implements LocalTransaction { } } } catch (SystemException e) { - // this.onError(); throw new ResourceException(e); } - // Not to be invoked for local transactions managed by the container - // Iterator itr = this.listeners.iterator(); - // ConnectionEvent ce = new ConnectionEvent(this, - // ConnectionEvent.LOCAL_TRANSACTION_STARTED); - // while (itr.hasNext()) { - // itr.next().localTransactionStarted(ce); - // } - } + @Override public void commit() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCALocalTransaction:commit"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } - LogWriter logger = cache.getLogger(); + LogWriter logger = this.cache.getLogger(); if (logger.fineEnabled()) { logger.fine("JCALocalTransaction:invoked commit"); } @@ -140,31 +106,18 @@ public class JCALocalTransaction implements LocalTransaction { this.gfTxMgr.commit(); this.tid = null; } catch (Exception e) { - throw new LocalTransactionException(e.toString()); + throw new LocalTransactionException(e.toString(), e); } - // Iterator itr = this.listeners.iterator(); - // ConnectionEvent ce = new - // ConnectionEvent(this,ConnectionEvent.LOCAL_TRANSACTION_COMMITTED); - // while( itr.hasNext()) { - // itr.next().localTransactionCommitted(ce); - // } - } + @Override public void rollback() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JJCALocalTransaction:rollback"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } TXStateProxy tsp = this.gfTxMgr.getTXState(); if (tsp != null && this.tid != tsp.getTransactionId()) { throw new IllegalStateException("Local Transaction associated with Tid = " + this.tid + " attempting to commit a different transaction"); } - LogWriter logger = cache.getLogger(); + LogWriter logger = this.cache.getLogger(); if (logger.fineEnabled()) { logger.fine("JCALocalTransaction:invoked rollback"); } @@ -175,31 +128,24 @@ public class JCALocalTransaction implements LocalTransaction { if (ise.getMessage() .equals(LocalizedStrings.TXManagerImpl_THREAD_DOES_NOT_HAVE_AN_ACTIVE_TRANSACTION .toLocalizedString())) { - // /ignore; + // ignore } else { throw new ResourceException(ise); } - } catch (Exception e) { + } catch (RuntimeException e) { throw new ResourceException(e); } finally { this.tid = null; } - // Iterator itr = this.listeners.iterator(); - // ConnectionEvent ce = new ConnectionEvent(this, - // ConnectionEvent.LOCAL_TRANSACTION_ROLLEDBACK); - // while (itr.hasNext()) { - // itr.next().localTransactionRolledback(ce); - // } - } - private void init() throws SystemException { - this.cache = (GemFireCacheImpl) CacheFactory.getAnyInstance(); + private void init() { + this.cache = (InternalCache) CacheFactory.getAnyInstance(); LogWriter logger = this.cache.getLogger(); if (logger.fineEnabled()) { logger.fine("JCAManagedConnection:init. Inside init"); } - gfTxMgr = cache.getTxManager(); + this.gfTxMgr = this.cache.getTxManager(); this.initDone = true; } http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCAManagedConnection.java ---------------------------------------------------------------------- diff --git a/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCAManagedConnection.java b/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCAManagedConnection.java index 520f7e2..c654e64 100644 --- a/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCAManagedConnection.java +++ b/geode-core/src/jca/java/org/apache/geode/internal/ra/spi/JCAManagedConnection.java @@ -31,239 +31,196 @@ import javax.resource.spi.LocalTransaction; import javax.resource.spi.ManagedConnection; import javax.resource.spi.ManagedConnectionMetaData; import javax.security.auth.Subject; -import javax.transaction.SystemException; import javax.transaction.xa.XAResource; import org.apache.geode.LogWriter; import org.apache.geode.cache.CacheFactory; -import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.TXManagerImpl; import org.apache.geode.internal.ra.GFConnectionImpl; -/** - * - * - */ -public class JCAManagedConnection implements ManagedConnection +public class JCAManagedConnection implements ManagedConnection { -{ private final List listeners; - private volatile TXManagerImpl gfTxMgr; + private volatile TXManagerImpl transactionManager; - // private volatile TransactionId currentTxID; - private volatile GemFireCacheImpl cache; + private volatile InternalCache cache; - private volatile boolean initDone = false; + private volatile boolean initialized = false; - private volatile PrintWriter logger; + private volatile PrintWriter logWriter; - private JCAManagedConnectionFactory factory; + private final JCAManagedConnectionFactory connectionFactory; - private volatile Set connections; + private final Set connections; - private volatile JCALocalTransaction localTran; + private volatile JCALocalTransaction localTransaction; - private final static boolean DEBUG = false; - - public JCAManagedConnection(JCAManagedConnectionFactory fact) { - this.factory = fact; - this.listeners = Collections - .synchronizedList(new ArrayList()); - this.localTran = new JCALocalTransaction(); - this.connections = - Collections.synchronizedSet(new HashSet()); + JCAManagedConnection(JCAManagedConnectionFactory connectionFactory) { + this.connectionFactory = connectionFactory; + this.listeners = Collections.synchronizedList(new ArrayList<>()); + this.localTransaction = new JCALocalTransaction(); + this.connections = Collections.synchronizedSet(new HashSet<>()); } + @Override public void addConnectionEventListener(ConnectionEventListener listener) { this.listeners.add(listener); - } - public void associateConnection(Object conn) throws ResourceException { - if (!(conn instanceof GFConnectionImpl)) { + @Override + public void associateConnection(Object connection) throws ResourceException { + if (!(connection instanceof GFConnectionImpl)) { throw new ResourceException("Connection is not of type GFConnection"); } - ((GFConnectionImpl) conn).resetManagedConnection(this); - this.connections.add((GFConnectionImpl) conn); + ((GFConnectionImpl) connection).resetManagedConnection(this); + this.connections.add((GFConnectionImpl) connection); } + @Override public void cleanup() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCAManagedConnection:cleanup"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } synchronized (this.connections) { - Iterator connsItr = this.connections.iterator(); - while (connsItr.hasNext()) { - GFConnectionImpl conn = connsItr.next(); - conn.invalidate(); - connsItr.remove(); + Iterator iterator = this.connections.iterator(); + while (iterator.hasNext()) { + GFConnectionImpl connection = iterator.next(); + connection.invalidate(); + iterator.remove(); } } - if (this.localTran == null || this.localTran.transactionInProgress()) { - if (this.initDone && !this.cache.isClosed()) { - this.localTran = new JCALocalTransaction(cache, gfTxMgr); + if (this.localTransaction == null || this.localTransaction.transactionInProgress()) { + if (this.initialized && !this.cache.isClosed()) { + this.localTransaction = new JCALocalTransaction(this.cache, this.transactionManager); } else { - this.localTran = new JCALocalTransaction(); + this.localTransaction = new JCALocalTransaction(); } } - } + @Override public void destroy() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCAManagedConnection:destroy"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } synchronized (this.connections) { - Iterator connsItr = this.connections.iterator(); - while (connsItr.hasNext()) { - GFConnectionImpl conn = connsItr.next(); - conn.invalidate(); - connsItr.remove(); + Iterator iterator = this.connections.iterator(); + while (iterator.hasNext()) { + GFConnectionImpl connection = iterator.next(); + connection.invalidate(); + iterator.remove(); } } - this.gfTxMgr = null; + this.transactionManager = null; this.cache = null; - this.localTran = null; + this.localTransaction = null; this.listeners.clear(); } + @Override public Object getConnection(Subject arg0, ConnectionRequestInfo arg1) throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCAManagedConnection:getConnection"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } + if (!this.initialized || this.cache.isClosed()) { + init(); } - try { - if (!this.initDone || this.cache.isClosed()) { - init(); - } - LogWriter logger = this.cache.getLogger(); - if (logger.fineEnabled()) { - logger.fine("JCAManagedConnection:getConnection. Returning new Connection"); - } - - GFConnectionImpl conn = new GFConnectionImpl(this); - this.connections.add(conn); - return conn; - } catch (SystemException e) { - this.onError(e); - throw new ResourceException("GemFire Resource unavailable", e); + LogWriter logger = this.cache.getLogger(); + if (logger.fineEnabled()) { + logger.fine("JCAManagedConnection:getConnection. Returning new Connection"); } + + GFConnectionImpl connection = new GFConnectionImpl(this); + this.connections.add(connection); + return connection; } - private void init() throws SystemException { - this.cache = (GemFireCacheImpl) CacheFactory.getAnyInstance(); + private void init() { + this.cache = (InternalCache) CacheFactory.getAnyInstance(); LogWriter logger = this.cache.getLogger(); if (logger.fineEnabled()) { logger.fine("JCAManagedConnection:init. Inside init"); } - gfTxMgr = cache.getTxManager(); - this.initDone = true; + this.transactionManager = this.cache.getTxManager(); + this.initialized = true; } + @Override public LocalTransaction getLocalTransaction() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCAManagedConnection:getLocalTransaction"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } - - return this.localTran; + return this.localTransaction; } + @Override public PrintWriter getLogWriter() throws ResourceException { - return this.logger; + return this.logWriter; } + @Override public ManagedConnectionMetaData getMetaData() throws ResourceException { - if (DEBUG) { - try { - throw new NullPointerException("Asif:JCAManagedConnection:getMetaData"); - } catch (NullPointerException npe) { - npe.printStackTrace(); - } - } - if (this.initDone && !this.cache.isClosed()) { + if (this.initialized && !this.cache.isClosed()) { LogWriter logger = this.cache.getLogger(); if (logger.fineEnabled()) { logger.fine("JCAManagedConnection:getMetaData"); } } - return new JCAManagedConnectionMetaData(this.factory.getProductName(), - this.factory.getVersion(), this.factory.getUserName()); + return new JCAManagedConnectionMetaData(this.connectionFactory.getProductName(), + this.connectionFactory.getVersion(), this.connectionFactory.getUserName()); } + @Override public XAResource getXAResource() throws ResourceException { throw new NotSupportedException("XA Transaction not supported"); } + @Override public void removeConnectionEventListener(ConnectionEventListener arg0) { this.listeners.remove(arg0); } + @Override public void setLogWriter(PrintWriter logger) throws ResourceException { - this.logger = logger; + this.logWriter = logger; } - private void onError(Exception e) { - - this.localTran = null; + private void onError(Exception e) { // TODO: currently unused + this.localTransaction = null; synchronized (this.connections) { - Iterator connsItr = this.connections.iterator(); - while (connsItr.hasNext()) { - GFConnectionImpl conn = connsItr.next(); - conn.invalidate(); + Iterator iterator = this.connections.iterator(); + while (iterator.hasNext()) { + GFConnectionImpl connection = iterator.next(); + connection.invalidate(); + synchronized (this.listeners) { - Iterator itr = this.listeners.iterator(); - ConnectionEvent ce = + ConnectionEvent event = new ConnectionEvent(this, ConnectionEvent.CONNECTION_ERROR_OCCURRED, e); - ce.setConnectionHandle(conn); - while (itr.hasNext()) { - itr.next().connectionErrorOccurred(ce); + event.setConnectionHandle(connection); + for (ConnectionEventListener listener : this.listeners) { + listener.connectionErrorOccurred(event); } } - connsItr.remove(); + + iterator.remove(); } } - } - public void onClose(GFConnectionImpl conn) throws ResourceException { - conn.invalidate(); - this.connections.remove(conn); + public void onClose(GFConnectionImpl connection) { + connection.invalidate(); + this.connections.remove(connection); + synchronized (this.listeners) { - Iterator itr = this.listeners.iterator(); - ConnectionEvent ce = new ConnectionEvent(this, ConnectionEvent.CONNECTION_CLOSED); - ce.setConnectionHandle(conn); - while (itr.hasNext()) { - itr.next().connectionClosed(ce); + Iterator iterator = this.listeners.iterator(); + ConnectionEvent event = new ConnectionEvent(this, ConnectionEvent.CONNECTION_CLOSED); + event.setConnectionHandle(connection); + while (iterator.hasNext()) { + iterator.next().connectionClosed(event); } } + if (this.connections.isEmpty()) { - // safe to dissociate this managedconnection so that it can go to pool - if (this.initDone && !this.cache.isClosed()) { - this.localTran = new JCALocalTransaction(this.cache, this.gfTxMgr); + // safe to dissociate this managed connection so that it can go to pool + if (this.initialized && !this.cache.isClosed()) { + this.localTransaction = new JCALocalTransaction(this.cache, this.transactionManager); } else { - this.localTran = new JCALocalTransaction(); + this.localTransaction = new JCALocalTransaction(); } } - } } http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/geode-core/src/main/java/org/apache/geode/DataSerializer.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/DataSerializer.java b/geode-core/src/main/java/org/apache/geode/DataSerializer.java index 58518f4..fa6d6da 100644 --- a/geode-core/src/main/java/org/apache/geode/DataSerializer.java +++ b/geode-core/src/main/java/org/apache/geode/DataSerializer.java @@ -59,6 +59,7 @@ import org.apache.geode.internal.Version; import org.apache.geode.internal.cache.CachedDeserializable; 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.tier.sockets.ClientProxyMembershipID; import org.apache.geode.internal.i18n.LocalizedStrings; import org.apache.geode.internal.logging.LogService; @@ -1837,7 +1838,7 @@ public abstract class DataSerializer { typeString = readString(in); } - GemFireCacheImpl cache = GemFireCacheImpl.getInstance(); + InternalCache cache = GemFireCacheImpl.getInstance(); boolean lookForPdxInstance = false; ClassNotFoundException cnfEx = null; if (typeCode == DSCODE.CLASS && cache != null http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java b/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java index 5b4e59e..9dda7c1 100755 --- a/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java +++ b/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java @@ -14,6 +14,16 @@ */ package org.apache.geode.admin; +import java.io.IOException; +import java.io.Serializable; +import java.net.InetAddress; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + import org.apache.geode.cache.Cache; import org.apache.geode.cache.Region; import org.apache.geode.cache.client.PoolManager; @@ -24,17 +34,18 @@ import org.apache.geode.distributed.internal.DM; import org.apache.geode.distributed.internal.DistributionConfig; import org.apache.geode.distributed.internal.InternalDistributedSystem; import org.apache.geode.distributed.internal.membership.InternalDistributedMember; -import org.apache.geode.internal.net.SocketCreator; import org.apache.geode.internal.admin.ClientHealthMonitoringRegion; import org.apache.geode.internal.admin.remote.ClientHealthStats; -import org.apache.geode.internal.cache.*; +import org.apache.geode.internal.cache.CacheClientStatus; +import org.apache.geode.internal.cache.HARegion; +import org.apache.geode.internal.cache.InternalCache; +import org.apache.geode.internal.cache.LocalRegion; +import org.apache.geode.internal.cache.PartitionedRegion; +import org.apache.geode.internal.cache.PartitionedRegionStatus; +import org.apache.geode.internal.cache.RegionStatus; import org.apache.geode.internal.cache.tier.InternalClientMembership; import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID; - -import java.io.IOException; -import java.io.Serializable; -import java.net.InetAddress; -import java.util.*; +import org.apache.geode.internal.net.SocketCreator; /** * Class GemFireMemberStatus provides the status of a specific GemFire member VM. This @@ -528,7 +539,7 @@ public class GemFireMemberStatus implements Serializable { // Get Client Health Stats // Assert.assertTrue(cache != null); (cannot be null) Region clientHealthMonitoringRegion = - ClientHealthMonitoringRegion.getInstance((GemFireCacheImpl) cache); + ClientHealthMonitoringRegion.getInstance((InternalCache) cache); if (clientHealthMonitoringRegion != null) { String[] clients = (String[]) clientHealthMonitoringRegion.keySet().toArray(new String[0]); for (int i = 0; i < clients.length; i++) { http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java index f7ff9ed..434ea5a 100644 --- a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java +++ b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java @@ -14,6 +14,8 @@ */ package org.apache.geode.admin.internal; +import static java.lang.Long.*; + import java.util.List; import org.apache.logging.log4j.Logger; @@ -28,13 +30,13 @@ import org.apache.geode.internal.OSProcess; import org.apache.geode.internal.cache.CacheLifecycleListener; 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.logging.LogService; /** - * Contains the logic for evaluating the health of a GemFire Cache instance according - * to the thresholds provided in a {@link CacheHealthConfig}. - * + * Contains the logic for evaluating the health of a GemFire {@code Cache} instance according to the + * thresholds provided in a {@link CacheHealthConfig}. * * @since GemFire 3.5 */ @@ -42,15 +44,15 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec private static final Logger logger = LogService.getLogger(); - /** The config from which we get the evaulation criteria */ - private CacheHealthConfig config; + /** The config from which we get the evaluation criteria */ + private final CacheHealthConfig config; /** The description of the cache being evaluated */ private String description; /** - * Statistics about the Cache instance. If no cache has been created in this VM, this - * field will be null + * Statistics about the {@code Cache} instance. If no cache has been created in this VM, this + * field will be {@code null} */ private CachePerfStats cacheStats; @@ -69,21 +71,18 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec /** The previous value of the gets stat */ private long prevGets; - ////////////////////// Constructors ////////////////////// - /** - * Creates a new CacheHealthEvaluator + * Creates a new {@code CacheHealthEvaluator} */ CacheHealthEvaluator(GemFireHealthConfig config, DM dm) { super(config, dm); this.config = config; InternalDistributedSystem system = dm.getSystem(); - GemFireCacheImpl cache; + InternalCache cache; try { - cache = (GemFireCacheImpl) CacheFactory.getInstance(system); - - } catch (CancelException ex) { + cache = (InternalCache) CacheFactory.getInstance(system); + } catch (CancelException ignore) { // No cache in this VM cache = null; } @@ -92,8 +91,6 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec GemFireCacheImpl.addCacheLifecycleListener(this); } - //////////////////// Instance Methods //////////////////// - @Override protected String getDescription() { return this.description; @@ -102,14 +99,14 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec /** * Initializes the state of this evaluator based on the given cache instance. */ - private void initialize(GemFireCacheImpl cache, DM dm) { - StringBuffer sb = new StringBuffer(); + private void initialize(InternalCache cache, DM dm) { + StringBuilder sb = new StringBuilder(); if (cache != null) { this.cacheStats = cache.getCachePerfStats(); sb.append("Cache \""); sb.append(cache.getName()); - sb.append("\""); + sb.append('"'); } else { sb.append("No Cache"); @@ -125,21 +122,22 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec this.description = sb.toString(); } - public void cacheCreated(GemFireCacheImpl cache) { + @Override + public void cacheCreated(InternalCache cache) { InternalDistributedSystem system = (InternalDistributedSystem) cache.getDistributedSystem(); DM dm = system.getDistributionManager(); initialize(cache, dm); } /** - * Checks to make sure that the average netSearch time during the previous health - * check interval is less than the {@linkplain CacheHealthConfig#getMaxNetSearchTime threshold}. - * If not, the status is "okay" health. + * Checks to make sure that the average {@code netSearch} time during the previous health check + * interval is less than the {@linkplain CacheHealthConfig#getMaxNetSearchTime threshold}. If not, + * the status is "okay" health. * * @see CachePerfStats#getNetsearchTime * @see CachePerfStats#getNetsearchesCompleted */ - void checkNetSearchTime(List status) { + private void checkNetSearchTime(List status) { if (this.cacheStats == null || isFirstEvaluation() || this.cacheStats.isClosed()) { return; } @@ -156,21 +154,21 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec if (ratio > threshold) { String s = LocalizedStrings.CacheHealthEvaluator_THE_AVERAGE_DURATION_OF_A_CACHE_NETSEARCH_0_MS_EXCEEDS_THE_THRESHOLD_1_MS - .toLocalizedString(new Object[] {ratio, threshold}); + .toLocalizedString(ratio, threshold); status.add(okayHealth(s)); } } } /** - * Checks to make sure that the average load time during the previous health check + * Checks to make sure that the average {@code load} time during the previous health check * interval is less than the {@linkplain CacheHealthConfig#getMaxLoadTime threshold}. If not, the * status is "okay" health. * * @see CachePerfStats#getLoadTime * @see CachePerfStats#getLoadsCompleted */ - void checkLoadTime(List status) { + private void checkLoadTime(List status) { if (this.cacheStats == null || isFirstEvaluation() || this.cacheStats.isClosed()) { return; } @@ -180,7 +178,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec long deltaLoadsCompleted = this.cacheStats.getLoadsCompleted() - this.prevLoadsCompleted; if (logger.isDebugEnabled()) { - logger.debug("Completed {} loads in {} ms", deltaLoadsCompleted, (deltaLoadTime / 1000000)); + logger.debug("Completed {} loads in {} ms", deltaLoadsCompleted, deltaLoadTime / 1000000); } if (deltaLoadsCompleted != 0) { @@ -191,7 +189,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec if (ratio > threshold) { String s = LocalizedStrings.CacheHealthEvaluator_THE_AVERAGE_DURATION_OF_A_CACHE_LOAD_0_MS_EXCEEDS_THE_THRESHOLD_1_MS - .toLocalizedString(new Object[] {ratio, threshold}); + .toLocalizedString(ratio, threshold); if (logger.isDebugEnabled()) { logger.debug(s); } @@ -219,7 +217,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec * @see CachePerfStats#getLoadsCompleted * @see CachePerfStats#getNetsearchesCompleted */ - void checkHitRatio(List status) { + private void checkHitRatio(List status) { if (this.cacheStats == null || isFirstEvaluation() || this.cacheStats.isClosed()) { return; } @@ -230,12 +228,12 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec long deltaNetsearchesCompleted = this.cacheStats.getNetsearchesCompleted() - this.prevNetsearchesCompleted; - double hits = (deltaGets - (deltaLoadsCompleted + deltaNetsearchesCompleted)); + double hits = deltaGets - (deltaLoadsCompleted + deltaNetsearchesCompleted); double hitRatio = hits / deltaGets; double threshold = this.config.getMinHitRatio(); if (hitRatio < threshold) { String s = "The hit ratio of this Cache (" + hitRatio + ") is below the threshold (" - + threshold + ")"; + + threshold + ')'; status.add(okayHealth(s)); } } @@ -246,7 +244,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec * size} does not exceed the {@linkplain CacheHealthConfig#getMaxEventQueueSize threshold}. If it * does, the status is "okay" health. */ - void checkEventQueueSize(List status) { + private void checkEventQueueSize(List status) { if (this.cacheStats == null || isFirstEvaluation() || this.cacheStats.isClosed()) { return; } @@ -256,13 +254,11 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec if (eventQueueSize > threshold) { String s = LocalizedStrings.CacheHealthEvaluator_THE_SIZE_OF_THE_CACHE_EVENT_QUEUE_0_MS_EXCEEDS_THE_THRESHOLD_1_MS - .toLocalizedString( - new Object[] {Long.valueOf(eventQueueSize), Long.valueOf(threshold)}); + .toLocalizedString(valueOf(eventQueueSize), valueOf(threshold)); status.add(okayHealth(s)); } } - /** * Updates the previous values of statistics */ @@ -285,7 +281,6 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec @Override protected void check(List status) { - checkNetSearchTime(status); checkLoadTime(status); checkHitRatio(status); @@ -300,7 +295,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec } @Override - public void cacheClosed(GemFireCacheImpl cache) { + public void cacheClosed(InternalCache cache) { // do nothing } } http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java index 25abd7e..f01666d 100644 --- a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java +++ b/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java @@ -40,6 +40,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.i18n.LocalizedStrings; import org.apache.geode.internal.logging.LogService; import org.apache.geode.internal.logging.log4j.LocalizedMessage; @@ -92,7 +93,7 @@ public class FinishBackupRequest extends CliLegacyMessage { @Override protected AdminResponse createResponse(DistributionManager dm) { - GemFireCacheImpl cache = GemFireCacheImpl.getInstance(); + InternalCache cache = GemFireCacheImpl.getInstance(); HashSet persistentIds; if (cache == null || cache.getBackupManager() == null) { persistentIds = new HashSet(); http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/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..71348be 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 @@ -19,6 +19,7 @@ import java.util.HashSet; import java.util.Set; import org.apache.geode.CancelException; +import org.apache.geode.cache.DiskStore; import org.apache.geode.cache.persistence.PersistentID; import org.apache.geode.distributed.internal.DM; import org.apache.geode.distributed.internal.DistributionManager; @@ -28,6 +29,7 @@ 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; /** * A request to from an admin VM to all non admin members to start a backup. In the prepare phase of @@ -65,11 +67,10 @@ public class FlushToDiskRequest extends CliLegacyMessage { @Override protected AdminResponse createResponse(DistributionManager dm) { - GemFireCacheImpl cache = GemFireCacheImpl.getInstance(); - HashSet persistentIds; + InternalCache cache = GemFireCacheImpl.getInstance(); if (cache != null) { - Collection diskStores = cache.listDiskStoresIncludingRegionOwned(); - for (DiskStoreImpl store : diskStores) { + Collection diskStores = cache.listDiskStoresIncludingRegionOwned(); + for (DiskStore store : diskStores) { store.flush(); } } http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/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 MemberHealthEvaluator */ @@ -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/0d0bf253/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 persistentIds; if (cache == null) { - persistentIds = new HashSet(); + persistentIds = new HashSet<>(); } else { try { BackupManager manager = cache.startBackup(getSender()); http://git-wip-us.apache.org/repos/asf/geode/blob/0d0bf253/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 AttributesFactory instance - * maintains state for creating 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 RegionAttributes, 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. * *

- * Once a RegionAttributes is created, it can only be modified after it has been used - * to create a Region, 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. * *

Attributes

*

Callbacks

*
- *
{@link CacheLoader} [default: null, meaning no loader]
+ *
{@link CacheLoader} [default: {@code null}, meaning no loader]
*
User-implemented plug-in for loading data on cache misses.
* {@link #setCacheLoader} {@link RegionAttributes#getCacheLoader} * {@link AttributesMutator#setCacheLoader}
* - *
{@link CacheWriter} [default: null, meaning no writer]
+ *
{@link CacheWriter} [default: {@code null}, meaning no writer]
*
User-implemented plug-in for intercepting cache modifications, e.g. for writing to an * external data source.
* {@link #setCacheWriter} {@link RegionAttributes#getCacheWriter} * {@link AttributesMutator#setCacheWriter}
* - *
{@link CacheListener} [default: null, meaning no listener ]
+ *
{@link CacheListener} [default: {@code null}, meaning no listener ]
*
User-implemented plug-in for receiving and handling cache related events.
* {@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.
* {@link #setScope} {@link RegionAttributes#getScope}
* - *
EarlyAck [default: false]
- *
Whether or not acks required by Scope.DISTRIBUTED_ACK are sent after an - * operation is processed. If true then remote caches will ACK before processing an - * operation sent by the cache that has set earlyAck to 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.
+ *
EarlyAck [default: {@code false}]
+ *
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.
* {@link #setEarlyAck} {@link RegionAttributes#getEarlyAck}
* *
{@link SubscriptionAttributes} [default: {@link InterestPolicy#DEFAULT}]
@@ -119,44 +117,42 @@ import org.apache.geode.internal.i18n.LocalizedStrings; *
* {@link #setSubscriptionAttributes} {@link RegionAttributes#getSubscriptionAttributes} * - *
EnableAsyncConflation [default: false]
+ *
EnableAsyncConflation [default: {@code false}]
*
Whether or not conflation is enabled for sending messages to async peers. Async peers are - * those whose async-distribution-timeout gemfire.property is greater than zero. - * AsyncConflation is ignored if the scope is DISTRIBUTED_ACK or GLOBAL. - * 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}
- *
poolName [default: null, meaning no pool]
+ *
poolName [default: {@code null}, meaning no pool]
*
Whether or not this region is a client that is to use connections from the named pool to - * communicate with servers. If null, then it is not a client. If - * non-null, then the named pool will be used. {@link #setPoolName} - * {@link RegionAttributes#getPoolName}
+ * 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} * * - *
EnableSubscriptionConflation [default: false]
+ *
EnableSubscriptionConflation [default: {@code false}]
*
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 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} + * 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}
- *
Publisher [default: false]
+ *
Publisher [default: {@code false}]
*
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}
- *
isCloningEnabled [default: false]
- *
Whether or not value is cloned before appling Deltas If false, - * value will not be cloned {@link #setCloningEnabled} - * {@link RegionAttributes#getCloningEnabled()}
+ *
isCloningEnabled [default: {@code false}]
+ *
Whether or not value is cloned before appling {@code Delta}s If {@code false}, value will not + * be cloned {@link #setCloningEnabled} {@link RegionAttributes#getCloningEnabled()}
*
*

Storage (see also package summary )

*
- *
{@link DataPolicy} [default: DataPolicy.NORMAL]
+ *
{@link DataPolicy} [default: {@code DataPolicy.NORMAL}]
*
Specifies the data storage policy.
* {@link #setDataPolicy} {@link RegionAttributes#getDataPolicy}
* - *
{@link MirrorType} [default: MirrorType.NONE]
+ *
{@link MirrorType} [default: {@code MirrorType.NONE}]
*
Deprecated, use DataPolicy instead.
* *
{@link #setEvictionAttributes(EvictionAttributes) EvictionAttributes}
@@ -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 EvictionAttributes installs an eviction controller - * on the Region instantiated with the associated RegionAttributes + * heap has been consumed. Setting {@code EvictionAttributes} installs an eviction controller on the + * Region instantiated with the associated RegionAttributes * - *
KeyConstraint [default: null, meaning no constraint]
+ *
KeyConstraint [default: {@code null}, meaning no constraint]
*
The Class to constrain the keys to in the region.
* {@link #setKeyConstraint} {@link RegionAttributes#getKeyConstraint}
* - *
ValueConstraint [default: null, meaning no constraint]
+ *
ValueConstraint [default: {@code null}, meaning no constraint]
*
The Class to constrain the values to in the region. In addition to the utility of this for - * applications in general, a valueConstraint is helpful for compiling queries.
+ * applications in general, a {@code valueConstraint} is helpful for compiling queries.
* {@link #setValueConstraint} {@link RegionAttributes#getValueConstraint}
* - *
InitialCapacity [default: 16]
+ *
InitialCapacity [default: {@code 16}]
*
The initial capacity of the map used for storing the entries.
* {@link java.util.HashMap} {@link #setInitialCapacity} * {@link RegionAttributes#getInitialCapacity}
* - *
LoadFactor [default: 0.75]
+ *
LoadFactor [default: {@code 0.75}]
*
The load factor of the map used for storing the entries.
* {@link java.util.HashMap} {@link #setLoadFactor} {@link RegionAttributes#getLoadFactor}
* - *
ConcurrencyLevel [default: 16]
+ *
ConcurrencyLevel [default: {@code 16}]
*
The allowed concurrency among updates to values in the region is guided by the * concurrencyLevel, 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.
* {@link #setConcurrencyLevel} {@link RegionAttributes#getConcurrencyLevel}
* - *
ConcurrencyChecksEnabled [default: false]
+ *
ConcurrencyChecksEnabled [default: {@code false}]
*
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.
* - *
StatisticsEnabled [default: false]
+ *
StatisticsEnabled [default: {@code false}]
*
Whether statistics are enabled for this region. The default is disabled, which conserves on * memory.
* {@link #setStatisticsEnabled} {@link RegionAttributes#getStatisticsEnabled}
* - *
IgnoreJTA [default: false]
+ *
IgnoreJTA [default: {@code false}]
*
Whether JTA transactions are ignored for this region. The default is to look for and join JTA * transactions for operations performed on a region. * - *
DiskStoreName [default: null, meaning no disk store]
- *
If not null then this region will write its data to the named - * {@link DiskStore}.
+ *
DiskStoreName [default: {@code null}, meaning no disk store]
+ *
If not {@code null} then this region will write its data to the named {@link DiskStore}.
* {@link #setDiskStoreName} {@link RegionAttributes#getDiskStoreName}
* - *
DiskSynchronous [default: true]
- *
If 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 + *
DiskSynchronous [default: {@code true}]
+ *
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.
- * If 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.
+ * 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.
* {@link #setDiskSynchronous} {@link RegionAttributes#isDiskSynchronous}
* - *
PersistBackup [default: false]
+ *
PersistBackup [default: {@code false}]
*
Whether or not a persistent backup should be made of the region.
* {@link #setPersistBackup} {@link RegionAttributes#getPersistBackup}
*
Deprecated, use {@link DataPolicy#PERSISTENT_REPLICATE} or * {@link DataPolicy#PERSISTENT_PARTITION} instead.
* *
DiskWriteAttributes [default: Asynchronously write to disk every second (a - * timeInterval of 1000 and a byteThreshold of 0). rollOplogs - * is set to true and maxOplogSize is set to 1024 MB]
+ * {@code timeInterval} of 1000 and a {@code byteThreshold} of 0). {@code rollOplogs} is set to true + * and {@code maxOplogSize} is set to 1024 MB] *
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; *
Deprecated, use {@link #setDiskStoreName} and {@link #setDiskSynchronous} * instead.
* - *
DiskDirs [default: Current working directory (user.dir + *
DiskDirs [default: Current working directory ({@code user.dir} * {@linkplain System#getProperties system property})]
*
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.
@@ -258,7 +253,7 @@ import org.apache.geode.internal.i18n.LocalizedStrings; *
Deprecated, use {@link #setDiskStoreName} instead.
* * - *
{@link PartitionAttributes} [default: null, meaning no region + *
{@link PartitionAttributes} [default: {@code null}, meaning no region * partitioning]
*
How region data is partitioned among the members of the distributed system.
* {@link #setPartitionAttributes} {@link RegionAttributes#getPartitionAttributes}
@@ -272,14 +267,14 @@ import org.apache.geode.internal.i18n.LocalizedStrings; * *

Locking

*
- *
LockGrantor [default: false]
+ *
LockGrantor [default: {@code false}]
*
Should this process become lock grantor for the region?

* {@link #setLockGrantor} {@link RegionAttributes#isLockGrantor} {@link Region#becomeLockGrantor} *
* *

Querying

*
- *
IndexMaintenanceSynchronous [default: false]
+ *
IndexMaintenanceSynchronous [default: {@code false}]
*
Are indexes built over in this region updated synchronously when the underlying data is * modified?

* {@link #setIndexMaintenanceSynchronous} {@link RegionAttributes#getIndexMaintenanceSynchronous} @@ -291,29 +286,26 @@ import org.apache.geode.internal.i18n.LocalizedStrings; * *

Compatibility Rules

*

RegionAttributes Creation Constraints

If any of the following compatibility rules are - * violated when {@link #create} 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}. * *
*

Creation Constraints

*

Region Creation Constraints on RegionAttributes

* * If any of the following rules are violated when {@link Region#createSubregion createSubregion} or - * {@link Cache#createRegion createRegion} are called, then an IllegalStateException is + * {@link Cache#createRegion createRegion} are called, then an {@code IllegalStateException} is * thrown. * *
    - *
  • A region with Scope.LOCAL can only have subregions with - * Scope.LOCAL.
  • - *
  • Scope.GLOBAL is illegal if there is any other cache in the distributed system - * that has the same region with Scope.DISTRIBUTED_NO_ACK or - * Scope.DISTRIBUTED_ACK.
  • - *
  • Scope.DISTRIBUTED_ACK is illegal if there is any other cache in the distributed - * system that has the same region with Scope.DISTRIBUTED_NO_ACK or - * Scope.GLOBAL.
  • - *
  • Scope.DISTRIBUTED_NO_ACK is illegal if there is any other cache in the - * distributed system that has the same region with Scope.DISTRIBUTED_ACK or - * Scope.GLOBAL.
  • + *
  • A region with {@code Scope.LOCAL} can only have subregions with {@code Scope.LOCAL}.
  • + *
  • {@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}.
  • + *
  • {@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}.
  • + *
  • {@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}.
  • *
* * @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 { private final RegionAttributesImpl regionAttributes = new RegionAttributesImpl(); @@ -331,24 +324,23 @@ public class AttributesFactory { /** * The default disk synchronous write setting *

- * Current value: true 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 RegionAttributes - * 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 RegionAttributes - * with the same settings as those in the specified RegionAttributes. + * 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 RegionAttributes used to initialize this - * AttributesFactory + * @param regionAttributes the {@code RegionAttributes} used to initialize this AttributesFactory */ @SuppressWarnings("deprecation") public AttributesFactory(RegionAttributes regionAttributes) { @@ -448,7 +440,7 @@ public class AttributesFactory { // CALLBACKS /** - * Sets the cache loader for the next RegionAttributes 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 { } /** - * Sets the cache writer for the next RegionAttributes 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 { } /** - * Sets the CacheListener for the next RegionAttributes 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 { * 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 aListener is null + * @throws IllegalArgumentException if {@code aListener} is null * @since GemFire 5.0 */ public void addCacheListener(CacheListener aListener) { @@ -513,7 +505,7 @@ public class AttributesFactory { * 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 newListeners array has a null element + * @throws IllegalArgumentException if the {@code newListeners} array has a null element * @since GemFire 5.0 */ public void initCacheListeners(CacheListener[] newListeners) { @@ -538,7 +530,7 @@ public class AttributesFactory { /** * Sets the idleTimeout expiration attributes for region entries for the next - * RegionAttributes 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 { } /** - * Sets the idleTimeout CustomExpiry for the next RegionAttributes 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 { /** * Sets the timeToLive expiration attributes for region entries for the next - * RegionAttributes 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 { } /** - * Sets the custom timeToLive for the next RegionAttributes 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 { /** * Sets the idleTimeout expiration attributes for the region itself for the next - * RegionAttributes 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 { /** * Sets the timeToLive expiration attributes for the region itself for the next - * RegionAttributes 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 { // DISTRIBUTION ATTRIBUTES /** - * Sets the scope for the next RegionAttributes 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 { // STORAGE ATTRIBUTES /** - * Sets the EvictionController for the next RegionAttributes 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 - * AttributesFactory + * {@code AttributesFactory} * * @param evictAttrs Explains how and when eviction occurs in the Region. */ @@ -663,7 +655,7 @@ public class AttributesFactory { } /** - * Sets the mirror type for the next RegionAttributes 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 { } /** - * Sets the data policy for the next RegionAttributes 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 { /** - * Sets the key constraint for the next 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 ClassCastException 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 keyConstraint is a class denoting a primitive - * type + * @throws IllegalArgumentException if {@code keyConstraint} is a class denoting a primitive type */ public void setKeyConstraint(Class keyConstraint) { if (keyConstraint != null && keyConstraint.isPrimitive()) @@ -734,13 +725,13 @@ public class AttributesFactory { } /** - * Sets the value constraint for the next 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 ClassCastException 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 valueConstraint is a class denoting a - * primitive type + * @throws IllegalArgumentException if {@code valueConstraint} is a class denoting a primitive + * type */ public void setValueConstraint(Class valueConstraint) { if (valueConstraint != null && valueConstraint.isPrimitive()) @@ -755,8 +746,8 @@ public class AttributesFactory { // MAP ATTRIBUTES /** - * Sets the entry initial capacity for the next RegionAttributes 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 { } /** - * Sets the entry load factor for the next RegionAttributes 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 { } /** - * Sets the concurrency level of the next RegionAttributes 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 { * adds a gateway sender to the end of list of gateway senders on this factory * * @param gatewaySenderId - * @throws IllegalArgumentException if gatewaySender 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 { * Adds a AsyncEventQueue to the end of list of async event queues on this factory * * @param asyncEventQueueId - * @throws IllegalArgumentException if gatewaySender 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 { } /** - * Sets the 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. + * 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 { } /** - * Sets the MembershipAttributes 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 SubscriptionAttributes 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 { * 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 { * 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. *

- * If this attribute is set to null or "" 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. *

* 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 null or "" - * 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 { // FACTORY METHOD /** - * Creates a RegionAttributes with the current settings. + * Creates a {@code RegionAttributes} with the current settings. * - * @return the newly created RegionAttributes + * @return the newly created {@code RegionAttributes} * @throws IllegalStateException if the current settings violate the * compatibility rules * @deprecated as of GemFire 5.0, use {@link #create} instead @@ -1281,9 +1271,9 @@ public class AttributesFactory { } /** - * Creates a RegionAttributes with the current settings. + * Creates a {@code RegionAttributes} with the current settings. * - * @return the newly created RegionAttributes + * @return the newly created {@code RegionAttributes} * @throws IllegalStateException if the current settings violate the * compatibility rules * @since GemFire 5.0 @@ -1612,7 +1602,7 @@ public class AttributesFactory { boolean offHeap = false; /** - * Constructs an instance of RegionAttributes with default settings. + * Constructs an instance of {@code RegionAttributes} with default settings. * * @see AttributesFactory */ @@ -1956,6 +1946,7 @@ public class AttributesFactory { /** * @deprecated this API is scheduled to be removed */ + @Deprecated public MembershipAttributes getMembershipAttributes() { return this.membershipAttributes; }