{
+
+ public String indexName;
+
+ public IndexType indexType;
+
+ public IndexCreationHelper helper;
+
+ public String origFromClause;
+
+ public String origIndexedExpression;
+
+ public boolean isCompactOrHash = false;
+
+ public boolean isLDM = false;
+
+ public PartitionedIndex prIndex;
+
+ public boolean loadEntries;
+
+ IndexTask (String indexName, IndexType type, String origFromClause, String origIndexedExpression, IndexCreationHelper helper, boolean isCompactOrHash, PartitionedIndex prIndex, boolean loadEntries){
+ this.indexName = indexName;
+ this.indexType = type;
+ this.origFromClause = origFromClause;
+ this.origIndexedExpression = origIndexedExpression;
+ this.helper = helper;
+ this.isCompactOrHash = isCompactOrHash;
+ this.prIndex = prIndex;
+ this.loadEntries = loadEntries;
+ }
+
+ /* For name based index search */
+ IndexTask (String indexName){
+ this.indexName = indexName;
+ }
+
+ @Override
+ public boolean equals (Object other){
+ if (other == null) {
+ return false;
+ }
+ IndexTask otherIndexTask = (IndexTask) other;
+ // compare indexName.
+ if (this.indexName.equals(otherIndexTask.indexName)){
+ return true;
+ }
+
+ if (otherIndexTask.helper == null || this.helper == null) {
+ return false;
+ }
+
+ String[] indexDefinitions = this.helper.getCanonicalizedIteratorDefinitions();
+ int[] mapping = new int[indexDefinitions.length];
+ // compare index based on its type, expression and definition.
+ if(compareIndexData(this.indexType, indexDefinitions,
+ this.helper.getCanonicalizedIndexedExpression(), otherIndexTask.indexType,
+ otherIndexTask.helper.getCanonicalizedIteratorDefinitions(),
+ otherIndexTask.helper.getCanonicalizedIndexedExpression(), mapping) == 0)
+ {
+ return true;
+ }
+ return false;
+ }
+
+ public int hashCode(){
+ // It returns a constant number as the equality check is based on
+ // the OR condition between the indexName and its characteristics
+ // (involving type, expression and definition), because of this
+ // its not possible to come-up with an accurate hash code.
+ return 99;
+ }
+
+ /*
+ * Creates and initializes the index.
+ */
+ public Index call() {
+ Index index = null;
+ String indexedExpression = helper.getCanonicalizedIndexedExpression();
+ String fromClause = helper.getCanonicalizedFromClause();
+ String projectionAttributes = helper.getCanonicalizedProjectionAttributes();
+ String[] definitions = helper.getCanonicalizedIteratorDefinitions();
+ IndexStatistics stats = null;
+ this.isLDM = IndexManager.IS_TEST_LDM;
+
+ if (this.prIndex != null) {
+ stats = this.prIndex.getStatistics();
+ }
+ //Hash index not supported for overflow but we "thought" we were so let's maintain backwards compatibility
+ //and create a regular compact range index instead
+ if (indexType == IndexType.HASH && isOverFlowRegion()) {
+ indexType = IndexType.FUNCTIONAL;
+ }
+ if (indexType == IndexType.PRIMARY_KEY) {
+ index = new PrimaryKeyIndex(indexName, region, fromClause,indexedExpression,
+ projectionAttributes, origFromClause,
+ origIndexedExpression, definitions, stats);
+ logger.info("Using Primary Key index implementation for '{}' on region {}", indexName, region.getFullPath());
+ } else if (indexType == IndexType.HASH){
+ index = new HashIndex(indexName, region, fromClause,
+ indexedExpression, projectionAttributes, origFromClause,
+ origIndexedExpression, definitions, stats);
+
+ logger.info("Using Hash index implementation for '{}' on region {}", indexName, region.getFullPath());
+ }
+ else {
+ //boolean isCompact = !helper.isMapTypeIndex() && shouldCreateCompactIndex((FunctionalIndexCreationHelper)helper);
+ if (this.isCompactOrHash || this.isLDM) {
+ if (indexType == IndexType.FUNCTIONAL && !helper.isMapTypeIndex()) {
+ index = new CompactRangeIndex(indexName, region, fromClause,
+ indexedExpression, projectionAttributes, origFromClause,
+ origIndexedExpression, definitions, stats);
+ logger.info("Using Compact Range index implementation for '{}' on region {}", indexName, region.getFullPath());
+ }
+ else {
+ FunctionalIndexCreationHelper fich = (FunctionalIndexCreationHelper)helper;
+ index = new CompactMapRangeIndex(indexName, region, fromClause, indexedExpression,
+ projectionAttributes, origFromClause,
+ origIndexedExpression, definitions, fich.isAllKeys(), fich.multiIndexKeysPattern,
+ fich.mapKeys, stats);
+ logger.info("Using Compact Map Range index implementation for '{}' on region {}", indexName, region.getFullPath());
+ }
+ }
+ else {
+ assert indexType == IndexType.FUNCTIONAL;
+ if ( !helper.isMapTypeIndex() ) {
+ index = new RangeIndex(indexName, region, fromClause,
+ indexedExpression, projectionAttributes, origFromClause,
+ origIndexedExpression, definitions, stats);
+ logger.info("Using Non-Compact index implementation for '{}' on region {}", indexName, region.getFullPath());
+ }
+ else {
+ FunctionalIndexCreationHelper fich = (FunctionalIndexCreationHelper)helper;
+ index = new MapRangeIndex(indexName, region, fromClause, indexedExpression,
+ projectionAttributes, origFromClause,
+ origIndexedExpression, definitions, fich.isAllKeys(), fich.multiIndexKeysPattern,
+ fich.mapKeys, stats);
+ logger.info("Using Non-Compact Map index implementation for '{}' on region {}", indexName, region.getFullPath());
+ }
+ }
+ }
+ ((AbstractIndex)index).setPRIndex(prIndex);
+
+ if (index.getType() != IndexType.PRIMARY_KEY) {
+ AbstractIndex aIndex = ((AbstractIndex) index);
+ aIndex.instantiateEvaluator(helper);
+ waitBeforeUpdate();
+ boolean indexCreatedSuccessfully = false;
+ try {
+ ((LocalRegion)region).setFlagForIndexCreationThread(true);
+ aIndex.initializeIndex(loadEntries);
+ logger.info((loadEntries ? "Initialized and loaded entries into the index "
+ : "Initialized but entries not yet loaded into the index "
+ + indexName + " on region: " + region.getFullPath()));
+ aIndex.markValid(true);
+ indexCreatedSuccessfully = true;
+ if(loadEntries) {
+ aIndex.setPopulated(true);
+ if (this.prIndex != null) {
+ ((AbstractIndex)this.prIndex).setPopulated(true);
+ }
+ }
+ indexes.put(this, index);
+ if (region instanceof BucketRegion && prIndex != null) {
+ prIndex.addToBucketIndexes(region, index);
+ prIndex.incNumBucketIndexes();
+ }
+ } catch (Exception e) {
+ throw new IndexInvalidException(e);
+ }
+ finally {
+ notifyAfterUpdate();
+ ((LocalRegion)region).setFlagForIndexCreationThread(false);
+ if (!indexCreatedSuccessfully){
+ ((InternalIndexStatistics)index.getStatistics()).close();
+ }
+ }
+ } else {
+ // For PrimaryKey index
+ ((AbstractIndex)index).setPopulated(true);
+ indexes.put(this, index);
+ if (region instanceof BucketRegion && prIndex != null) {
+ prIndex.addToBucketIndexes(region, index);
+ }
+ }
+ return index;
+ }
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java
new file mode 100644
index 000000000000..cb9dc2516f4c
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/*
+ * PrimaryIndexCreationHelper.java
+ *
+ * Created on March 20, 2005, 7:21 PM
+ */
+package com.gemstone.gemfire.cache.query.internal.index;
+
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.query.IndexInvalidException;
+import com.gemstone.gemfire.cache.query.internal.*;
+import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
+import java.util.List;
+
+/**
+ *
+ */
+public class PrimaryKeyIndexCreationHelper extends IndexCreationHelper {
+
+ ExecutionContext context = null;
+
+ final Region region;
+
+ public PrimaryKeyIndexCreationHelper(String fromClause,
+ String indexedExpression, String projectionAttributes, Cache cache,
+ ExecutionContext externalContext, IndexManager imgr) throws IndexInvalidException {
+ super(fromClause, projectionAttributes, cache);
+ if( externalContext == null) {
+ context = new ExecutionContext(null, cache);
+ }else {
+ this.context = externalContext;
+ }
+ context.newScope(1);
+ this.region = imgr.region;
+ prepareFromClause( imgr);
+ prepareIndexExpression(indexedExpression);
+ prepareProjectionAttributes(projectionAttributes);
+ }
+
+ private void prepareFromClause(IndexManager imgr)
+ throws IndexInvalidException {
+ List list = this.compiler.compileFromClause(fromClause);
+ if (list.size() > 1) { throw new IndexInvalidException(LocalizedStrings.PrimaryKeyIndexCreationHelper_THE_FROMCLAUSE_FOR_A_PRIMARY_KEY_INDEX_SHOULD_ONLY_HAVE_ONE_ITERATOR_AND_THE_COLLECTION_MUST_BE_A_REGION_PATH_ONLY.toLocalizedString()); }
+ try {
+ CompiledIteratorDef iterDef = (CompiledIteratorDef) list.get(0);
+ if (iterDef.getCollectionExpr().getType() != OQLLexerTokenTypes.RegionPath) { throw new IndexInvalidException(LocalizedStrings.PrimaryKeyIndexCreationHelper_THE_FROMCLAUSE_FOR_A_PRIMARY_KEY_INDEX_SHOULD_BE_A_REGION_PATH_ONLY.toLocalizedString()); }
+ iterDef.computeDependencies(this.context);
+ RuntimeIterator rIter = (iterDef.getRuntimeIterator(this.context));
+ String definition = rIter.getDefinition();
+ this.canonicalizedIteratorDefinitions = new String[1];
+ this.canonicalizedIteratorDefinitions[0] = definition;
+ // Asif: Bind the Index_Internal_ID to the RuntimeIterator
+ String name = imgr.putCanonicalizedIteratorNameIfAbsent(definition);
+ rIter.setIndexInternalID(name);
+ this.canonicalizedIteratorNames = new String[1];
+ this.canonicalizedIteratorNames[0] = name;
+ this.fromClause = new StringBuffer(definition).append(' ').append(name)
+ .toString();
+ context.bindIterator(rIter);
+ }
+ catch (IndexInvalidException e) {
+ throw e; //propagate
+ }
+ catch (Exception e) {
+ throw new IndexInvalidException(e); // wrap any other exceptions
+ }
+ }
+
+ private void prepareIndexExpression(String indexedExpression)
+ throws IndexInvalidException {
+ List indexedExprs = this.compiler
+ .compileProjectionAttributes(indexedExpression);
+ if (indexedExprs == null || indexedExprs.size() != 1) { throw new IndexInvalidException(LocalizedStrings.PrimaryKeyIndexCreationHelper_INVALID_INDEXED_EXPRESSOION_0.toLocalizedString(indexedExpression)); }
+ CompiledValue expr = (CompiledValue) ((Object[]) indexedExprs.get(0))[1];
+ if (expr.getType() == CompiledValue.LITERAL)
+ throw new IndexInvalidException(LocalizedStrings.PrimaryKeyIndexCreationHelper_INVALID_INDEXED_EXPRESSOION_0.toLocalizedString(indexedExpression));
+ try {
+ StringBuffer sb = new StringBuffer();
+ expr.generateCanonicalizedExpression(sb, context);
+ this.indexedExpression = sb.toString();
+ }
+ catch (Exception e) {
+ //e.printStackTrace();
+ throw new IndexInvalidException(LocalizedStrings.PrimaryKeyIndexCreationHelper_INVALID_INDEXED_EXPRESSOION_0_N_1.toLocalizedString(new Object[] {indexedExpression, e.getMessage()}));
+ }
+ }
+
+ private void prepareProjectionAttributes(String projectionAttributes)
+ throws IndexInvalidException {
+ if (projectionAttributes != null && !projectionAttributes.equals("*")) { throw new IndexInvalidException(LocalizedStrings.PrimaryKeyIndexCreationHelper_INVALID_PROJECTION_ATTRIBUTES_0.toLocalizedString(projectionAttributes)); }
+ this.projectionAttributes = projectionAttributes;
+ }
+
+ public Region getRegion() {
+ return region;
+ }
+
+ public List getIterators() {
+ return null;
+ }
+
+ public CompiledValue getCompiledIndexedExpression() {
+ return null;
+ }
+
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
new file mode 100644
index 000000000000..224e741ffbdf
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -0,0 +1,4804 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.NotSerializableException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.ForcedDisconnectException;
+import com.gemstone.gemfire.IncompatibleSystemException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.InternalGemFireException;
+import com.gemstone.gemfire.InvalidDeltaException;
+import com.gemstone.gemfire.SystemConnectException;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.ToDataException;
+import com.gemstone.gemfire.admin.GemFireHealthConfig;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.Role;
+import com.gemstone.gemfire.distributed.internal.locks.ElderState;
+import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.MemberFactory;
+import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
+import com.gemstone.gemfire.distributed.internal.membership.NetView;
+import com.gemstone.gemfire.i18n.StringId;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.NanoTimer;
+import com.gemstone.gemfire.internal.OSProcess;
+import com.gemstone.gemfire.internal.SetUtils;
+import com.gemstone.gemfire.internal.SocketCreator;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.admin.remote.AdminConsoleDisconnectMessage;
+import com.gemstone.gemfire.internal.admin.remote.RemoteGfManagerAgent;
+import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
+import com.gemstone.gemfire.internal.cache.InitialImageOperation;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
+import com.gemstone.gemfire.internal.logging.log4j.AlertAppender;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.sequencelog.MembershipLogger;
+import com.gemstone.gemfire.internal.tcp.Connection;
+import com.gemstone.gemfire.internal.tcp.ConnectionTable;
+import com.gemstone.gemfire.internal.tcp.ReenteredConnectException;
+import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
+
+/**
+ * The DistributionManager
uses a {@link
+ * MembershipManager} to distribute {@link DistributionMessage messages}
+ * queued in {@link MQueue}s.
+ *
+ *
+ *
+ * Code that wishes to send a {@link DistributionMessage} must get
+ * the DistributionManager
and invoke {@link
+ * #putOutgoing}.
+ *
+ *
+ *
+ * DistributionManager
is not intended to be
+ * serialized. It is Externalizable
only to prevent it
+ * from being copy shared. See {@link #writeExternal}.
+ *
+ *
+ *
+ * Prior to GemFire 4.0, DistributionManager
was an
+ * abstract class with two concrete subclasses,
+ * LocalDistributionManager
and
+ * ConsoleDistributionManager
. We decided that
+ * ConsoleDistributionManager
(which was used for the GUI
+ * console and admin APIs) did not offer enough interesting
+ * functionality to warrant a separate class. More importantly, it
+ * prevented the Cache and admin APIs from being used in the same VM.
+ * So, we refactored the code of those two subclasses into
+ * DistributionManager
.
+ *
+ * @since 2.0
+ *
+ * @see DistributionMessage#process
+ * @see IgnoredByManager
+ */
+public class DistributionManager
+ implements Externalizable, DM {
+
+ private static final Logger logger = LogService.getLogger();
+
+ private static final boolean SYNC_EVENTS =
+ Boolean.getBoolean("DistributionManager.syncEvents");
+
+ /**
+ * WARNING: setting this to true may break dunit tests.
+ *
see com.gemstone.gemfire.cache30.ClearMultiVmCallBkDUnitTest
+ */
+ public static final boolean INLINE_PROCESS =
+ !Boolean.getBoolean("DistributionManager.enqueueOrderedMessages");
+
+ /** Flag indicating whether to use single Serial-Executor thread or
+ * Multiple Serial-executor thread,
+ */
+ public static final boolean MULTI_SERIAL_EXECUTORS =
+ !Boolean.getBoolean("DistributionManager.singleSerialExecutor");
+
+ /** The name of the distribution manager (identifies it in GemFire) */
+ public static final String NAME = "GemFire";
+
+ /** The number of milliseconds to wait for distribution-related
+ * things to happen */
+ public static final long TIMEOUT =
+ Long.getLong("DistributionManager.TIMEOUT", -1).longValue();
+
+ public static final int PUSHER_THREADS =
+ Integer.getInteger("DistributionManager.PUSHER_THREADS", 50).intValue();
+
+ public static final int PUSHER_QUEUE_SIZE =
+ Integer.getInteger("DistributionManager.PUSHER_QUEUE_SIZE", 4096).intValue();
+
+
+ public static final int MAX_WAITING_THREADS =
+ Integer.getInteger("DistributionManager.MAX_WAITING_THREADS", Integer.MAX_VALUE).intValue();
+
+ public static final int MAX_PR_META_DATA_CLEANUP_THREADS =
+ Integer.getInteger("DistributionManager.MAX_PR_META_DATA_CLEANUP_THREADS", 1).intValue();
+
+ public static final int MAX_THREADS = Integer.getInteger("DistributionManager.MAX_THREADS", 100).intValue();
+ public static final int MAX_PR_THREADS = Integer.getInteger("DistributionManager.MAX_PR_THREADS", Math.max(Runtime.getRuntime().availableProcessors()*4, 16)).intValue();
+ public static final int MAX_FE_THREADS = Integer.getInteger("DistributionManager.MAX_FE_THREADS", Math.max(Runtime.getRuntime().availableProcessors()*4, 16)).intValue();
+ // Integer.getInteger("DistributionManager.MAX_THREADS", max(Runtime.getRuntime().availableProcessors()*2, 2)).intValue();
+
+ public static final int INCOMING_QUEUE_LIMIT =
+ Integer.getInteger("DistributionManager.INCOMING_QUEUE_LIMIT", 80000).intValue();
+ public static final int INCOMING_QUEUE_THROTTLE =
+ Integer.getInteger("DistributionManager.INCOMING_QUEUE_THROTTLE", (int)(INCOMING_QUEUE_LIMIT * 0.75)).intValue();
+
+ /** Throttling based on the Queue byte size */
+ public static final double THROTTLE_PERCENT =
+ (double) (Integer.getInteger("DistributionManager.SERIAL_QUEUE_THROTTLE_PERCENT", 75).intValue())/100;
+ public static final int SERIAL_QUEUE_BYTE_LIMIT =
+ Integer.getInteger("DistributionManager.SERIAL_QUEUE_BYTE_LIMIT", (40 * (1024 * 1024))).intValue();
+ public static final int SERIAL_QUEUE_THROTTLE =
+ Integer.getInteger("DistributionManager.SERIAL_QUEUE_THROTTLE", (int)(SERIAL_QUEUE_BYTE_LIMIT * THROTTLE_PERCENT)).intValue();
+ public static final int TOTAL_SERIAL_QUEUE_BYTE_LIMIT =
+ Integer.getInteger("DistributionManager.TOTAL_SERIAL_QUEUE_BYTE_LIMIT", (80 * (1024 * 1024))).intValue();
+ public static final int TOTAL_SERIAL_QUEUE_THROTTLE =
+ Integer.getInteger("DistributionManager.TOTAL_SERIAL_QUEUE_THROTTLE", (int)(SERIAL_QUEUE_BYTE_LIMIT * THROTTLE_PERCENT)).intValue();
+
+ /** Throttling based on the Queue item size */
+ public static final int SERIAL_QUEUE_SIZE_LIMIT =
+ Integer.getInteger("DistributionManager.SERIAL_QUEUE_SIZE_LIMIT", 20000).intValue();
+ public static final int SERIAL_QUEUE_SIZE_THROTTLE =
+ Integer.getInteger("DistributionManager.SERIAL_QUEUE_SIZE_THROTTLE", (int)(SERIAL_QUEUE_SIZE_LIMIT * THROTTLE_PERCENT)).intValue();
+
+ /** Max number of serial Queue executors, in case of multi-serial-queue executor */
+ public static final int MAX_SERIAL_QUEUE_THREAD =
+ Integer.getInteger("DistributionManager.MAX_SERIAL_QUEUE_THREAD", 20).intValue();
+
+ /**
+ * Whether or not to include link local addresses in the list of addresses we use
+ * to determine if two members are no the same host.
+ *
+ * Added for normura issue 7033 - they have duplicate link local addresses on different boxes
+ */
+ public static volatile boolean INCLUDE_LINK_LOCAL_ADDRESSES =
+ Boolean.getBoolean("gemfire.IncludeLinkLocalAddresses");
+
+ /** The DM type for regular distribution managers */
+ public static final int NORMAL_DM_TYPE = 10;
+
+ /** The DM type for locator distribution managers
+ * @since 7.0
+ */
+ public static final int LOCATOR_DM_TYPE = 11;
+
+ /** The DM type for Console (admin-only) distribution managers */
+ public static final int ADMIN_ONLY_DM_TYPE = 12;
+
+ public static final int LONER_DM_TYPE = 13;
+
+ /**
+ * an NIO priority type
+ * @see com.gemstone.gemfire.distributed.internal.PooledDistributionMessage
+ * @see #SERIAL_EXECUTOR
+ * @see #HIGH_PRIORITY_EXECUTOR
+ * @see #WAITING_POOL_EXECUTOR
+ */
+ public static final int STANDARD_EXECUTOR = 73;
+
+ /**
+ * an NIO priority type
+ *
+ * @see com.gemstone.gemfire.distributed.internal.SerialDistributionMessage
+ * @see #STANDARD_EXECUTOR
+ */
+ public static final int SERIAL_EXECUTOR = 74;
+
+ /**
+ * an NIO priority type
+
+ * @see com.gemstone.gemfire.distributed.internal.HighPriorityDistributionMessage
+ * @see #STANDARD_EXECUTOR
+ */
+ public static final int HIGH_PRIORITY_EXECUTOR = 75;
+
+ // 76 not in use
+
+ /**
+ * an NIO priority type
+ *
+ * @see com.gemstone.gemfire.internal.cache.InitialImageOperation
+ * @see #STANDARD_EXECUTOR
+ */
+ public static final int WAITING_POOL_EXECUTOR = 77;
+
+ /**
+ * an NIO priority type
+ *
+ * @see com.gemstone.gemfire.internal.cache.InitialImageOperation
+ * @see #STANDARD_EXECUTOR
+ */
+ public static final int PARTITIONED_REGION_EXECUTOR = 78;
+
+
+ /**
+ * Executor for view related messages
+ *
+ * @see com.gemstone.gemfire.distributed.internal.membership.gms.messages.ViewAckMessage
+ * @see #STANDARD_EXECUTOR
+ */
+ public static final int VIEW_EXECUTOR = 79;
+
+
+ public static final int REGION_FUNCTION_EXECUTION_EXECUTOR = 80;
+
+ /** The number of open distribution managers in this VM */
+ private static int openDMs = 0;
+
+// /** The stack trace of the last time a console DM was opened */
+// private static Exception openStackTrace;
+
+ /** Is this VM dedicated to administration (like a GUI console or a
+ * JMX agent)? If so, then it creates {@link #ADMIN_ONLY_DM_TYPE}
+ * type distribution managers.
+ *
+ * @since 4.0
+ */
+ public static volatile boolean isDedicatedAdminVM = false;
+
+ /**
+ * Is this admin agent used for a command line console.
+ * This flag controls whether connect will throw
+ * an exception or just wait for a DS if one is not
+ * available. If true, we will throw an exception.
+ *
+ */
+ public static volatile boolean isCommandLineAdminVM = false;
+
+
+
+ ///////////////////// Instance Fields //////////////////////
+
+ /** The id of this distribution manager */
+ final protected InternalDistributedMember myid;
+
+ /** The distribution manager type of this dm; set in its constructor. */
+ private final int dmType;
+
+ /** The MembershipListener
s that are registered on this
+ * manager. */
+ private final ConcurrentMap membershipListeners;
+
+ /** A lock to hold while adding and removing membership listeners */
+ protected final Object membershipListenersLock =
+ new MembershipListenersLock();
+ /** The MembershipListener
s that are registered on this
+ * manager for ALL members.
+ * @since 5.7
+ */
+ protected volatile Set allMembershipListeners = Collections.EMPTY_SET;
+ /**
+ * A lock to hold while adding and removing all membership listeners.
+ * @since 5.7
+ */
+ protected final Object allMembershipListenersLock =
+ new MembershipListenersLock();
+ /** A queue of MemberEvent instances */
+ protected final BlockingQueue membershipEventQueue =
+ new LinkedBlockingQueue();
+ /** Used to invoke registered membership listeners in the background. */
+ private Thread memberEventThread;
+
+
+ /** A brief description of this DistributionManager */
+ protected final String description;
+
+ /** Statistics about distribution */
+ protected /*final*/ DistributionStats stats;
+
+ /** Did an exception occur in one of the DM threads? */
+ protected boolean exceptionInThreads;
+
+ static ThreadLocal isStartupThread = new ThreadLocal();
+
+ protected volatile boolean shutdownMsgSent = false;
+
+ /** Set to true when this manager is being shutdown */
+ protected volatile boolean closeInProgress = false;
+
+ private volatile boolean receivedStartupResponse = false;
+
+ private volatile String rejectionMessage = null;
+
+ protected MembershipManager membershipManager;
+
+ /** The channel through which distributed communication occurs. */
+ protected DistributionChannel channel;
+
+ /**
+ * The (non-admin-only) members of the distributed system. This is a
+ * map of memberid->memberid for fast access to canonical ID references.
+ * All accesses to this
+ * field must be synchronized on {@link #membersLock}.
+ */
+ private Map members = Collections.emptyMap();
+ /**
+ * All (admin and non-admin) members of the distributed system. All accesses
+ * to this field must be synchronized on {@link #membersLock}.
+ */
+ private Set membersAndAdmin = Collections.emptySet();
+ /**
+ * Map of all locator members of the distributed system. The value is a
+ * collection of locator strings that are hosted in that member. All accesses
+ * to this field must be synchronized on {@link #membersLock}.
+ */
+ private Map> hostedLocatorsAll = Collections.emptyMap();
+
+ /**
+ * Map of all locator members of the distributed system which have the shared configuration. The value is a
+ * collection of locator strings that are hosted in that member. All accesses
+ * to this field must be synchronized on {@link #membersLock}.
+ */
+ private Map> hostedLocatorsWithSharedConfiguration = Collections.emptyMap();
+
+ /**
+ * Since 6.6.2 and hereafter we will save the versions here. But pre-6.6.2's
+ * StartupResponseMessage does not contain version. We will assign a default
+ * version for them.
+ */
+ public static final String DEFAULT_VERSION_PRE_6_6_2 = "6.6.0.0";
+ /**
+ * The lock held while accessing the field references to the following:
+ * 1) {@link #members}
+ * 2) {@link #membersAndAdmin}
+ * 3) {@link #hostedLocatorsAll}
+ * 4) {@link #hostedLocatorsWithSharedConfiguration}
+ */
+ private final Object membersLock = new MembersLock();
+
+ /**
+ * The lock held while writing {@link #adminConsoles}.
+ */
+ private final Object adminConsolesLock = new Object();
+ /**
+ * The ids of all known admin consoles
+ * Uses Copy on Write. Writers must sync on adminConsolesLock.
+ * Readers don't need to sync.
+ */
+ private volatile Set adminConsoles = Collections.emptySet();
+
+ /** The pusher thread */
+ //private Thread pusher;
+
+ /** The group of distribution manager threads */
+ protected LoggingThreadGroup threadGroup;
+
+ /** Message processing thread pool */
+ private ThreadPoolExecutor threadPool;
+
+ /** High Priority processing thread pool, used for initializing messages
+ * such as UpdateAttributes and CreateRegion messages
+ */
+ private ThreadPoolExecutor highPriorityPool;
+
+ /** Waiting Pool, used for messages that may have to wait on something.
+ * Use this separate pool with an unbounded queue so that waiting
+ * runnables don't get in the way of other processing threads.
+ * Used for threads that will most likely have to wait for a region to be
+ * finished initializing before it can proceed
+ */
+ private ThreadPoolExecutor waitingPool;
+
+ private ThreadPoolExecutor prMetaDataCleanupThreadPool;
+
+ /**
+ * Thread used to decouple {@link com.gemstone.gemfire.internal.cache.partitioned.PartitionMessage}s from
+ * {@link com.gemstone.gemfire.internal.cache.DistributedCacheOperation}s
+ * @see #SERIAL_EXECUTOR
+ */
+ private ThreadPoolExecutor partitionedRegionThread;
+ private ThreadPoolExecutor partitionedRegionPool;
+ private ThreadPoolExecutor functionExecutionThread;
+ private ThreadPoolExecutor functionExecutionPool;
+
+ /** Message processing executor for serial, ordered, messages. */
+ private ThreadPoolExecutor serialThread;
+
+ /** Message processing executor for view messages
+ * @see com.gemstone.gemfire.distributed.internal.membership.gms.messages.ViewAckMessage
+ */
+ private ThreadPoolExecutor viewThread;
+
+ /** If using a throttling queue for the serialThread, we cache the queue here
+ so we can see if delivery would block */
+ private ThrottlingMemLinkedQueueWithDMStats serialQueue;
+
+ /** a map keyed on InternalDistributedMember, to direct channels to other systems */
+ //protected final Map channelMap = CFactory.createCM();
+
+ protected volatile boolean readyForMessages = false;
+
+ /**
+ * Set to true once this DM is ready to send messages.
+ * Note that it is always ready to send the startup message.
+ */
+ private volatile boolean readyToSendMsgs = false;
+ private final Object readyToSendMsgsLock = new Object();
+
+ /** Is this distribution manager closed? */
+ protected volatile boolean closed = false;
+
+ /** The distributed system to which this distribution manager is
+ * connected. */
+ private InternalDistributedSystem system;
+
+ /** The remote transport configuration for this dm */
+ private RemoteTransportConfig transport;
+
+ /** The administration agent associated with this distribution
+ * manager. */
+ private volatile RemoteGfManagerAgent agent;
+
+ private SerialQueuedExecutorPool serialQueuedExecutorPool;
+
+ private final Semaphore parallelGIIs = new Semaphore(InitialImageOperation.MAX_PARALLEL_GIIS);
+
+ /**
+ * Map of InetAddress to HashSets of InetAddress, to define equivalences
+ * between network interface cards and hosts.
+ */
+ private final HashMap> equivalentHosts = new HashMap>();
+
+ private int distributedSystemId = DistributionConfig.DEFAULT_DISTRIBUTED_SYSTEM_ID;
+
+
+ private final Map redundancyZones = Collections.synchronizedMap(new HashMap());
+
+ private boolean enforceUniqueZone = false;
+
+ private volatile boolean isSharedConfigEnabledForDS = false;
+
+ @Override
+ public boolean isSharedConfigurationServiceEnabledForDS() {
+ return isSharedConfigEnabledForDS;
+ }
+
+ /**
+ * Identifier for function execution threads and any of their children
+ */
+ public static final InheritableThreadLocal isFunctionExecutionThread = new InheritableThreadLocal() {
+ @Override
+ protected Boolean initialValue() {
+ return Boolean.FALSE;
+ }
+ };
+ ////////////////////// Static Methods //////////////////////
+
+ /**
+ * Given two DistributionManager ids, check to see if they are
+ * from the same host address.
+ * @param id1 a DistributionManager id
+ * @param id2 a DistributionManager id
+ * @return true if id1 and id2 are from the same host, false otherwise
+ */
+ public static boolean isSameHost(InternalDistributedMember id1, InternalDistributedMember id2) {
+ return (id1.getInetAddress().equals(id2.getInetAddress()));
+ }
+
+ // @todo davidw Modify JGroups so that we do not have to send out a
+ // {@link StartupMessage}
+ /**
+ * Creates a new distribution manager and discovers the other members of the
+ * distributed system. Note that it does not check to see whether or not this
+ * VM already has a distribution manager.
+ *
+ * @param system
+ * The distributed system to which this distribution manager
+ * will send messages.
+ */
+ public static DistributionManager create(InternalDistributedSystem system)
+ {
+
+ DistributionManager dm = null;
+
+ try {
+
+ int vmKind;
+
+ if (Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
+ // if this DM is starting for a locator, set it to be a locator DM
+ vmKind = LOCATOR_DM_TYPE;
+
+ } else if (isDedicatedAdminVM) {
+ vmKind = ADMIN_ONLY_DM_TYPE;
+
+ } else {
+ vmKind = NORMAL_DM_TYPE;
+ }
+
+ RemoteTransportConfig transport = new RemoteTransportConfig(system.getConfig(), vmKind);
+ transport.setIsReconnectingDS(system.isReconnectingDS());
+ transport.setOldDSMembershipInfo(system.oldDSMembershipInfo());
+
+ long start = System.currentTimeMillis();
+
+ dm = new DistributionManager(system, transport);
+ dm.assertDistributionManagerType();
+
+ {
+ InternalDistributedMember id = dm.getDistributionManagerId();
+ if (!"".equals(id.getName())) {
+ for (InternalDistributedMember m: (List)dm.getViewMembers()) {
+ if (m.equals(id)) {
+ // I'm counting on the members returned by getViewMembers being ordered such that
+ // members that joined before us will precede us AND members that join after us
+ // will succeed us.
+ // SO once we find ourself break out of this loop.
+ break;
+ }
+ if (id.getName().equals(m.getName())) {
+ if (dm.getMembershipManager().verifyMember(m, "member is using the name of " + id)) {
+ throw new IncompatibleSystemException("Member " + id + " could not join this distributed system because the existing member " + m + " used the same name. Set the \"name\" gemfire property to a unique value.");
+ }
+ }
+ }
+ }
+ dm.addNewMember(id); // add ourselves
+ dm.selectElder(); // ShutdownException could be thrown here
+ }
+
+ // Send out a StartupMessage to the other members.
+ StartupOperation op = new StartupOperation(dm, transport);
+
+ try {
+ if (!dm.sendStartupMessage(op, true)) {
+ // We'll we didn't hear back from anyone else. We assume that
+ // we're the first one.
+ if (dm.getOtherDistributionManagerIds().size() == 0) {
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_DIDNT_HEAR_BACK_FROM_ANY_OTHER_SYSTEM_I_AM_THE_FIRST_ONE));
+ } else if (transport.isMcastEnabled()) {
+ // perform a multicast ping test
+ if (!dm.testMulticast()) {
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_RECEIVED_NO_STARTUP_RESPONSES_BUT_OTHER_MEMBERS_EXIST_MULTICAST_IS_NOT_RESPONSIVE));
+ }
+ }
+ }
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ // This is ALWAYS bad; don't consult a CancelCriterion.
+ throw new InternalGemFireException(LocalizedStrings.DistributionManager_INTERRUPTED_WHILE_WAITING_FOR_FIRST_STARTUPRESPONSEMESSAGE.toLocalizedString(), ex);
+ } catch (IncompatibleSystemException ex) {
+ logger.fatal(ex.getMessage(), ex);
+ throw ex;
+ } finally {
+ dm.readyToSendMsgs();
+ }
+
+ if (logger.isInfoEnabled()) {
+ long delta = System.currentTimeMillis() - start;
+ Object[] logArgs = new Object[] {
+ dm.getDistributionManagerId(),
+ transport,
+ Integer.valueOf(dm.getOtherDistributionManagerIds().size()),
+ dm.getOtherDistributionManagerIds(),
+ (logger.isInfoEnabled(LogMarker.DM) ? " (VERBOSE, took " + delta + " ms)" : ""),
+ ((dm.getDMType() == ADMIN_ONLY_DM_TYPE) ? " (admin only)" : (dm.getDMType() == LOCATOR_DM_TYPE) ? " (locator)" : "")
+ };
+ logger.info(LogMarker.DM, LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGER_0_STARTED_ON_1_THERE_WERE_2_OTHER_DMS_3_4_5, logArgs));
+
+ MembershipLogger.logStartup(dm.getDistributionManagerId());
+ }
+ return dm;
+ }
+ catch (RuntimeException r) {
+ if (dm != null) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("cleaning up incompletely started DistributionManager due to exception", r);
+ }
+ dm.uncleanShutdown(true);
+ }
+ throw r;
+ }
+ }
+
+ void runUntilShutdown(Runnable r) {
+ try {
+ r.run();
+ }
+ catch (CancelException e) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("Caught shutdown exception", e);
+ }
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ if (isCloseInProgress()) {
+ logger.debug("Caught unusual exception during shutdown: {}", t.getMessage(), t);
+ }
+ else {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_TASK_FAILED_WITH_EXCEPTION), t);
+ }
+ } }
+
+ volatile Throwable rootCause = null;
+
+ private static class Stopper extends CancelCriterion {
+ private DistributionManager dm;
+
+ // validateDM is commented out because expiry threads hit it with
+ // an ugly failure... use only for debugging lingering DM bugs
+// private String validateDM() {
+// GemFireCache cache = GemFireCache.getInstance();
+// if (cache == null) {
+// return null; // Distributed system with no cache
+// }
+// Object obj = cache.getDistributedSystem();
+// if (obj == null) {
+// return null; // Cache is very dead
+// }
+// InternalDistributedSystem ids = (InternalDistributedSystem)obj;
+// DM current = ids.getDistributionManager();
+// if (current != dm) {
+// String response = LocalizedStrings.DistributionManager_CURRENT_CACHE_DISTRIBUTIONMANAGER_0_IS_NOT_THE_SAME_AS_1
+// .toLocalizedString(new Object[] { current, dm});
+// return response;
+// }
+// return null;
+// }
+
+ Stopper(DistributionManager dm) {
+ this.dm = dm;
+ }
+ @Override
+ public String cancelInProgress() {
+ checkFailure();
+
+ // remove call to validateDM() to fix bug 38356
+
+ if (dm.shutdownMsgSent) {
+ return LocalizedStrings.DistributionManager__0_MESSAGE_DISTRIBUTION_HAS_TERMINATED.toLocalizedString(dm.toString());
+ }
+ if (dm.rootCause != null) {
+ return dm.toString() + ": " + dm.rootCause.getMessage();
+ }
+
+ // Nope.
+ return null;
+ }
+
+ @Override
+ public RuntimeException generateCancelledException(Throwable e) {
+ String reason = cancelInProgress();
+ if (reason == null) {
+ return null;
+ }
+ Throwable rc = dm.rootCause; // volatile read
+ if (rc == null) {
+ // No root cause, specify the one given and be done with it.
+ return new DistributedSystemDisconnectedException(reason, e);
+ }
+
+ if (e == null) {
+ // Caller did not specify any root cause, so just use our own.
+ return new DistributedSystemDisconnectedException(reason, rc);
+ }
+
+ // Attempt to stick rootCause at tail end of the exception chain.
+ Throwable nt = e;
+ while (nt.getCause() != null) {
+ nt = nt.getCause();
+ }
+ if (nt == rc) {
+ // Root cause already in place; we're done
+ return new DistributedSystemDisconnectedException(reason, e);
+ }
+
+ try {
+ nt.initCause(rc);
+ return new DistributedSystemDisconnectedException(reason, e);
+ }
+ catch (IllegalStateException e2) {
+ // Bug 39496 (Jrockit related) Give up. The following
+ // error is not entirely sane but gives the correct general picture.
+ return new DistributedSystemDisconnectedException(reason, rc);
+ }
+ }
+ }
+ private final Stopper stopper = new Stopper(this);
+
+ public CancelCriterion getCancelCriterion() {
+ return stopper;
+ }
+
+ /////////////////////// Constructors ///////////////////////
+
+ /**
+ * no-arg constructor for Externalizable
+ * TODO: does this class really need to implement Externalizable? I
+ * think it only implements that interface for the old copy-sharing
+ * shared-memory stuff that's no longer in GemFire
+ */
+ public DistributionManager() {
+ this.elderLock = null;
+ this.membershipListeners = null;
+ this.myid = null;
+ this.description = null;
+ this.dmType = 0;
+ throw new IllegalAccessError("this constructor should never be invoked");
+ }
+
+ /**
+ * Creates a new DistributionManager
by initializing
+ * itself, creating the membership manager and executors
+ *
+ * @param transport
+ * The configuration for the communications transport
+ *
+ */
+ private DistributionManager(RemoteTransportConfig transport,
+ InternalDistributedSystem system) {
+
+ this.dmType = transport.getVmKind();
+ this.system = system;
+ this.elderLock = new StoppableReentrantLock(stopper);
+ this.transport = transport;
+
+ this.membershipListeners = new ConcurrentHashMap();
+ this.distributedSystemId = system.getConfig().getDistributedSystemId();
+ {
+ long statId = OSProcess.getId();
+ /* deadcoded since we don't know the channel id yet.
+ if (statId == 0 || statId == -1) {
+ statId = getChannelId();
+ }
+ */
+ this.stats = new DistributionStats(system, statId);
+ DistributionStats.enableClockStats = system.getConfig().getEnableTimeStatistics();
+ }
+
+ this.exceptionInThreads = false;
+
+ // Start the processing threads
+ final LoggingThreadGroup group =
+ LoggingThreadGroup.createThreadGroup("DistributionManager Threads", logger);
+ this.threadGroup = group;
+
+ boolean finishedConstructor = false;
+ try {
+
+ if (MULTI_SERIAL_EXECUTORS) {
+ if (logger.isInfoEnabled(LogMarker.DM)) {
+ logger.info(LogMarker.DM,
+ "Serial Queue info :" +
+ " THROTTLE_PERCENT: " + THROTTLE_PERCENT +
+ " SERIAL_QUEUE_BYTE_LIMIT :" + SERIAL_QUEUE_BYTE_LIMIT +
+ " SERIAL_QUEUE_THROTTLE :" + SERIAL_QUEUE_THROTTLE +
+ " TOTAL_SERIAL_QUEUE_BYTE_LIMIT :" + TOTAL_SERIAL_QUEUE_BYTE_LIMIT +
+ " TOTAL_SERIAL_QUEUE_THROTTLE :" + TOTAL_SERIAL_QUEUE_THROTTLE +
+ " SERIAL_QUEUE_SIZE_LIMIT :" + SERIAL_QUEUE_SIZE_LIMIT +
+ " SERIAL_QUEUE_SIZE_THROTTLE :" + SERIAL_QUEUE_SIZE_THROTTLE
+ );
+ }
+ // when TCP/IP is disabled we can't throttle the serial queue or we run the risk of
+ // distributed deadlock when we block the UDP reader thread
+ boolean throttlingDisabled = system.getConfig().getDisableTcp();
+ this.serialQueuedExecutorPool = new SerialQueuedExecutorPool(this.threadGroup, this.stats, throttlingDisabled);
+ }
+
+ {
+ BlockingQueue poolQueue;
+ if (SERIAL_QUEUE_BYTE_LIMIT == 0) {
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getSerialQueueHelper());
+ } else {
+ this.serialQueue = new ThrottlingMemLinkedQueueWithDMStats(TOTAL_SERIAL_QUEUE_BYTE_LIMIT,
+ TOTAL_SERIAL_QUEUE_THROTTLE, SERIAL_QUEUE_SIZE_LIMIT, SERIAL_QUEUE_SIZE_THROTTLE,
+ this.stats.getSerialQueueHelper());
+ poolQueue = this.serialQueue;
+ }
+ ThreadFactory tf = new ThreadFactory() {
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incSerialThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ DistributionManager.this.stats.incNumSerialThreads(1);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ // command.run();
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ DistributionManager.this.stats.incNumSerialThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r, LocalizedStrings.DistributionManager_SERIAL_MESSAGE_PROCESSOR.toLocalizedString());
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ SerialQueuedExecutorWithDMStats executor = new SerialQueuedExecutorWithDMStats(poolQueue,
+ this.stats.getSerialProcessorHelper(), tf);
+ this.serialThread = executor;
+ }
+ {
+ BlockingQueue q = new LinkedBlockingQueue();
+ ThreadFactory tf = new ThreadFactory() {
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incViewThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ DistributionManager.this.stats.incNumViewThreads(1);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ DistributionManager.this.stats.incNumViewThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r, LocalizedStrings.DistributionManager_VIEW_MESSAGE_PROCESSOR.toLocalizedString());
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ this.viewThread = new SerialQueuedExecutorWithDMStats(q,
+ this.stats.getViewProcessorHelper(), tf);
+ }
+
+ {
+ BlockingQueue poolQueue;
+ if (INCOMING_QUEUE_LIMIT == 0) {
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getOverflowQueueHelper());
+ } else {
+ poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT, this.stats.getOverflowQueueHelper());
+ }
+ ThreadFactory tf = new ThreadFactory() {
+ private int next = 0;
+
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incProcessingThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ DistributionManager.this.stats.incNumProcessingThreads(1);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ DistributionManager.this.stats.incNumProcessingThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r,
+ LocalizedStrings.DistributionManager_POOLED_MESSAGE_PROCESSOR.toLocalizedString() + (next++));
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ ThreadPoolExecutor pool =
+ new PooledExecutorWithDMStats(poolQueue, MAX_THREADS, this.stats.getNormalPoolHelper(), tf);
+ this.threadPool = pool;
+ }
+
+
+ {
+ BlockingQueue poolQueue;
+ if (INCOMING_QUEUE_LIMIT == 0) {
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getHighPriorityQueueHelper());
+ } else {
+ poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT, this.stats.getHighPriorityQueueHelper());
+ }
+ ThreadFactory tf = new ThreadFactory() {
+ private int next = 0;
+
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incHighPriorityThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ DistributionManager.this.stats.incHighPriorityThreads(1);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ DistributionManager.this.stats.incHighPriorityThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r,
+ LocalizedStrings.DistributionManager_POOLED_HIGH_PRIORITY_MESSAGE_PROCESSOR.toLocalizedString() + (next++));
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ this.highPriorityPool = new PooledExecutorWithDMStats(poolQueue, MAX_THREADS, this.stats.getHighPriorityPoolHelper(), tf);
+ }
+
+
+ {
+ ThreadFactory tf = new ThreadFactory() {
+ private int next = 0;
+
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incWaitingThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ DistributionManager.this.stats.incWaitingThreads(1);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ DistributionManager.this.stats.incWaitingThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r,
+ LocalizedStrings.DistributionManager_POOLED_WAITING_MESSAGE_PROCESSOR.toLocalizedString() + (next++));
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ BlockingQueue poolQueue;
+ if (MAX_WAITING_THREADS == Integer.MAX_VALUE) {
+ // no need for a queue since we have infinite threads
+ poolQueue = new SynchronousQueue();
+ } else {
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getWaitingQueueHelper());
+ }
+ this.waitingPool = new PooledExecutorWithDMStats(poolQueue,
+ MAX_WAITING_THREADS,
+ this.stats.getWaitingPoolHelper(),
+ tf);
+ }
+
+ {
+ ThreadFactory tf = new ThreadFactory() {
+ private int next = 0;
+
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incWaitingThreadStarts();//will it be ok?
+ final Runnable r = new Runnable() {
+ public void run() {
+ DistributionManager.this.stats.incWaitingThreads(1);//will it be ok
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ DistributionManager.this.stats.incWaitingThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r,
+ LocalizedStrings.DistributionManager_PR_META_DATA_CLEANUP_MESSAGE_PROCESSOR.toLocalizedString() + (next++));
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ BlockingQueue poolQueue;
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getWaitingQueueHelper());
+ this.prMetaDataCleanupThreadPool = new PooledExecutorWithDMStats(poolQueue,
+ MAX_PR_META_DATA_CLEANUP_THREADS,
+ this.stats.getWaitingPoolHelper(),
+ tf);
+ }
+
+ {
+ BlockingQueue poolQueue;
+ if (INCOMING_QUEUE_LIMIT == 0) {
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getPartitionedRegionQueueHelper());
+ } else {
+ poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT, this.stats.getPartitionedRegionQueueHelper());
+ }
+ ThreadFactory tf = new ThreadFactory() {
+ private int next = 0;
+
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incPartitionedRegionThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ stats.incPartitionedRegionThreads(1);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ stats.incPartitionedRegionThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r,
+ "PartitionedRegion Message Processor" + (next++));
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ if (MAX_PR_THREADS > 1) {
+ this.partitionedRegionPool = new PooledExecutorWithDMStats(poolQueue,
+ MAX_PR_THREADS, this.stats.getPartitionedRegionPoolHelper(), tf);
+ } else {
+ SerialQueuedExecutorWithDMStats executor = new SerialQueuedExecutorWithDMStats(poolQueue,
+ this.stats.getPartitionedRegionPoolHelper(), tf);
+ this.partitionedRegionThread = executor;
+ }
+
+ }
+
+ {
+ BlockingQueue poolQueue;
+ if (INCOMING_QUEUE_LIMIT == 0) {
+ poolQueue = new OverflowQueueWithDMStats(this.stats.getFunctionExecutionQueueHelper());
+ } else {
+ poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT, this.stats.getFunctionExecutionQueueHelper());
+ }
+ ThreadFactory tf = new ThreadFactory() {
+ private int next = 0;
+
+ public Thread newThread(final Runnable command) {
+ DistributionManager.this.stats.incFunctionExecutionThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ stats.incFunctionExecutionThreads(1);
+ isFunctionExecutionThread.set(Boolean.TRUE);
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ runUntilShutdown(command);
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ stats.incFunctionExecutionThreads(-1);
+ }
+ }
+ };
+ Thread thread = new Thread(group, r,
+ "Function Execution Processor" + (next++));
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+
+ if(MAX_FE_THREADS > 1){
+ this.functionExecutionPool = new FunctionExecutionPooledExecutor(poolQueue,
+ MAX_FE_THREADS, this.stats.getFunctionExecutionPoolHelper(), tf,true /*for fn exec*/);
+ } else {
+ SerialQueuedExecutorWithDMStats executor = new SerialQueuedExecutorWithDMStats(poolQueue,
+ this.stats.getFunctionExecutionPoolHelper(), tf);
+ this.functionExecutionThread = executor;
+ }
+
+ }
+
+ if (!SYNC_EVENTS) {
+ this.memberEventThread = new Thread(group, new MemberEventInvoker(),
+ "DM-MemberEventInvoker");
+ this.memberEventThread.setDaemon(true);
+ }
+
+ StringBuffer sb = new StringBuffer(" (took ");
+
+ long start = System.currentTimeMillis();
+
+ // Create direct channel first
+// DirectChannel dc = new DirectChannel(new MyListener(this), system.getConfig(), logger, null);
+// setDirectChannelPort(dc.getPort()); // store in a thread local
+
+ // connect to JGroups
+ start = System.currentTimeMillis();
+
+ MyListener l = new MyListener(this);
+ membershipManager = MemberFactory.newMembershipManager(l, system.getConfig(), transport, stats);
+
+ sb.append(System.currentTimeMillis() - start);
+
+ this.myid = membershipManager.getLocalMember();
+
+// dc.patchUpAddress(this.myid);
+// id.setDirectChannelPort(dc.getPort());
+
+ // create the distribution channel
+ this.channel = new DistributionChannel(membershipManager);
+
+ membershipManager.postConnect();
+
+ //Assert.assertTrue(this.getChannelMap().size() >= 1);
+ // System.out.println("Channel Map:");
+ // for (Iterator iter = this.getChannelMap().entrySet().iterator();
+ // iter.hasNext(); ) {
+ // Map.Entry entry = (Map.Entry) iter.next();
+ // Object key = entry.getKey();
+ // System.out.println(" " + key + " a " +
+ // key.getClass().getName() + " -> " +
+ // entry.getValue());
+ // }
+
+ sb.append(" ms)");
+
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_STARTING_DISTRIBUTIONMANAGER_0_1,
+ new Object[] { this.myid, (logger.isInfoEnabled(LogMarker.DM) ? sb.toString() : "")}));
+
+ this.description = NAME + " on " + this.myid + " started at "
+ + (new Date(System.currentTimeMillis())).toString();
+
+ finishedConstructor = true;
+ } finally {
+ if (!finishedConstructor) {
+ askThreadsToStop(); // fix for bug 42039
+ }
+ }
+ }
+
+ /**
+ * Creates a new distribution manager
+ *
+ * @param system
+ * The distributed system to which this distribution manager
+ * will send messages.
+ */
+ private DistributionManager(
+ InternalDistributedSystem system,
+ RemoteTransportConfig transport)
+ {
+ this(transport, system);
+
+ boolean finishedConstructor = false;
+ try {
+
+ isStartupThread.set(Boolean.TRUE);
+
+ startThreads();
+
+ // Since we need a StartupResponseMessage to make sure licenses
+ // are compatible the following has been deadcoded.
+// // For the time being, invoke processStartupResponse()
+// String rejectionMessage = null;
+// if (GemFireVersion.getGemFireVersion().
+// equals(state.getGemFireVersion())) {
+// rejectionMessage = "Rejected new system node " +
+// this.getDistributionManagerId() + " with version \"" +
+// GemFireVersion.getGemFireVersion() +
+// "\" because the distributed system's version is \"" +
+// state.getGemFireVersion() + "\".";
+// }
+// this.processStartupResponse(state.getCacheTime(),
+// rejectionMessage);
+
+ // Allow events to start being processed.
+ membershipManager.startEventProcessing();
+ for (;;) {
+ this.getCancelCriterion().checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ membershipManager.waitForEventProcessing();
+ break;
+ }
+ catch (InterruptedException e) {
+ interrupted = true;
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+
+ synchronized (DistributionManager.class) {
+ openDMs++;
+ }
+ finishedConstructor = true;
+ } finally {
+ if (!finishedConstructor) {
+ askThreadsToStop(); // fix for bug 42039
+ }
+ }
+ }
+
+ //////////////////// Instance Methods /////////////////////
+
+ /**
+ * Returns true if the two members are on the same equivalent host based
+ * on overlapping IP addresses collected for all NICs during exchange of
+ * startup messages.
+ *
+ * @param member1 First member
+ * @param member2 Second member
+ */
+ public boolean areOnEquivalentHost(InternalDistributedMember member1,
+ InternalDistributedMember member2) {
+ Set equivalents1 = getEquivalents(member1.getInetAddress());
+ return equivalents1.contains(member2.getInetAddress());
+ }
+
+ /**
+ * Set the host equivalencies for a given host. This overrides any
+ * previous information in the tables.
+ * @param equivs list of InetAddress's that all point at same host
+ */
+ public void setEquivalentHosts(Set equivs) {
+ Iterator it = equivs.iterator();
+ synchronized (equivalentHosts) {
+ while (it.hasNext()) {
+ equivalentHosts.put(it.next(), Collections.unmodifiableSet(equivs));
+ }
+ }
+ }
+
+ public HashMap> getEquivalentHostsSnapshot() {
+ synchronized (this.equivalentHosts) {
+ return new HashMap>(this.equivalentHosts);
+ }
+ }
+
+ /**
+ * Return all of the InetAddress's that are equivalent to the given one (same
+ * host)
+ * @param in host to match up
+ * @return all the addresses thus equivalent
+ */
+ public Set getEquivalents(InetAddress in) {
+ Set result;
+ synchronized (equivalentHosts) {
+ result = equivalentHosts.get(in);
+ }
+ //DS 11/25/08 - It appears that when using VPN, the distributed member
+ //id is the vpn address, but that doesn't show up in the equivalents.
+ if(result == null) {
+ result = Collections.singleton(in);
+ }
+ return result;
+ }
+
+ public void setRedundancyZone(InternalDistributedMember member, String redundancyZone) {
+ if(redundancyZone != null && !redundancyZone.equals("")) {
+ this.redundancyZones.put(member, redundancyZone);
+ }
+ if (member != getDistributionManagerId()) {
+ String relationship = areInSameZone(getDistributionManagerId(), member) ? ""
+ : "not ";
+ Object[] logArgs = new Object[] { member, relationship };
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGER_MEMBER_0_IS_1_EQUIVALENT, logArgs));
+ }
+ }
+
+ /**
+ * Set the flag indicating that we should enforce unique zones.
+ * If we are already enforcing unique zones, keep it that way.
+ */
+ public void setEnforceUniqueZone(boolean enforceUniqueZone) {
+ this.enforceUniqueZone |= enforceUniqueZone;
+ }
+
+ public boolean enforceUniqueZone() {
+ return enforceUniqueZone;
+ }
+
+ public String getRedundancyZone(InternalDistributedMember member) {
+ return redundancyZones.get(member);
+ }
+
+ /**
+ * Asserts that distributionManagerType is LOCAL, GEMFIRE, or
+ * ADMIN_ONLY. Also asserts that the distributionManagerId
+ * (jgroups DistributedMember) has a VmKind that matches.
+ */
+ private void assertDistributionManagerType() {
+ // Assert that dmType is one of the three DM types...
+ int theDmType = getDMType();
+ switch (theDmType) {
+ case NORMAL_DM_TYPE:
+ case LONER_DM_TYPE:
+ case ADMIN_ONLY_DM_TYPE:
+ case LOCATOR_DM_TYPE:
+ break;
+ default:
+ Assert.assertTrue(false, "unknown distribution manager type");
+ }
+
+ // Assert InternalDistributedMember VmKind matches this DistributionManagerType...
+ final InternalDistributedMember theId = getDistributionManagerId();
+ final int vmKind = theId.getVmKind();
+ if (theDmType != vmKind) {
+ Assert.assertTrue(false,
+ "InternalDistributedMember has a vmKind of " + vmKind +
+ " instead of " + theDmType);
+ }
+ }
+
+ public int getDMType() {
+ return this.dmType;
+ }
+
+ public List getViewMembers() {
+ NetView result = null;
+ DistributionChannel ch = this.channel;
+ if (ch != null) {
+ MembershipManager mgr = ch.getMembershipManager();
+ if (mgr != null) {
+ result = mgr.getView();
+ }
+ }
+ if (result == null) {
+ result = new NetView();
+ }
+ return result.getMembers();
+ }
+ /* implementation of DM.getOldestMember */
+ public DistributedMember getOldestMember(Collection c) throws NoSuchElementException {
+ List view = getViewMembers();
+ for (int i=0; i it = v.getMembers().iterator();
+ while (it.hasNext()) {
+ addNewMember(it.next());
+ }
+
+ // Figure out who the elder is...
+ selectElder(); // ShutdownException could be thrown here
+ } catch (Exception ex) {
+ throw new InternalGemFireException(LocalizedStrings.DistributionManager_COULD_NOT_PROCESS_INITIAL_VIEW.toLocalizedString(), ex);
+ }
+ try {
+ getWaitingThreadPool().execute(new Runnable() {
+ public void run() {
+ // call in background since it might need to send a reply
+ // and we are not ready to send messages until startup is finished
+ isStartupThread.set(Boolean.TRUE);
+ readyForMessages();
+ }
+ });
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributionManager_UNCAUGHT_EXCEPTION_CALLING_READYFORMESSAGES), t);
+ }
+ }
+
+ protected void readyForMessages() {
+ synchronized (this) {
+ this.readyForMessages = true;
+ this.notifyAll();
+ }
+ membershipManager.startEventProcessing();
+ }
+
+ protected void waitUntilReadyForMessages() {
+ if (readyForMessages)
+ return;
+// membershipManager.waitForEventProcessing();
+ synchronized (this) {
+ for (;;) {
+ if (readyForMessages)
+ break;
+ stopper.checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ this.wait();
+ }
+ catch (InterruptedException e) {
+ interrupted = true;
+ stopper.checkCancelInProgress(e);
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } // for
+ } // synchronized
+ }
+
+ /**
+ * Call when the DM is ready to send messages.
+ */
+ private void readyToSendMsgs() {
+ synchronized (this.readyToSendMsgsLock) {
+ this.readyToSendMsgs = true;
+ this.readyToSendMsgsLock.notifyAll();
+ }
+ }
+ /**
+ * Return when DM is ready to send out messages.
+ * @param msg the messsage that is currently being sent
+ */
+ protected void waitUntilReadyToSendMsgs(DistributionMessage msg) {
+ if (this.readyToSendMsgs) {
+ return;
+ }
+ // another process may have been started in the same view, so we need
+ // to be responsive to startup messages and be able to send responses
+ if (msg instanceof StartupMessage || msg instanceof StartupResponseMessage
+ || msg instanceof AdminMessageType) {
+ return;
+ }
+ if (isStartupThread.get() != null) {
+ // let the startup thread send messages
+ // the only case I know of that does this is if we happen to log a
+ // message during startup and an alert listener has registered.
+ return;
+ }
+// membershipManager.waitForEventProcessing();
+ synchronized (this.readyToSendMsgsLock) {
+ for (;;) {
+ if (this.readyToSendMsgs)
+ break;
+ stopper.checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ this.readyToSendMsgsLock.wait();
+ }
+ catch (InterruptedException e) {
+ interrupted = true;
+ stopper.checkCancelInProgress(e);
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } // for
+ } // synchronized
+ }
+
+ // DM method
+ @Override
+ public void forceUDPMessagingForCurrentThread() {
+ membershipManager.forceUDPMessagingForCurrentThread();
+ }
+
+ // DM method
+ @Override
+ public void releaseUDPMessagingForCurrentThread() {
+ membershipManager.releaseUDPMessagingForCurrentThread();
+ }
+
+ /**
+ * Did an exception occur in one of the threads launched by this
+ * distribution manager?
+ */
+ public boolean exceptionInThreads() {
+ return this.exceptionInThreads || this.threadGroup.getUncaughtExceptionsCount() > 0;
+ }
+
+ /**
+ * Clears the boolean that determines whether or not an exception
+ * occurred in one of the worker threads. This method should be
+ * used for testing purposes only!
+ */
+ void clearExceptionInThreads() {
+ this.exceptionInThreads = false;
+ this.threadGroup.clearUncaughtExceptionsCount();
+ }
+
+ /**
+ * Returns the current "cache time" in milliseconds since the epoch.
+ * The "cache time" takes into account skew among the local clocks
+ * on the various machines involved in the cache.
+ */
+ public long cacheTimeMillis() {
+ return this.system.getClock().cacheTimeMillis();
+ }
+
+
+
+ /**
+ * Returns the id of this distribution manager.
+ */
+ public InternalDistributedMember getDistributionManagerId() {
+ return this.myid;
+ }
+
+ /**
+ * Returns an unmodifiable set containing the identities of all of
+ * the known (non-admin-only) distribution managers.
+ */
+ public Set getDistributionManagerIds() {
+ // access to members synchronized under membersLock in order to
+ // ensure serialization
+ synchronized (this.membersLock) {
+ return this.members.keySet();
+ }
+ }
+
+ /**
+ * Adds the entry in {@link #hostedLocatorsAll} for a member with one or more
+ * hosted locators. The value is a collection of host[port] strings. If a
+ * bind-address was used for a locator then the form is bind-addr[port].
+ *
+ * @since 6.6.3
+ */
+ public void addHostedLocators(InternalDistributedMember member, Collection locators, boolean isSharedConfigurationEnabled) {
+ synchronized (this.membersLock) {
+ if (locators == null || locators.isEmpty()) {
+ throw new IllegalArgumentException("Cannot use empty collection of locators");
+ }
+ if (this.hostedLocatorsAll.isEmpty()) {
+ this.hostedLocatorsAll = new HashMap>();
+ }
+
+ if (!this.isSharedConfigEnabledForDS) {
+ this.isSharedConfigEnabledForDS = isSharedConfigurationEnabled;
+ }
+
+ Map> tmp =
+ new HashMap>(this.hostedLocatorsAll);
+ tmp.remove(member);
+ tmp.put(member, locators);
+ tmp = Collections.unmodifiableMap(tmp);
+ this.hostedLocatorsAll = tmp;
+
+ if (isSharedConfigurationEnabled) {
+ if (locators == null || locators.isEmpty()) {
+ throw new IllegalArgumentException("Cannot use empty collection of locators");
+ }
+ if (this.hostedLocatorsWithSharedConfiguration.isEmpty()) {
+ this.hostedLocatorsWithSharedConfiguration = new HashMap>();
+ }
+ tmp = new HashMap>(this.hostedLocatorsWithSharedConfiguration);
+ tmp.remove(member);
+ tmp.put(member, locators);
+ tmp = Collections.unmodifiableMap(tmp);
+ this.hostedLocatorsWithSharedConfiguration = tmp;
+ }
+
+ }
+ }
+
+
+ private void removeHostedLocators(InternalDistributedMember member) {
+ synchronized (this.membersLock) {
+ if (this.hostedLocatorsAll.containsKey(member)) {
+ Map> tmp =
+ new HashMap>(this.hostedLocatorsAll);
+ tmp.remove(member);
+ if (tmp.isEmpty()) {
+ tmp = Collections.emptyMap();
+ } else {
+ tmp = Collections.unmodifiableMap(tmp);
+ }
+ this.hostedLocatorsAll = tmp;
+ }
+ if (this.hostedLocatorsWithSharedConfiguration.containsKey(member)) {
+ Map> tmp =
+ new HashMap>(this.hostedLocatorsWithSharedConfiguration);
+ tmp.remove(member);
+ if (tmp.isEmpty()) {
+ tmp = Collections.emptyMap();
+ } else {
+ tmp = Collections.unmodifiableMap(tmp);
+ }
+ this.hostedLocatorsWithSharedConfiguration = tmp;
+ }
+ }
+ }
+
+
+
+ /**
+ * Gets the value in {@link #hostedLocatorsAll} for a member with one or more
+ * hosted locators. The value is a collection of host[port] strings. If a
+ * bind-address was used for a locator then the form is bind-addr[port].
+ *
+ * @since 6.6.3
+ */
+ public Collection getHostedLocators(InternalDistributedMember member) {
+ synchronized (this.membersLock) {
+ return this.hostedLocatorsAll.get(member);
+ }
+ }
+
+ /**
+ * Returns a copy of the map of all members hosting locators. The key is the
+ * member, and the value is a collection of host[port] strings. If a
+ * bind-address was used for a locator then the form is bind-addr[port].
+ *
+ * @since 6.6.3
+ */
+ public Map> getAllHostedLocators() {
+ synchronized (this.membersLock) {
+ return this.hostedLocatorsAll;
+ }
+ }
+ /**
+ * Returns a copy of the map of all members hosting locators with shared configuration. The key is the
+ * member, and the value is a collection of host[port] strings. If a
+ * bind-address was used for a locator then the form is bind-addr[port].
+ *
+ * @since 8.0
+ */
+ @Override
+ public Map> getAllHostedLocatorsWithSharedConfiguration() {
+ synchronized (this.membersLock) {
+ return this.hostedLocatorsWithSharedConfiguration;
+ }
+ }
+ /**
+ * Returns an unmodifiable set containing the identities of all of
+ * the known (including admin) distribution managers.
+ */
+ public Set getDistributionManagerIdsIncludingAdmin() {
+ // access to members synchronized under membersLock in order to
+ // ensure serialization
+ synchronized (this.membersLock) {
+ return this.membersAndAdmin;
+ }
+ }
+
+
+ /**
+ * Returns the low-level distribution channel for this distribution
+ * manager. (brought over from ConsoleDistributionManager)
+ *
+ * @since 4.0
+ */
+ public DistributionChannel getDistributionChannel() {
+ return this.channel;
+ }
+
+
+ /**
+ * Returns a private-memory list containing the identities of all
+ * the other known distribution managers not including me.
+ */
+ public Set getOtherDistributionManagerIds() {
+ // We return a modified copy of the list, so
+ // collect the old list and copy under the lock.
+ Set result = new HashSet(getDistributionManagerIds());
+
+ InternalDistributedMember me = getDistributionManagerId();
+ result.remove(me);
+
+ // It's okay for my own id to not be in the list of all ids yet.
+ return result;
+ }
+ @Override
+ public Set getOtherNormalDistributionManagerIds() {
+ // We return a modified copy of the list, so
+ // collect the old list and copy under the lock.
+ Set result = new HashSet(getNormalDistributionManagerIds());
+
+ InternalDistributedMember me = getDistributionManagerId();
+ result.remove(me);
+
+ // It's okay for my own id to not be in the list of all ids yet.
+ return result;
+ }
+
+ public InternalDistributedMember getCanonicalId(DistributedMember id) {
+ // the members set is copy-on-write, so it is safe to iterate over it
+ InternalDistributedMember result = this.members.get(id);
+ if (result == null) {
+ return (InternalDistributedMember)id;
+ }
+ return result;
+ }
+
+ /**
+ * Add a membership listener and return other DistribtionManagerIds
+ * as an atomic operation
+ */
+ public Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l) {
+ // switched sync order to fix bug 30360
+ synchronized (this.membersLock) {
+ // Don't let the members come and go while we are adding this
+ // listener. This ensures that the listener (probably a
+ // ReplyProcessor) gets a consistent view of the members.
+ addMembershipListener(l);
+ // Note it is ok to return the members set
+ // because we will never modify the returned set.
+ return members.keySet();
+ }
+ }
+
+ public void addNewMember(InternalDistributedMember member) {
+ // This is the place to cleanup the zombieMembers
+ int vmType = member.getVmKind();
+ switch (vmType) {
+ case ADMIN_ONLY_DM_TYPE:
+ handleConsoleStartup(member);
+ break;
+ case LOCATOR_DM_TYPE:
+ case NORMAL_DM_TYPE:
+ handleManagerStartup(member);
+ break;
+ default:
+ throw new InternalGemFireError(LocalizedStrings.DistributionManager_UNKNOWN_MEMBER_TYPE_0.toLocalizedString(Integer.valueOf(vmType)));
+ }
+ }
+
+ /**
+ * Returns the identity of this DistributionManager
+ */
+ public InternalDistributedMember getId() {
+ return this.myid;
+ }
+
+ /**
+ * Returns the id of the underlying distribution channel used for
+ * communication.
+ *
+ * @since 3.0
+ */
+ public long getChannelId() {
+ return this.channel.getId();
+ }
+
+ /**
+ * Adds a message to the outgoing queue. Note that
+ * message
should not be modified after it has been
+ * added to the queue. After message
is distributed,
+ * it will be recycled.
+ *
+ * @return list of recipients who did not receive the message
+ * @throws NotSerializableException if the content is not serializable
+ */
+ public Set putOutgoingUserData(final DistributionMessage message)
+ throws NotSerializableException {
+ return sendMessage(message);
+ }
+
+ /**
+ * Send outgoing data; message is guaranteed to be serialized.
+ * @return list of recipients who did not receive the message
+ * @throws InternalGemFireException if message is not serializable
+ */
+ public Set putOutgoing(final DistributionMessage msg) {
+ try {
+ DistributionMessageObserver observer = DistributionMessageObserver.getInstance();
+ if(observer != null) {
+ observer.beforeSendMessage(this, msg);
+ }
+ return sendMessage(msg);
+ }
+ catch (NotSerializableException e) {
+ throw new InternalGemFireException(e);
+ }
+ catch (ToDataException e) {
+ // exception from user code
+ throw e;
+ }
+ }
+
+ @Override
+ public String toString() {
+ return this.description;
+ }
+
+ /**
+ * @see #closeInProgress
+ */
+ private final Object shutdownMutex = new Object();
+
+ /**
+ * Informs other members that this dm is shutting down.
+ * Stops the pusher, puller, and processor threads and closes the
+ * connection to the transport layer.
+ */
+ protected void shutdown() {
+ // Make sure only one thread initiates shutdown...
+ synchronized (shutdownMutex) {
+ if (closeInProgress) {
+ return;
+ }
+ this.closeInProgress = true;
+ } // synchronized
+
+ // [bruce] log shutdown at info level and with ID to balance the
+ // "Starting" message. recycleConn.conf is hard to debug w/o this
+ final String exceptionStatus = (this.exceptionInThreads() ? LocalizedStrings.DistributionManager_AT_LEAST_ONE_EXCEPTION_OCCURRED.toLocalizedString() : "");
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_SHUTTING_DOWN_DISTRIBUTIONMANAGER_0_1,
+ new Object[] {this.myid, exceptionStatus}));
+
+ final long start = System.currentTimeMillis();
+ try {
+ if (this.rootCause instanceof ForcedDisconnectException) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("inhibiting sending of shutdown message to other members due to forced-disconnect");
+ }
+ } else {
+ // Don't block indefinitely trying to send the shutdown message, in
+ // case other VMs in the system are ill-behaved. (bug 34710)
+ final Runnable r = new Runnable() {
+ public void run() {
+ try {
+ ConnectionTable.threadWantsSharedResources();
+ sendShutdownMessage();
+ }
+ catch (final CancelException e) {
+ // We were terminated.
+ logger.debug("Cancelled during shutdown message", e);
+ }
+ }
+ };
+ final Thread t = new Thread(threadGroup,
+ r, LocalizedStrings.DistributionManager_SHUTDOWN_MESSAGE_THREAD_FOR_0.toLocalizedString(this.myid));
+ t.start();
+ boolean interrupted = Thread.interrupted();
+ try {
+ t.join(MAX_STOP_TIME);
+ }
+ catch (final InterruptedException e) {
+ interrupted = true;
+ t.interrupt();
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_INTERRUPTED_SENDING_SHUTDOWN_MESSAGE_TO_PEERS), e);
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ if (t.isAlive()) {
+ t.interrupt();
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_FAILED_SENDING_SHUTDOWN_MESSAGE_TO_PEERS_TIMEOUT));
+ }
+ }
+
+ }
+ finally {
+ this.shutdownMsgSent = true; // in case sendShutdownMessage failed....
+ try {
+ this.uncleanShutdown(false);
+ }
+ finally {
+ final Long delta = Long.valueOf(System.currentTimeMillis() - start);
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGER_STOPPED_IN_0_MS, delta));
+ }
+ }
+ }
+
+ private void askThreadsToStop() {
+ // Stop executors after they have finished
+ ExecutorService es;
+ es = this.serialThread;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.viewThread;
+ if (es != null) {
+ // Hmmm...OK, I'll let any view events currently in the queue be
+ // processed. Not sure it's very important whether they get
+ // handled...
+ es.shutdown();
+ }
+ if (this.serialQueuedExecutorPool != null) {
+ this.serialQueuedExecutorPool.shutdown();
+ }
+ es = this.functionExecutionThread;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.functionExecutionPool;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.partitionedRegionThread;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.partitionedRegionPool;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.highPriorityPool;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.waitingPool;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.prMetaDataCleanupThreadPool;
+ if (es != null) {
+ es.shutdown();
+ }
+ es = this.threadPool;
+ if (es != null) {
+ es.shutdown();
+ }
+
+ Thread th = this.memberEventThread;
+ if (th != null)
+ th.interrupt();
+ }
+
+ private void waitForThreadsToStop(long timeInMillis) throws InterruptedException {
+ long start = System.currentTimeMillis();
+ long remaining = timeInMillis;
+
+ ExecutorService[] allExecutors = new ExecutorService[] {
+ this.serialThread,
+ this.viewThread,
+ this.functionExecutionThread,
+ this.functionExecutionPool,
+ this.partitionedRegionThread,
+ this.partitionedRegionPool,
+ this.highPriorityPool,
+ this.waitingPool,
+ this.prMetaDataCleanupThreadPool,
+ this.threadPool};
+ for(ExecutorService es : allExecutors) {
+ if (es != null) {
+ es.awaitTermination(remaining, TimeUnit.MILLISECONDS);
+ }
+ remaining = timeInMillis - (System.currentTimeMillis() - start);
+ if(remaining <= 0) {
+ return;
+ }
+ }
+
+
+ this.serialQueuedExecutorPool.awaitTermination(remaining, TimeUnit.MILLISECONDS);
+ remaining = timeInMillis - (System.currentTimeMillis() - start);
+ if(remaining <= 0) {
+ return;
+ }
+ Thread th = this.memberEventThread;
+ if (th != null) {
+ th.interrupt(); // bug #43452 - this thread sometimes eats interrupts, so we interrupt it again here
+ th.join(remaining);
+ }
+
+ }
+
+ /**
+ * maximum time, in milliseconds, to wait for all threads to exit
+ */
+ static private final int MAX_STOP_TIME = 20000;
+
+ /**
+ * Time to sleep, in milliseconds, while polling to see if threads have
+ * finished
+ */
+ static private final int STOP_PAUSE_TIME = 1000;
+
+ /**
+ * Maximum number of interrupt attempts to stop a thread
+ */
+ static private final int MAX_STOP_ATTEMPTS = 10;
+
+ /**
+ * Cheap tool to kill a referenced thread
+ *
+ * @param t the thread to kill
+ */
+ private void clobberThread(Thread t) {
+ if (t == null)
+ return;
+ if (t.isAlive()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_FORCING_THREAD_STOP_ON__0_, t));
+
+ // Start by being nice.
+ t.interrupt();
+
+// we could be more violent here...
+// t.stop();
+ try {
+ for (int i = 0; i < MAX_STOP_ATTEMPTS && t.isAlive(); i++) {
+ t.join(STOP_PAUSE_TIME);
+ t.interrupt();
+ }
+ }
+ catch (InterruptedException ex) {
+ logger.debug("Interrupted while attempting to terminate threads.");
+ Thread.currentThread().interrupt();
+ // just keep going
+ }
+
+ if (t.isAlive()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_CLOBBERTHREAD_THREAD_REFUSED_TO_DIE__0, t));
+ }
+ }
+ }
+
+ /**
+ * Cheap tool to examine an executor to see if it is still working
+ * @param tpe
+ * @return true if executor is still active
+ */
+ private boolean executorAlive(ThreadPoolExecutor tpe, String name)
+ {
+ if (tpe == null) {
+ return false;
+ } else {
+ int ac = tpe.getActiveCount();
+// boolean result = tpe.getActiveCount() > 0;
+ if (ac > 0) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Still waiting for {} threads in '{}' pool to exit", ac, name);
+ }
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+
+ /**
+ * Wait for the ancillary queues to exit. Kills them if they are
+ * still around.
+ *
+ */
+ private void forceThreadsToStop() {
+ long endTime = System.currentTimeMillis() + MAX_STOP_TIME;
+ String culprits = "";
+ for (;;) {
+ boolean stillAlive = false;
+ culprits = "";
+ if (executorAlive(this.serialThread, "serial thread")) {
+ stillAlive = true;
+ culprits = culprits + " serial thread;";
+ }
+ if (executorAlive(this.viewThread, "view thread")) {
+ stillAlive = true;
+ culprits = culprits + " view thread;";
+ }
+ if (executorAlive(this.partitionedRegionThread, "partitioned region thread")) {
+ stillAlive = true;
+ culprits = culprits + " partitioned region thread;";
+ }
+ if (executorAlive(this.partitionedRegionPool, "partitioned region pool")) {
+ stillAlive = true;
+ culprits = culprits + " partitioned region pool;";
+ }
+ if (executorAlive(this.highPriorityPool, "high priority pool")) {
+ stillAlive = true;
+ culprits = culprits + " high priority pool;";
+ }
+ if (executorAlive(this.waitingPool, "waiting pool")) {
+ stillAlive = true;
+ culprits = culprits + " waiting pool;";
+ }
+ if (executorAlive(this.prMetaDataCleanupThreadPool, "prMetaDataCleanupThreadPool")) {
+ stillAlive = true;
+ culprits = culprits + " special waiting pool;";
+ }
+ if (executorAlive(this.threadPool, "thread pool")) {
+ stillAlive = true;
+ culprits = culprits + " thread pool;";
+ }
+
+ if (!stillAlive)
+ return;
+
+ long now = System.currentTimeMillis();
+ if (now >= endTime)
+ break;
+
+ try {
+ Thread.sleep(STOP_PAUSE_TIME);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ // Desperation, the shutdown thread is being killed. Don't
+ // consult a CancelCriterion.
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_INTERRUPTED_DURING_SHUTDOWN), e);
+ break;
+ }
+ } // for
+
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_DAEMON_THREADS_ARE_SLOW_TO_STOP_CULPRITS_INCLUDE_0, culprits));
+
+ // Kill with no mercy
+ if (this.serialThread != null) {
+ this.serialThread.shutdownNow();
+ }
+ if (this.viewThread != null) {
+ this.viewThread.shutdownNow();
+ }
+ if (this.functionExecutionThread != null) {
+ this.functionExecutionThread.shutdownNow();
+ }
+ if (this.functionExecutionPool != null) {
+ this.functionExecutionPool.shutdownNow();
+ }
+ if (this.partitionedRegionThread != null) {
+ this.partitionedRegionThread.shutdownNow();
+ }
+ if (this.partitionedRegionPool != null) {
+ this.partitionedRegionPool.shutdownNow();
+ }
+ if (this.highPriorityPool != null) {
+ this.highPriorityPool.shutdownNow();
+ }
+ if (this.waitingPool != null) {
+ this.waitingPool.shutdownNow();
+ }
+ if (this.prMetaDataCleanupThreadPool != null) {
+ this.prMetaDataCleanupThreadPool.shutdownNow();
+ }
+ if (this.threadPool != null) {
+ this.threadPool.shutdownNow();
+ }
+
+ Thread th = this.memberEventThread;
+ if (th != null) {
+ clobberThread(th);
+ }
+ }
+
+ private volatile boolean shutdownInProgress = false;
+
+ /** guard for membershipViewIdAcknowledged */
+ private final Object membershipViewIdGuard = new Object();
+
+ /** the latest view ID that has been processed by all membership listeners */
+ private long membershipViewIdAcknowledged;
+
+ public boolean shutdownInProgress() {
+ return this.shutdownInProgress;
+ }
+
+ /**
+ * Stops the pusher, puller and processor threads and closes the
+ * connection to the transport layer. This should only be used from
+ * shutdown() or from the dm initialization code
+ */
+ private void uncleanShutdown(boolean duringStartup)
+ {
+ try {
+ this.closeInProgress = true; // set here also to fix bug 36736
+ removeAllHealthMonitors();
+ shutdownInProgress = true;
+ if (this.channel != null) {
+ this.channel.setShutDown();
+ }
+
+ askThreadsToStop();
+
+ // wait a moment before asking threads to terminate
+ try { waitForThreadsToStop(1000); }
+ catch (InterruptedException ie) {
+ // No need to reset interrupt bit, we're really trying to quit...
+ }
+ forceThreadsToStop();
+
+// // bug36329: desperation measure, send a second interrupt?
+// try { Thread.sleep(1000); }
+// catch (InterruptedException ie) {
+// // No need to reset interrupt bit, we're really trying to quit...
+// }
+// forceThreadsToStop();
+ } // try
+ finally {
+ // ABSOLUTELY ESSENTIAL that we close the distribution channel!
+ try {
+ // For safety, but channel close in a finally AFTER this...
+ if (this.stats != null) {
+ this.stats.close();
+ try { Thread.sleep(100); }
+ catch (InterruptedException ie) {
+ // No need to reset interrupt bit, we're really trying to quit...
+ }
+ }
+ }
+ finally {
+ if (this.channel != null) {
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_NOW_CLOSING_DISTRIBUTION_FOR__0, this.myid));
+ this.channel.disconnect(duringStartup);
+// this.channel = null; DO NOT NULL OUT INSTANCE VARIABLES AT SHUTDOWN - bug #42087
+ }
+ }
+ }
+ }
+
+ /**
+ * Returns the distributed system to which this distribution manager
+ * is connected.
+ */
+ public InternalDistributedSystem getSystem() {
+ return this.system;
+ }
+
+ /**
+ * Returns the transport configuration for this distribution manager
+ * @since 5.0
+ */
+ public RemoteTransportConfig getTransport() {
+ return this.transport;
+ }
+
+
+ /**
+ * Adds a MembershipListener
to this distribution
+ * manager.
+ */
+ public void addMembershipListener(MembershipListener l) {
+ this.membershipListeners.putIfAbsent(l, Boolean.TRUE);
+ }
+
+ /**
+ * Removes a MembershipListener
from this distribution
+ * manager.
+ *
+ * @throws IllegalArgumentException
+ * l
was not registered on this distribution
+ * manager
+ */
+ public void removeMembershipListener(MembershipListener l) {
+ this.membershipListeners.remove(l);
+ }
+
+ /**
+ * Adds a MembershipListener
to this distribution
+ * manager.
+ * @since 5.7
+ */
+ public void addAllMembershipListener(MembershipListener l) {
+ synchronized (this.allMembershipListenersLock) {
+ Set newAllMembershipListeners =
+ new HashSet(this.allMembershipListeners);
+ newAllMembershipListeners.add(l);
+ this.allMembershipListeners = newAllMembershipListeners;
+ }
+ }
+
+ /**
+ * Removes a MembershipListener
listening for all members
+ * from this distribution manager.
+ *
+ * @throws IllegalArgumentException
+ * l
was not registered on this distribution
+ * manager
+ * @since 5.7
+ */
+ public void removeAllMembershipListener(MembershipListener l) {
+ synchronized (this.allMembershipListenersLock) {
+ Set newAllMembershipListeners =
+ new HashSet(this.allMembershipListeners);
+ if (!newAllMembershipListeners.remove(l)) {
+ // There seems to be a race condition in which
+ // multiple departure events can be registered
+ // on the same peer. We regard this as benign.
+ // FIXME when membership events become sane again
+// String s = "MembershipListener was never registered";
+// throw new IllegalArgumentException(s);
+ }
+ this.allMembershipListeners = newAllMembershipListeners;
+ }
+ }
+
+ /**
+ * Returns true if this DM or the DistributedSystem owned by
+ * it is closing or is closed.
+ */
+ private boolean isCloseInProgress() {
+ if (closeInProgress) {
+ return true;
+ }
+ InternalDistributedSystem ds = getSystem();
+ if (ds != null && ds.isDisconnecting()) {
+ return true;
+ }
+ return false;
+ }
+
+ private void handleViewInstalledEvent(ViewInstalledEvent ev) {
+ synchronized(this.membershipViewIdGuard) {
+ this.membershipViewIdAcknowledged = ev.getViewId();
+ this.membershipViewIdGuard.notifyAll();
+ }
+ }
+
+ /**
+ * This stalls waiting for the current membership view (as seen by the
+ * membership manager) to be acknowledged by all membership listeners
+ */
+ public void waitForViewInstallation(long id) throws InterruptedException {
+ if (id <= this.membershipViewIdAcknowledged) {
+ return;
+ }
+ synchronized(this.membershipViewIdGuard) {
+ while (this.membershipViewIdAcknowledged < id && this.stopper.cancelInProgress() == null) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("waiting for view {}. Current DM view processed by all listeners is {}", id, this.membershipViewIdAcknowledged);
+ }
+ this.membershipViewIdGuard.wait();
+ }
+ }
+ }
+
+ protected void handleMemberEvent(MemberEvent ev) {
+ ev.handleEvent(this);
+ }
+
+ /**
+ * This thread processes member events as they occur.
+ *
+ * @see com.gemstone.gemfire.distributed.internal.DistributionManager.MemberCrashedEvent
+ * @see com.gemstone.gemfire.distributed.internal.DistributionManager.MemberJoinedEvent
+ * @see com.gemstone.gemfire.distributed.internal.DistributionManager.MemberDepartedEvent
+ *
+ */
+ protected class MemberEventInvoker implements Runnable {
+
+
+ @SuppressWarnings("synthetic-access")
+ public void run() {
+ for (;;) {
+ SystemFailure.checkFailure();
+ // bug 41539 - member events need to be delivered during shutdown
+ // or reply processors may hang waiting for replies from
+ // departed members
+// if (getCancelCriterion().cancelInProgress() != null) {
+// break; // no message, just quit
+// }
+ if (!DistributionManager.this.system.isConnected &&
+ DistributionManager.this.isClosed()) {
+ break;
+ }
+ try {
+ MemberEvent ev = (MemberEvent)DistributionManager.this
+ .membershipEventQueue.take();
+ handleMemberEvent(ev);
+ }
+ catch (InterruptedException e) {
+ if (isCloseInProgress()) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("MemberEventInvoker: InterruptedException during shutdown");
+ }
+ }
+ else {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_UNEXPECTED_INTERRUPTEDEXCEPTION), e);
+ }
+ break;
+ }
+ catch (DistributedSystemDisconnectedException e) {
+ break;
+ }
+ catch (CancelException e) {
+ if (isCloseInProgress()) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("MemberEventInvoker: cancelled");
+ }
+ }
+ else {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_UNEXPECTED_CANCELLATION), e);
+ }
+ break;
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributionManager_UNCAUGHT_EXCEPTION_PROCESSING_MEMBER_EVENT), e);
+ }
+ } // for
+ if (logger.isTraceEnabled()) {
+ logger.trace("MemberEventInvoker on {} stopped", DistributionManager.this);
+ }
+ }
+ }
+
+ private void addMemberEvent(MemberEvent ev) {
+ if (SYNC_EVENTS) {
+ handleMemberEvent(ev);
+ } else {
+ stopper.checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ this.membershipEventQueue.put(ev);
+ } catch (InterruptedException ex) {
+ interrupted = true;
+ stopper.checkCancelInProgress(ex);
+ handleMemberEvent(ev); // FIXME why???
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+ }
+
+
+ /**
+ * Stops the threads associated with this distribution manager and
+ * closes the connection to the transport layer.
+ */
+ public void close() {
+ if (!closed) {
+ this.shutdown();
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_MARKING_DISTRIBUTIONMANAGER_0_AS_CLOSED, this.myid));
+ MembershipLogger.logShutdown(this.myid);
+ closed = true;
+ synchronized (DistributionManager.class) {
+ openDMs--;
+ }
+ }
+ }
+
+ public void throwIfDistributionStopped() {
+ if (this.shutdownMsgSent) {
+ throw new DistributedSystemDisconnectedException(LocalizedStrings.DistributionManager_MESSAGE_DISTRIBUTION_HAS_TERMINATED.toLocalizedString(), this.getRootCause());
+ }
+ }
+
+ /**
+ * Returns true if this distribution manager has been closed.
+ */
+ public boolean isClosed() {
+ return this.closed;
+ }
+
+ /**
+ * Makes note of a new administration console (admin-only member).
+ */
+ public void addAdminConsole(InternalDistributedMember theId) {
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_NEW_ADMINISTRATION_MEMBER_DETECTED_AT_0, theId));
+ synchronized(this.adminConsolesLock) {
+ HashSet tmp = new HashSet(this.adminConsoles);
+ tmp.add(theId);
+ this.adminConsoles = Collections.unmodifiableSet(tmp);
+ }
+ }
+
+ public DMStats getStats() {
+ return this.stats;
+ }
+
+ public DistributionConfig getConfig() {
+ DistributionConfig result = null;
+ InternalDistributedSystem sys = getSystem();
+ if (sys != null) {
+ result = system.getConfig();
+ }
+ return result;
+ }
+
+// /**
+// * Initializes and returns a DistributedSystem
to be
+// * sent to new members of the distributed system.
+// *
+// * @since 3.0
+// */
+// protected DistributedState getNewDistributedState() {
+// DistributedState state = new DistributedState();
+// state.setGemFireVersion(GemFireVersion.getGemFireVersion());
+// state.setCacheTime(this.cacheTimeMillis());
+// return state;
+//}
+
+ private static final int STARTUP_TIMEOUT =
+ Integer.getInteger("DistributionManager.STARTUP_TIMEOUT", 15000).intValue();
+
+ public static final boolean DEBUG_NO_ACKNOWLEDGEMENTS = Boolean.getBoolean("DistributionManager.DEBUG_NO_ACKNOWLEDGEMENTS");
+
+ public Set getAllOtherMembers() {
+ Set result = new HashSet(getDistributionManagerIdsIncludingAdmin());
+ result.remove(getDistributionManagerId());
+ return result;
+ }
+
+ @Override // DM method
+ public void retainMembersWithSameOrNewerVersion(Collection members, Version version) {
+ for (Iterator it = members.iterator(); it.hasNext(); ) {
+ InternalDistributedMember id = it.next();
+ if (id.getVersionObject().compareTo(version) < 0) {
+ it.remove();
+ }
+ }
+ }
+
+ @Override // DM method
+ public void removeMembersWithSameOrNewerVersion(Collection members, Version version) {
+ for (Iterator it = members.iterator(); it.hasNext(); ) {
+ InternalDistributedMember id = it.next();
+ if (id.getVersionObject().compareTo(version) >= 0) {
+ it.remove();
+ }
+ }
+ }
+
+ /**
+ * Add a membership listener for all members
+ * and return other DistribtionManagerIds as an atomic operation
+ * @since 5.7
+ */
+ public Set addAllMembershipListenerAndGetAllIds(MembershipListener l) {
+ // TO fix this deadlock:
+ // "View Message Processor":
+ // waiting to lock monitor 0x080f691c (object 0xe3ba7680, a com.gemstone.gemfire.distributed.internal.DistributionManager$MembersLock),
+ // which is held by "RMI TCP Connection(259)-10.80.10.55"
+ // "RMI TCP Connection(259)-10.80.10.55":
+ // waiting to lock monitor 0x080f6598 (object 0xe3bacd90, a com.gemstone.gemfire.distributed.internal.membership.jgroup.JGroupMembershipManager$ViewLock),
+ // which is held by "View Message Processor"
+ // NEED to sync on viewLock first.
+ DistributionChannel ch = this.channel;
+ if (ch != null) {
+ MembershipManager mgr = ch.getMembershipManager();
+ if (mgr != null) {
+ mgr.getViewLock().writeLock().lock();
+ try {
+ synchronized (this.membersLock) {
+ // Don't let the members come and go while we are adding this
+ // listener. This ensures that the listener (probably a
+ // ReplyProcessor) gets a consistent view of the members.
+ addAllMembershipListener(l);
+ return getDistributionManagerIdsIncludingAdmin();
+ }
+ } finally {
+ mgr.getViewLock().writeLock().unlock();
+ }
+ }
+ }
+ // If we have no channel or MembershipManager then the view is empty
+ synchronized (this.membersLock) {
+ // Don't let the members come and go while we are adding this
+ // listener. This ensures that the listener (probably a
+ // ReplyProcessor) gets a consistent view of the members.
+ addAllMembershipListener(l);
+ return Collections.EMPTY_SET;
+ }
+ }
+
+ /**
+ * Sends a startup message and waits for a response.
+ * Returns true if response received; false if it timed out or there are no peers.
+ */
+ protected boolean sendStartupMessage(StartupOperation op, boolean cancelOnTimeout)
+ throws InterruptedException
+ {
+ if (Thread.interrupted()) throw new InterruptedException();
+ this.receivedStartupResponse = false;
+ boolean ok = false;
+
+ // Be sure to add ourself to the equivalencies list!
+ Set equivs = StartupMessage.getMyAddresses(this);
+ if (equivs == null || equivs.size() == 0) {
+ // no network interface
+ equivs = new HashSet();
+ try {
+ equivs.add(SocketCreator.getLocalHost());
+ } catch (UnknownHostException e) {
+ // can't even get localhost
+ if (getViewMembers().size() > 1) {
+ throw new SystemConnectException("Unable to examine network cards and other members exist");
+ }
+ }
+ }
+ setEquivalentHosts(equivs);
+ setEnforceUniqueZone(getConfig().getEnforceUniqueHost());
+ String redundancyZone = getConfig().getRedundancyZone();
+ if(redundancyZone != null && !redundancyZone.equals("")) {
+ setEnforceUniqueZone(true);
+ }
+ setRedundancyZone(getDistributionManagerId(), redundancyZone);
+ if (logger.isDebugEnabled()) {
+ StringBuffer sb = new StringBuffer();
+ sb.append("Equivalent IPs for this host: ");
+ Iterator it = equivs.iterator();
+ while (it.hasNext()) {
+ InetAddress in = (InetAddress)it.next();
+ sb.append(in.toString());
+ if (it.hasNext()) {
+ sb.append(", ");
+ }
+ } // while
+ logger.debug(sb);
+ }
+
+ // we need to send this to everyone else; even admin vm
+ Set allOthers = new HashSet(getViewMembers());
+ allOthers.remove(getDistributionManagerId());
+
+ if (allOthers.isEmpty()) {
+ return false; // no peers, we are alone.
+ }
+
+ try {
+ ok = op.sendStartupMessage(allOthers, STARTUP_TIMEOUT, equivs,
+ redundancyZone, enforceUniqueZone());
+ }
+ catch (Exception re) {
+ throw new SystemConnectException(LocalizedStrings.DistributionManager_ONE_OR_MORE_PEERS_GENERATED_EXCEPTIONS_DURING_CONNECTION_ATTEMPT.toLocalizedString(), re);
+ }
+ if (this.rejectionMessage != null) {
+ throw new IncompatibleSystemException(rejectionMessage);
+ }
+
+ boolean isAdminDM =
+ getId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE
+ || getId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE
+ || DistributionManager.isDedicatedAdminVM
+ || Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE);
+
+ boolean receivedAny = this.receivedStartupResponse;
+
+ if (!ok) { // someone didn't reply
+ int unresponsiveCount;
+
+ synchronized (unfinishedStartupsLock) {
+ if (unfinishedStartups == null)
+ unresponsiveCount = 0;
+ else
+ unresponsiveCount = unfinishedStartups.size();
+
+ if (unresponsiveCount != 0) {
+ if (Boolean.getBoolean("DistributionManager.requireAllStartupResponses")) {
+ throw new SystemConnectException(LocalizedStrings.DistributionManager_NO_STARTUP_REPLIES_FROM_0.toLocalizedString(unfinishedStartups));
+ }
+ }
+ } // synchronized
+
+
+ // Bug 35887:
+ // If there are other members, we must receive at least _one_ response
+ if (allOthers.size() != 0) { // there exist others
+ if (!receivedAny) { // and none responded
+ StringBuffer sb = new StringBuffer();
+ Iterator itt = allOthers.iterator();
+ while (itt.hasNext()) {
+ Object m = itt.next();
+ sb.append(m.toString());
+ if (itt.hasNext())
+ sb.append(", ");
+ }
+ if (DEBUG_NO_ACKNOWLEDGEMENTS) {
+ printStacks(allOthers, false);
+ }
+ throw new SystemConnectException(LocalizedStrings.DistributionManager_RECEIVED_NO_CONNECTION_ACKNOWLEDGMENTS_FROM_ANY_OF_THE_0_SENIOR_CACHE_MEMBERS_1.toLocalizedString(new Object[] {Integer.toString(allOthers.size()), sb.toString()}));
+ } // and none responded
+ } // there exist others
+
+ InternalDistributedMember e = getElderId();
+ if (e != null) { // an elder exists
+ boolean unresponsiveElder;
+ synchronized (unfinishedStartupsLock) {
+ if (unfinishedStartups == null)
+ unresponsiveElder = false;
+ else
+ unresponsiveElder = unfinishedStartups.contains(e);
+ }
+ if (unresponsiveElder) {
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_FORCING_AN_ELDER_JOIN_EVENT_SINCE_A_STARTUP_RESPONSE_WAS_NOT_RECEIVED_FROM_ELDER__0_, e));
+ handleManagerStartup(e);
+ }
+ } // an elder exists
+ } // someone didn't reply
+ return receivedAny;
+ }
+
+ /**
+ * List of InternalDistributedMember's that we have
+ * not received startup replies from. If null, we have
+ * not finished sending the startup message.
+ *
+ * Must be synchronized using {@link #unfinishedStartupsLock}
+ */
+ private Set unfinishedStartups = null;
+
+ /**
+ * Synchronization for {@link #unfinishedStartups}
+ */
+ private final Object unfinishedStartupsLock = new Object();
+
+ public void setUnfinishedStartups(Collection s) {
+ synchronized (unfinishedStartupsLock) {
+ Assert.assertTrue(unfinishedStartups == null,
+ "Set unfinished startups twice");
+ unfinishedStartups = new HashSet(s);
+
+ // OK, I don't _quite_ trust the list to be current, so let's
+ // prune it here.
+ Iterator it = unfinishedStartups.iterator();
+ synchronized (this.membersLock) {
+ while (it.hasNext()) {
+ InternalDistributedMember m = (InternalDistributedMember)it.next();
+ if (!isCurrentMember(m)) {
+ it.remove();
+ }
+ } // while
+ } // synchronized
+ }
+ }
+
+ public void removeUnfinishedStartup(InternalDistributedMember m,
+ boolean departed) {
+ synchronized (unfinishedStartupsLock) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("removeUnfinishedStartup for {} wtih {}", m, unfinishedStartups);
+ }
+ if (unfinishedStartups == null)
+ return; // not yet done with startup
+ if (!unfinishedStartups.remove(m))
+ return;
+ StringId msg = null;
+ if (departed) {
+ msg = LocalizedStrings.DistributionManager_STOPPED_WAITING_FOR_STARTUP_REPLY_FROM_0_BECAUSE_THE_PEER_DEPARTED_THE_VIEW;
+ }
+ else {
+ msg = LocalizedStrings.DistributionManager_STOPPED_WAITING_FOR_STARTUP_REPLY_FROM_0_BECAUSE_THE_REPLY_WAS_FINALLY_RECEIVED;
+ }
+ logger.info(LocalizedMessage.create(msg, m));
+ int numLeft = unfinishedStartups.size();
+ if (numLeft != 0) {
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_STILL_AWAITING_0_RESPONSES_FROM_1,
+ new Object[] {Integer.valueOf(numLeft), unfinishedStartups}));
+ }
+ } // synchronized
+ }
+
+ /**
+ * Processes the first startup response.
+ *
+ * @see StartupResponseMessage#process
+ */
+ void processStartupResponse(InternalDistributedMember sender,
+ long otherCacheTime, String theRejectionMessage) {
+ removeUnfinishedStartup(sender, false);
+ synchronized (this) {
+ if (!this.receivedStartupResponse) {
+ this.receivedStartupResponse = true;
+ }
+ if (theRejectionMessage != null && this.rejectionMessage == null) {
+ // remember the first non-null rejection. This fixes bug 33266
+ this.rejectionMessage = theRejectionMessage;
+ }
+ }
+ }
+
+ /**
+ * Processes the first startup response.
+ *
+ * @see StartupResponseMessage#process
+ */
+ void processStartupResponse(InternalDistributedMember sender,
+ String theRejectionMessage) {
+ removeUnfinishedStartup(sender, false);
+ synchronized (this) {
+ if (!this.receivedStartupResponse) {
+ // only set the cacheTimeDelta once
+ this.receivedStartupResponse = true;
+ }
+ if (theRejectionMessage != null && this.rejectionMessage == null) {
+ // remember the first non-null rejection. This fixes bug 33266
+ this.rejectionMessage = theRejectionMessage;
+ }
+ }
+ }
+
+ /**
+ * Based on a recent JGroups view, return a member that might be the
+ * next elder.
+ * @return the elder candidate, possibly this VM.
+ */
+ private InternalDistributedMember getElderCandidate() {
+ List theMembers = getViewMembers();
+
+// Assert.assertTrue(!closeInProgress
+// && theMembers.contains(this.myid)); // bug36202?
+
+ int elderCandidates = 0;
+ Iterator it;
+
+ // for bug #50510 we need to know if there are any members older than v8.0
+ it = theMembers.iterator();
+ boolean anyPre80Members = false;
+ while (it.hasNext()) {
+ InternalDistributedMember member = it.next();
+ if (member.getVersionObject().compareTo(Version.GFE_80) < 0) {
+ anyPre80Members = true;
+ }
+ }
+
+ // determine number of elder candidates (unless adam)
+ if (!this.adam) {
+ it = theMembers.iterator();
+ while (it.hasNext()) {
+ InternalDistributedMember member = it.next();
+ int managerType = member.getVmKind();
+ if (managerType == ADMIN_ONLY_DM_TYPE)
+ continue;
+
+ if (managerType == LOCATOR_DM_TYPE) {
+ // Fix for #50510 - pre-8.0 members will not let a locator be the elder
+ // so we need to make the same decision here
+ if (anyPre80Members) {
+ continue;
+ }
+ }
+
+ // Fix for #45566. Using a surprise member as the elder can cause a
+ // deadlock.
+ if (getMembershipManager().isSurpriseMember(member)) {
+ continue;
+ }
+
+ elderCandidates++;
+ if (elderCandidates > 1) {
+ // If we have more than one candidate then we are not adam
+ break;
+ }
+ } // while
+ }
+
+ // Second pass over members...
+ it = theMembers.iterator();
+ while (it.hasNext()) {
+ InternalDistributedMember member = it.next();
+ int managerType = member.getVmKind();
+ if (managerType == ADMIN_ONLY_DM_TYPE)
+ continue;
+
+ if (managerType == LOCATOR_DM_TYPE) {
+ // Fix for #50510 - pre-8.0 members will not let a locator be the elder
+ // so we need to make the same decision here
+ if (anyPre80Members) {
+ continue;
+ }
+ }
+
+ // Fix for #45566. Using a surprise member as the elder can cause a
+ // deadlock.
+ if (getMembershipManager().isSurpriseMember(member)) {
+ continue;
+ }
+
+ if (member.equals(this.myid)) { // c'est moi
+ if (!this.adam && elderCandidates == 1) {
+ this.adam = true;
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_0_IS_THE_ELDER_AND_THE_ONLY_MEMBER, this.myid));
+ } else {
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_I_0_AM_THE_ELDER, this.myid));
+ }
+ } // c'est moi
+ return member;
+ } // while
+ // If we get this far then no elder exists
+ return null;
+ }
+
+ /**
+ * Select a new elder
+ *
+ */
+ protected void selectElder() {
+ getSystem().getCancelCriterion().checkCancelInProgress(null); // bug 37884, if DS is disconnecting, throw exception
+
+ // Once we are the elder, we're stuck until we leave the view.
+ if (this.myid.equals(this.elder)) {
+ return;
+ }
+
+ // Determine who is the elder...
+ InternalDistributedMember candidate = getElderCandidate();
+ if (candidate == null) {
+ changeElder(null);
+ return; // No valid elder in current context
+ }
+
+ // Carefully switch to new elder
+ synchronized (this.elderMonitor) {
+ if (!candidate.equals(this.elder)) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("The elder is: {} (was {})", candidate, this.elder);
+ }
+ changeElder(candidate);
+ }
+ } // synchronized
+ }
+
+ private String prettifyReason(String r) {
+ final String str = "java.io.IOException:";
+ if (r.startsWith(str)) {
+ return r.substring(str.length());
+ }
+ return r;
+ }
+
+ /**
+ * Returns true if id was removed.
+ * Returns false if it was not in the list of managers.
+ */
+ private boolean removeManager(InternalDistributedMember theId,
+ boolean crashed, String p_reason) {
+ String reason = p_reason;
+ boolean result = false; // initialization shouldn't be required, but...
+
+ // Test once before acquiring the lock, fault tolerance for potentially
+ // recursive (and deadlock) conditions -- bug33626
+ // Note that it is always safe to _read_ {@link members} without locking
+ if (isCurrentMember(theId)) {
+ // Destroy underlying member's resources
+ reason = prettifyReason(reason);
+ synchronized (this.membersLock) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("DistributionManager: removing member <{}>; crashed {}; reason = {}", theId, crashed, reason);
+ }
+ Map tmp = new HashMap(this.members);
+ if (tmp.remove(theId) != null) {
+ // Note we don't modify in place. This allows reader to get snapshots
+ // without locking.
+ if (tmp.isEmpty()) {
+ tmp = Collections.EMPTY_MAP;
+ } else {
+ tmp = Collections.unmodifiableMap(tmp);
+ }
+ this.members = tmp;
+ result = true;
+
+ } else {
+ result = false;
+ // Don't get upset since this can happen twice due to
+ // an explicit remove followed by an implicit one caused
+ // by a JavaGroup view change
+ }
+ Set tmp2 = new HashSet(this.membersAndAdmin);
+ if(tmp2.remove(theId)) {
+ if (tmp2.isEmpty()) {
+ tmp2 = Collections.EMPTY_SET;
+ } else {
+ tmp2 = Collections.unmodifiableSet(tmp2);
+ }
+ this.membersAndAdmin = tmp2;
+ }
+ this.removeHostedLocators(theId);
+ } // synchronized
+ } // if
+
+ // In any event, make sure that this member is no longer an elder.
+ if (!theId.equals(myid) && theId.equals(elder)) {
+ try {
+ selectElder();
+ }
+ catch (DistributedSystemDisconnectedException e) {
+ // ignore
+ }
+ }
+
+ redundancyZones.remove(theId);
+
+ return result;
+ }
+
+ /**
+ * Makes note of a new distribution manager that has started up in
+ * the distributed cache. Invokes the appropriately listeners.
+ *
+ * @param theId
+ * The id of the distribution manager starting up
+ *
+ */
+ private void handleManagerStartup(InternalDistributedMember theId) {
+ HashMap tmp = null;
+ synchronized (this.membersLock) {
+ // Note test is under membersLock
+ if (members.containsKey(theId)) {
+ return; // already accounted for
+ }
+
+ // Note we don't modify in place. This allows reader to get snapshots
+ // without locking.
+ tmp = new HashMap(this.members);
+ tmp.put(theId,theId);
+ this.members = Collections.unmodifiableMap(tmp);
+
+ Set stmp = new HashSet(this.membersAndAdmin);
+ stmp.add(theId);
+ this.membersAndAdmin = Collections.unmodifiableSet(stmp);
+ } // synchronized
+
+ if (theId.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
+ this.stats.incNodes(1);
+ }
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_ADMITTING_MEMBER_0_NOW_THERE_ARE_1_NONADMIN_MEMBERS,
+ new Object[] { theId, Integer.valueOf(tmp.size())}));
+ addMemberEvent(new MemberJoinedEvent(theId));
+ }
+
+ /**
+ * Return true if id is a current member of our system.
+ */
+ public boolean isCurrentMember(InternalDistributedMember id) {
+ Set m;
+ synchronized (this.membersLock) {
+ // access to members synchronized under membersLock in order to
+ // ensure serialization
+ m = this.membersAndAdmin;
+ }
+ return m.contains(id);
+ }
+
+ /**
+ * Makes note of a new console that has started up in
+ * the distributed cache.
+ *
+ */
+ private void handleConsoleStartup(InternalDistributedMember theId) {
+ // if we have an all listener then notify it NOW.
+ HashSet tmp = null;
+ synchronized (this.membersLock) {
+ // Note test is under membersLock
+ if (membersAndAdmin.contains(theId))
+ return; // already accounted for
+
+ // Note we don't modify in place. This allows reader to get snapshots
+ // without locking.
+ tmp = new HashSet(this.membersAndAdmin);
+ tmp.add(theId);
+ this.membersAndAdmin = Collections.unmodifiableSet(tmp);
+ } // synchronized
+
+ for (Iterator iter = allMembershipListeners.iterator();
+ iter.hasNext(); ) {
+ MembershipListener listener = (MembershipListener) iter.next();
+ listener.memberJoined(theId);
+ }
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_DMMEMBERSHIP_ADMITTING_NEW_ADMINISTRATION_MEMBER__0_, theId));
+ // Note that we don't add the member to the list of admin consoles until
+ // we receive a message from them.
+ }
+
+ /**
+ * Process an incoming distribution message.
+ * This includes scheduling it correctly based on the message's
+ * nioPriority (executor type)
+ */
+ public void handleIncomingDMsg(DistributionMessage message) {
+ /* disabled - not being used
+ if (message instanceof OutgoingMessageWrapper) {
+ putOutgoing(((OutgoingMessageWrapper)message).getMessage());
+ return;
+ }
+ */
+
+// long latency = message.getLatency();
+// this.stats.incMessageTransitTime(latency * 1000000L);
+// message.resetTimestamp();
+ stats.incReceivedMessages(1L);
+ stats.incReceivedBytes(message.getBytesRead());
+ stats.incMessageChannelTime(message.resetTimestamp());
+
+ // message.setRecipient(DistributionManager.this.getId());
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("Received message '{}' from <{}>", message, message.getSender());
+ }
+ scheduleIncomingMessage(message);
+ }
+
+ /**
+ * Makes note of a console that has shut down.
+ * @param theId
+ * The id of the console shutting down
+ * @param crashed only true if we detect this id to be gone from
+ * a javagroup view
+ *
+ * @see AdminConsoleDisconnectMessage#process
+ */
+ public void handleConsoleShutdown(InternalDistributedMember theId, boolean crashed,
+ String reason) {
+ boolean removedConsole = false;
+ boolean removedMember = false;
+ synchronized (this.membersLock) {
+ // to fix bug 39747 we can only remove this member from
+ // membersAndAdmin if he is not in members.
+ // This happens when we have an admin guy colocated with a normal DS.
+ // In this case we need for the normal DS to shutdown or crash.
+ if (!this.members.containsKey(theId)) {
+ if (logger.isDebugEnabled())
+ logger.debug("DistributionManager: removing admin member <{}>; crashed = {}; reason = {}", theId, crashed, reason);
+ Set tmp = new HashSet(this.membersAndAdmin);
+ if (tmp.remove(theId)) {
+ // Note we don't modify in place. This allows reader to get snapshots
+ // without locking.
+ if (tmp.isEmpty()) {
+ tmp = Collections.EMPTY_SET;
+ } else {
+ tmp = Collections.unmodifiableSet(tmp);
+ }
+ this.membersAndAdmin = tmp;
+ removedMember = true;
+ } else {
+ // Don't get upset since this can happen twice due to
+ // an explicit remove followed by an implicit one caused
+ // by a JavaGroup view change
+ }
+ }
+ removeHostedLocators(theId);
+ }
+ synchronized(this.adminConsolesLock) {
+ if (this.adminConsoles.contains(theId)) {
+ removedConsole = true;
+ Set tmp = new HashSet(this.adminConsoles);
+ tmp.remove(theId);
+ if (tmp.isEmpty()) {
+ tmp = Collections.EMPTY_SET;
+ } else {
+ tmp = Collections.unmodifiableSet(tmp);
+ }
+ this.adminConsoles = tmp;
+ }
+ }
+ if (removedMember) {
+ for (Iterator iter = allMembershipListeners.iterator();
+ iter.hasNext(); ) {
+ MembershipListener listener = (MembershipListener) iter.next();
+ listener.memberDeparted(theId, crashed);
+ }
+ }
+ if (removedConsole) {
+ StringId msg = null;
+ if (crashed) {
+ msg = LocalizedStrings.DistributionManager_ADMINISTRATION_MEMBER_AT_0_CRASHED_1;
+ } else {
+ msg = LocalizedStrings.DistributionManager_ADMINISTRATION_MEMBER_AT_0_CLOSED_1;
+ }
+ logger.info(LocalizedMessage.create(msg, new Object[] {theId, reason}));
+ }
+
+ redundancyZones.remove(theId);
+ }
+
+ public void shutdownMessageReceived(InternalDistributedMember theId, String reason) {
+ this.membershipManager.shutdownMessageReceived(theId, reason);
+ handleManagerDeparture(theId, false, LocalizedStrings.ShutdownMessage_SHUTDOWN_MESSAGE_RECEIVED.toLocalizedString());
+ }
+
+ /** used by the DistributedMembershipListener and startup and shutdown operations, this
+ method decrements the number of nodes and handles lower-level clean up of
+ the resources used by the departed manager */
+ public void handleManagerDeparture(InternalDistributedMember theId,
+ boolean p_crashed, String p_reason) {
+ boolean crashed = p_crashed;
+ String reason = p_reason;
+
+ AlertAppender.getInstance().removeAlertListener(theId);
+
+ // this fixes a race introduced in 5.0.1 by the fact that an explicit
+ // shutdown will cause a member to no longer be in our DM membership
+ // but still in the javagroup view.
+ try {
+ selectElder();
+ }
+ catch (DistributedSystemDisconnectedException e) {
+ // keep going
+ }
+
+
+
+ int vmType = theId.getVmKind();
+ if (vmType == ADMIN_ONLY_DM_TYPE) {
+ removeUnfinishedStartup(theId, true);
+ handleConsoleShutdown(theId, crashed, reason);
+ return;
+ }
+
+ // not an admin VM...
+ if (!isCurrentMember(theId)) {
+ return; // fault tolerance
+ }
+ removeUnfinishedStartup(theId, true);
+
+ if (removeManager(theId, crashed, reason)) {
+ if (theId.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
+ this.stats.incNodes(-1);
+ }
+ StringId msg;
+ if (crashed && ! isCloseInProgress()) {
+ msg = LocalizedStrings.DistributionManager_MEMBER_AT_0_UNEXPECTEDLY_LEFT_THE_DISTRIBUTED_CACHE_1;
+ addMemberEvent(new MemberCrashedEvent(theId, reason));
+ } else {
+ msg = LocalizedStrings.DistributionManager_MEMBER_AT_0_GRACEFULLY_LEFT_THE_DISTRIBUTED_CACHE_1;
+ addMemberEvent(new MemberDepartedEvent(theId, reason));
+ }
+ logger.info(LocalizedMessage.create(msg, new Object[] {theId, prettifyReason(reason)}));
+
+ // Remove this manager from the serialQueueExecutor.
+ if (this.serialQueuedExecutorPool != null)
+ {
+ serialQueuedExecutorPool.handleMemberDeparture(theId);
+ }
+ }
+ }
+
+ /**
+ * @param reason TODO
+ */
+ public void handleManagerSuspect(InternalDistributedMember suspect,
+ InternalDistributedMember whoSuspected, String reason) {
+ if (!isCurrentMember(suspect)) {
+ return; // fault tolerance
+ }
+
+ int vmType = suspect.getVmKind();
+ if (vmType == ADMIN_ONLY_DM_TYPE) {
+ return;
+ }
+
+ addMemberEvent(new MemberSuspectEvent(suspect, whoSuspected, reason));
+ }
+
+ public void handleViewInstalled(NetView view) {
+ addMemberEvent(new ViewInstalledEvent(view));
+ }
+
+ public void handleQuorumLost(Set failures, List remaining) {
+ addMemberEvent(new QuorumLostEvent(failures, remaining));
+ }
+
+ /**
+ * Sends the shutdown message. Not all DistributionManagers need to
+ * do this.
+ */
+ protected void sendShutdownMessage() {
+ if (getDMType() == ADMIN_ONLY_DM_TYPE && Locator.getLocators().size() == 0) {
+// [bruce] changed above "if" to have ShutdownMessage sent by locators.
+// Otherwise the system can hang because an admin member does not trigger
+// member-left notification unless a new view is received showing the departure.
+// If two locators are simultaneously shut down this may not occur.
+ return;
+ }
+
+ ShutdownMessage m = new ShutdownMessage();
+ InternalDistributedMember theId =
+ this.getDistributionManagerId();
+ m.setDistributionManagerId(theId);
+ Set allOthers = new HashSet(getViewMembers());
+ allOthers.remove(getDistributionManagerId());
+// ReplyProcessor21 rp = new ReplyProcessor21(this, allOthers);
+// m.setProcessorId(rp.getProcessorId());
+// m.setMulticast(system.getConfig().getMcastPort() != 0);
+ m.setRecipients(allOthers);
+
+ //Address recipient = (Address) m.getRecipient();
+ if (logger.isTraceEnabled()) {
+ logger.trace("{} Sending {} to {}", this.getDistributionManagerId(), m, m.getRecipientsDescription());
+ }
+
+ try {
+ //m.resetTimestamp(); // nanotimers across systems don't match
+ long startTime = DistributionStats.getStatTime();
+ channel.send(m.getRecipients(), m, this, stats);
+ this.stats.incSentMessages(1L);
+ if (DistributionStats.enableClockStats) {
+ stats.incSentMessagesTime(DistributionStats.getStatTime()-startTime);
+ }
+ } catch (CancelException e) {
+ logger.debug("CancelException caught sending shutdown: {}", e.getMessage(), e);
+ } catch (Exception ex2) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributionManager_WHILE_SENDING_SHUTDOWN_MESSAGE), ex2);
+ }
+ finally {
+ // Even if the message wasn't sent, *lie* about it, so that
+ // everyone believes that message distribution is done.
+ this.shutdownMsgSent = true;
+ }
+ }
+
+ /**
+ * Returns the executor for the given type of processor.
+ *
+ */
+ public final Executor getExecutor(int processorType, InternalDistributedMember sender) {
+ switch(processorType) {
+ case STANDARD_EXECUTOR:
+ return getThreadPool();
+ case SERIAL_EXECUTOR:
+ return getSerialExecutor(sender);
+ case VIEW_EXECUTOR:
+ return this.viewThread;
+ case HIGH_PRIORITY_EXECUTOR:
+ return getHighPriorityThreadPool();
+ case WAITING_POOL_EXECUTOR:
+ return getWaitingThreadPool();
+ case PARTITIONED_REGION_EXECUTOR:
+ return getPartitionedRegionExcecutor();
+ case REGION_FUNCTION_EXECUTION_EXECUTOR:
+ return getFunctionExcecutor();
+ default:
+ throw new InternalGemFireError(
+ LocalizedStrings.DistributionManager_UNKNOWN_PROCESSOR_TYPE
+ .toLocalizedString(processorType));
+ }
+ }
+
+// /**
+// * Return a shortened name of a class that excludes the package
+// */
+// private static String shortenClassName(String className) {
+// int index = className.lastIndexOf('.');
+// if (index != -1) {
+// return className.substring(index + 1);
+//
+// } else {
+// return className;
+// }
+// }
+
+ /**
+ * Send a message that is guaranteed to be serialized
+ * @param msg
+ * @return the recipients who did not receive the message
+ */
+ protected Set sendOutgoingSerialized(DistributionMessage msg) {
+ try {
+ return sendOutgoing(msg);
+ }
+ catch (NotSerializableException e) {
+ throw new InternalGemFireException(e);
+ }
+ catch (ToDataException e) {
+ // exception from user code
+ throw e;
+ }
+ }
+
+ /**
+ * Actually does the work of sending a message out over the
+ * distribution channel.
+ *
+ * @param message the message to send
+ * @return list of recipients that did not receive the message because
+ * they left the view (null if all received it or it was sent to
+ * {@link DistributionMessage#ALL_RECIPIENTS}.
+ * @throws NotSerializableException
+ * If message
cannot be serialized
+ */
+ protected Set sendOutgoing(DistributionMessage message)
+ throws NotSerializableException {
+ long startTime = DistributionStats.getStatTime();
+
+ Set result = channel.send(message.getRecipients(), message,
+ DistributionManager.this,
+ this.stats);
+ long endTime = 0L;
+ if (DistributionStats.enableClockStats) {
+ endTime = NanoTimer.getTime();
+ }
+ boolean sentToAll = message.forAll();
+
+ if (sentToAll) {
+ stats.incBroadcastMessages(1L);
+ if (DistributionStats.enableClockStats) {
+ stats.incBroadcastMessagesTime(endTime-startTime);
+ }
+ }
+ stats.incSentMessages(1L);
+ if (DistributionStats.enableClockStats) {
+ stats.incSentMessagesTime(endTime-startTime);
+ stats.incDistributeMessageTime(endTime - message.getTimestamp());
+ }
+
+ return result;
+ }
+
+
+
+ /**
+ * @return recipients who did not receive the message
+ * @throws NotSerializableException
+ * If message
cannot be serialized
+ */
+ Set sendMessage(DistributionMessage message)
+ throws NotSerializableException {
+ Set result = null;
+ try {
+ // Verify we're not too far into the shutdown
+ stopper.checkCancelInProgress(null);
+
+ // avoid race condition during startup
+ waitUntilReadyToSendMsgs(message);
+
+ result = sendOutgoing(message);
+ } catch (NotSerializableException ex) {
+ throw ex; // serialization error in user data
+ } catch (ToDataException ex) {
+ throw ex; // serialization error in user data
+ }
+ catch (ReenteredConnectException ex) {
+ throw ex; // Recursively tried to get the same connection
+ }
+ catch (CancelException ex) {
+ throw ex; // bug 37194, shutdown conditions
+ }
+ catch (InvalidDeltaException ide) {
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_CAUGHT_EXCEPTION_WHILE_SENDING_DELTA), ide.getCause());
+ throw (RuntimeException)ide.getCause();
+ }
+ catch (Exception ex) {
+ DistributionManager.this.exceptionInThreads = true;
+ String receiver = "NULL";
+ if (message != null) {
+ receiver = message.getRecipientsDescription();
+ }
+
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributionManager_WHILE_PUSHING_MESSAGE_0_TO_1, new Object[] {message, receiver}), ex);
+ if (message == null || message.forAll())
+ return null;
+ result = new HashSet();
+ for (int i = 0; i < message.getRecipients().length; i ++)
+ result.add(message.getRecipients()[i]);
+ return result;
+ /* if (ex instanceof com.gemstone.gemfire.GemFireIpcResourceException) {
+ return;
+ }*/
+ }
+ return result;
+ }
+
+
+ /**
+ * Schedule a given message appropriately, depending upon its
+ * executor kind.
+ *
+ * @param message
+ */
+ protected void scheduleIncomingMessage(DistributionMessage message)
+ {
+ /* Potential race condition between starting up and getting other
+ * distribution manager ids -- DM will only be initialized upto
+ * the point at which it called startThreads
+ */
+ waitUntilReadyForMessages();
+ message.schedule(DistributionManager.this);
+ }
+
+ /**
+ * Mutex to control access to {@link #waitingForElderChange}
+ * or {@link #elder}.
+ */
+ protected final Object elderMonitor = new Object();
+
+ /**
+ * Must be read/written while holding {@link #elderMonitor}
+ *
+ * @see #elderChangeWait()
+ */
+ private boolean waitingForElderChange = false;
+
+ /**
+ * @see DM#isAdam()
+ */
+ private boolean adam = false;
+
+ /**
+ * This is the "elder" member of the distributed system, responsible
+ * for certain types of arbitration.
+ *
+ * Must hold {@link #elderMonitor} in order to change this.
+ *
+ * @see #getElderId()
+ */
+ protected volatile InternalDistributedMember elder = null;
+
+ public boolean isAdam() {
+ return this.adam;
+ }
+
+ public InternalDistributedMember getElderId()
+ throws DistributedSystemDisconnectedException {
+// membershipManager.waitForEventProcessing();
+ if (closeInProgress) {
+ throw new DistributedSystemDisconnectedException(LocalizedStrings.DistributionManager_NO_VALID_ELDER_WHEN_SYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), this.getRootCause());
+ }
+ getSystem().getCancelCriterion().checkCancelInProgress(null);
+
+ // Cache a recent value of the elder
+ InternalDistributedMember result = elder;
+ if (result != null && membershipManager.memberExists(result)) {
+ return result;
+ }
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_ELDER__0__IS_NOT_CURRENTLY_AN_ACTIVE_MEMBER_SELECTING_NEW_ELDER, elder));
+
+ selectElder(); // ShutdownException can be thrown here
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_NEWLY_SELECTED_ELDER_IS_NOW__0_, elder));
+ return elder;
+ }
+
+ public boolean isElder() {
+ return getId().equals(elder);
+ }
+ public boolean isLoner() {
+ return false;
+ }
+
+ private final StoppableReentrantLock elderLock;
+ private ElderState elderState;
+ private volatile boolean elderStateInitialized;
+
+ public ElderState getElderState(boolean force, boolean useTryLock) {
+ if (force) {
+ if (logger.isDebugEnabled()) {
+ if (!this.myid.equals(this.elder)) {
+ logger.debug("Forcing myself, {}, to be the elder.", this.myid);
+ }
+ }
+ changeElder(this.myid);
+ }
+ if (force || this.myid.equals(elder)) {
+ // we are the elder
+ if (this.elderStateInitialized) {
+ return this.elderState;
+ }
+ return getElderStateWithTryLock(useTryLock);
+ } else {
+ // we are not the elder so return null
+ return null;
+ }
+ }
+
+ /**
+ * Usage: GrantorRequestProcessor calls getElderState with useTryLock set
+ * to true if the becomeGrantor Collaboration is already acquired.
+ *
+ * This tryLock is attempted and if it fails, an exception is thrown to
+ * cause a Doug Lea style back-off (p. 149). It throws an exception because
+ * it needs to back down a couple of packages and I didn't want to couple
+ * this pkg too tightly with the dlock pkg.
+ *
+ * GrantorRequestProcessor catches the exception, releases and reacquires
+ * the Collaboration, and then comes back here to attempt the tryLock
+ * again. Currently nothing will stop it from re-attempting forever. It
+ * has to get the ElderState and cannot give up, but it can free up the
+ * Collaboration and then re-enter it. The other thread holding the
+ * elder lock will hold it only briefly. I've added a volatile called
+ * elderStateInitialized which should cause this back-off to occur only
+ * once in the life of a vm... once the elder, always the elder.
+ *
+ * TODO: Collaboration lock is no longer used. Do we need to to use tryLock?
+ */
+ private ElderState getElderStateWithTryLock(boolean useTryLock) {
+ boolean locked = false;
+ if (useTryLock) {
+ boolean interrupted = Thread.interrupted();
+ try {
+ locked = this.elderLock.tryLock(2000);
+ }
+ catch (InterruptedException e) {
+ interrupted = true;
+ getCancelCriterion().checkCancelInProgress(e);
+ // one last attempt and then allow it to fail for back-off...
+ locked = this.elderLock.tryLock();
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } else {
+ locked = true;
+ this.elderLock.lock();
+ }
+ if (!locked) {
+ // try-lock must have failed
+ throw new IllegalStateException(LocalizedStrings.DistributionManager_POSSIBLE_DEADLOCK_DETECTED.toLocalizedString());
+ }
+ try {
+ if (this.elderState == null) {
+ this.elderState = new ElderState(this);
+ }
+ }
+ finally {
+ this.elderLock.unlock();
+ }
+ this.elderStateInitialized = true;
+// if (Thread.currentThread().isInterrupted())
+// throw new RuntimeException("Interrupted");
+ return this.elderState;
+ }
+
+ /**
+ * Waits until elder if newElder or newElder is no longer a member
+ * @return true if newElder is the elder; false if he is no longer a member
+ * or we are the elder.
+ */
+ public boolean waitForElder(final InternalDistributedMember desiredElder) {
+ MembershipListener l = null;
+ try {
+// Assert.assertTrue(
+// desiredElder.getVmKind() != DistributionManager.ADMIN_ONLY_DM_TYPE);
+ synchronized (this.elderMonitor) {
+ while (true) {
+ if (closeInProgress)
+ return false;
+ InternalDistributedMember currentElder = this.elder;
+// Assert.assertTrue(
+// currentElder.getVmKind() != DistributionManager.ADMIN_ONLY_DM_TYPE);
+ if (desiredElder.equals(currentElder)) {
+ return true;
+ }
+ if (!isCurrentMember(desiredElder)) {
+ return false; // no longer present
+ }
+ if (this.myid.equals(currentElder)) {
+ // Once we become the elder we no longer allow anyone else to be the
+ // elder so don't let them wait anymore.
+ return false;
+ }
+ if (l == null) {
+ l = new MembershipListener() {
+ public void memberJoined(InternalDistributedMember theId) {
+ // nothing needed
+ }
+ public void memberDeparted(InternalDistributedMember theId, boolean crashed) {
+ if (desiredElder.equals(theId)) {
+ notifyElderChangeWaiters();
+ }
+ }
+ public void memberSuspect(InternalDistributedMember id,
+ InternalDistributedMember whoSuspected, String reason) {
+ }
+ public void viewInstalled(NetView view) {
+ }
+ public void quorumLost(Set failures, List remaining) {
+ }
+ };
+ addMembershipListener(l);
+ }
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_CHANGING_ELDER_FROM_0_TO_1,
+ new Object[] {currentElder, desiredElder}));
+ elderChangeWait();
+ } // while true
+ }
+ } finally {
+ if (l != null) {
+ removeMembershipListener(l);
+ }
+ }
+ }
+ /**
+ * Set the elder to newElder and notify anyone waiting for it to change
+ */
+ protected void changeElder(InternalDistributedMember newElder) {
+ synchronized (this.elderMonitor) {
+ if (newElder != null &&
+ this.myid != null && !this.myid.equals(newElder)) {
+ if (this.myid.equals(this.elder)) {
+ // someone else changed the elder while this thread was off cpu
+ if (logger.isDebugEnabled()) {
+ logger.debug("changeElder found this VM to be the elder and is taking an early out");
+ }
+ return;
+ }
+ }
+ this.elder = newElder;
+ if (this.waitingForElderChange) {
+ this.waitingForElderChange = false;
+ this.elderMonitor.notifyAll();
+ }
+ }
+ }
+ /**
+ * Used to wakeup someone in elderChangeWait even though the elder has not changed
+ */
+ protected void notifyElderChangeWaiters() {
+ synchronized (this.elderMonitor) {
+ if (this.waitingForElderChange) {
+ this.waitingForElderChange = false;
+ this.elderMonitor.notifyAll();
+ }
+ }
+ }
+
+ /**
+ * Must be called holding {@link #elderMonitor} lock
+ */
+ private void elderChangeWait() {
+ // This is OK since we're holding the elderMonitor lock, so no
+ // new events will come through until the wait() below.
+ this.waitingForElderChange = true;
+
+ while (this.waitingForElderChange) {
+ stopper.checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ this.elderMonitor.wait();
+ break;
+ }
+ catch (InterruptedException ignore) {
+ interrupted = true;
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } // while
+ }
+
+ /**
+ * getThreadPool gets this distribution manager's message-processing thread
+ * pool
+ */
+ public ExecutorService getThreadPool() {
+ return this.threadPool;
+ }
+
+ /**
+ * Return the high-priority message-processing executor */
+ public ExecutorService getHighPriorityThreadPool() {
+ return this.highPriorityPool;
+ }
+
+ /**
+ * Return the waiting message-processing executor
+ */
+ public ExecutorService getWaitingThreadPool() {
+ return this.waitingPool;
+ }
+
+ /**
+ * Return the waiting message-processing executor
+ */
+ public ExecutorService getPrMetaDataCleanupThreadPool() {
+ return this.prMetaDataCleanupThreadPool;
+ }
+
+ /**
+ * Return the waiting message-processing executor
+ */
+ public Executor getPartitionedRegionExcecutor() {
+ if (this.partitionedRegionThread != null) {
+ return this.partitionedRegionThread;
+ } else {
+ return this.partitionedRegionPool;
+ }
+ }
+
+ /**
+ * Return the waiting message-processing executor
+ */
+ public Executor getFunctionExcecutor() {
+ if (this.functionExecutionThread != null) {
+ return this.functionExecutionThread;
+ } else {
+ return this.functionExecutionPool;
+ }
+ }
+
+ private Executor getSerialExecutor(InternalDistributedMember sender) {
+ if (MULTI_SERIAL_EXECUTORS) {
+ return this.serialQueuedExecutorPool.getThrottledSerialExecutor(sender);
+ } else {
+ return this.serialThread;
+ }
+ }
+
+ /** returns the serialThread's queue if throttling is being used, null if not */
+ public OverflowQueueWithDMStats getSerialQueue(InternalDistributedMember sender) {
+ if (MULTI_SERIAL_EXECUTORS) {
+ return this.serialQueuedExecutorPool.getSerialQueue(sender);
+ } else {
+ return this.serialQueue;
+ }
+ }
+
+ /**
+ * Sets the administration agent associated with this distribution
+ * manager.
+ *
+ * @since 4.0
+ */
+ public void setAgent(RemoteGfManagerAgent agent) {
+ // Don't let the agent be set twice. There should be a one-to-one
+ // correspondence between admin agent and distribution manager.
+ if (agent != null) {
+ if (this.agent != null) {
+ throw new IllegalStateException(LocalizedStrings.DistributionManager_THERE_IS_ALREADY_AN_ADMIN_AGENT_ASSOCIATED_WITH_THIS_DISTRIBUTION_MANAGER.toLocalizedString());
+ }
+
+ } else {
+ if (this.agent == null) {
+ throw new IllegalStateException(LocalizedStrings.DistributionManager_THERE_WAS_NEVER_AN_ADMIN_AGENT_ASSOCIATED_WITH_THIS_DISTRIBUTION_MANAGER.toLocalizedString());
+ }
+ }
+ this.agent = agent;
+ }
+
+ /**
+ * Returns the agent that owns this distribution manager.
+ * (in ConsoleDistributionManager)
+ * @since 3.5
+ */
+ public RemoteGfManagerAgent getAgent(){
+ return this.agent;
+ }
+
+ /**
+ * Returns a description of the distribution configuration used for
+ * this distribution manager. (in ConsoleDistributionManager)
+ *
+ * @return null
if no admin {@linkplain #getAgent
+ * agent} is associated with this distribution manager
+ *
+ * @since 3.5
+ */
+ public String getDistributionConfigDescription() {
+ if (this.agent == null) {
+ return null;
+
+ } else {
+ return this.agent.getTransport().toString();
+ }
+ }
+
+ /**
+ * A DistributionManager
is not intented to be
+ * serialized. This method throws an {@link
+ * UnsupportedOperationException} to prevent a
+ * DistributionManager
from being copy shared.
+ */
+ public void writeExternal(ObjectOutput out) throws IOException {
+ throw new UnsupportedOperationException(LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGERS_SHOULD_NOT_BE_COPY_SHARED.toLocalizedString());
+ }
+
+ /**
+ * A DistributionManager
is not intented to be
+ * serialized. This method throws an {@link
+ * UnsupportedOperationException} to prevent a
+ * DistributionManager
from being copy shared.
+ */
+ public void readExternal(ObjectInput out)
+ throws IOException, ClassNotFoundException {
+ throw new UnsupportedOperationException(LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGERS_SHOULD_NOT_BE_COPY_SHARED.toLocalizedString());
+ }
+
+ /* -----------------------------Health Monitor------------------------------ */
+ private final ConcurrentMap hmMap = new ConcurrentHashMap();
+
+ /**
+ * Returns the health monitor for this distribution manager and owner.
+ * @param owner the agent that owns the returned monitor
+ * @return the health monitor created by the owner; null
+ * if the owner has now created a monitor.
+ * @since 3.5
+ */
+ public HealthMonitor getHealthMonitor(InternalDistributedMember owner) {
+ return (HealthMonitor)this.hmMap.get(owner);
+ }
+ /**
+ * Returns the health monitor for this distribution manager.
+ *
+ * @param owner the agent that owns the created monitor
+ * @param cfg the configuration to use when creating the monitor
+ * @since 3.5
+ */
+ public void createHealthMonitor(InternalDistributedMember owner,
+ GemFireHealthConfig cfg) {
+ if (closeInProgress) {
+ return;
+ }
+ {
+ final HealthMonitor hm = getHealthMonitor(owner);
+ if (hm != null) {
+ hm.stop();
+ this.hmMap.remove(owner);
+ }
+ }
+ {
+ HealthMonitorImpl newHm = new HealthMonitorImpl(owner, cfg, this);
+ newHm.start();
+ this.hmMap.put(owner, newHm);
+ }
+ }
+ /**
+ * Remove a monitor that was previously created.
+ * @param owner the agent that owns the monitor to remove
+ */
+ public void removeHealthMonitor(InternalDistributedMember owner, int theId) {
+ final HealthMonitor hm = getHealthMonitor(owner);
+ if (hm != null && hm.getId() == theId) {
+ hm.stop();
+ this.hmMap.remove(owner);
+ }
+ }
+ public void removeAllHealthMonitors() {
+ Iterator it = this.hmMap.values().iterator();
+ while (it.hasNext()) {
+ HealthMonitor hm = (HealthMonitor)it.next();
+ hm.stop();
+ it.remove();
+ }
+ }
+
+ // For feature request #32887
+ public Set getAdminMemberSet() {
+ return this.adminConsoles;
+ }
+
+ /** Returns count of members filling the specified role */
+ public int getRoleCount(Role role) {
+ int count = 0;
+ Set mbrs = getDistributionManagerIds();
+ for (Iterator mbrIter = mbrs.iterator(); mbrIter.hasNext();) {
+ Set roles = ((InternalDistributedMember) mbrIter.next()).getRoles();
+ for (Iterator rolesIter = roles.iterator(); rolesIter.hasNext();) {
+ Role mbrRole = (Role) rolesIter.next();
+ if (mbrRole.equals(role)) {
+ count++;
+ break;
+ }
+ }
+ }
+ return count;
+ }
+
+ /** Returns true if at least one member is filling the specified role */
+ public boolean isRolePresent(Role role) {
+ Set mbrs = getDistributionManagerIds();
+ for (Iterator mbrIter = mbrs.iterator(); mbrIter.hasNext();) {
+ Set roles = ((InternalDistributedMember) mbrIter.next()).getRoles();
+ for (Iterator rolesIter = roles.iterator(); rolesIter.hasNext();) {
+ Role mbrRole = (Role) rolesIter.next();
+ if (mbrRole.equals(role)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /** Returns a set of all roles currently in the distributed system. */
+ public Set getAllRoles() {
+ Set allRoles = new HashSet();
+ Set mbrs = getDistributionManagerIds();
+ for (Iterator mbrIter = mbrs.iterator(); mbrIter.hasNext();) {
+ Set roles = ((InternalDistributedMember) mbrIter.next()).getRoles();
+ for (Iterator rolesIter = roles.iterator(); rolesIter.hasNext();) {
+ Role mbrRole = (Role) rolesIter.next();
+ allRoles.add(mbrRole);
+ }
+ }
+ return allRoles;
+ }
+
+ /** Returns the membership manager for this distributed system.
+ The membership manager owns the membership set and handles
+ all communications. The manager should NOT be used to
+ bypass DistributionManager to send or receive messages.
+ This method was added to allow hydra to obtain thread-local
+ data for transport from one thread to another. */
+ public MembershipManager getMembershipManager() {
+ // NOTE: do not add cancellation checks here. This method is
+ // used during auto-reconnect after the DS has been closed
+ return membershipManager;
+ }
+
+
+ ////////////////////// Inner Classes //////////////////////
+
+
+ /**
+ * This class is used for DM's multi serial executor.
+ * The serial messages are managed/executed by multiple serial thread.
+ * This class takes care of executing messages related to a sender
+ * using the same thread.
+ */
+ static private class SerialQueuedExecutorPool {
+ /** To store the serial threads */
+ ConcurrentMap serialQueuedExecutorMap = new ConcurrentHashMap(MAX_SERIAL_QUEUE_THREAD);
+
+ /** To store the queue associated with thread */
+ Map serialQueuedMap = new HashMap(MAX_SERIAL_QUEUE_THREAD);
+
+ /** Holds mapping between sender to the serial thread-id */
+ Map senderToSerialQueueIdMap = new HashMap();
+
+ /** Holds info about unused thread, a thread is marked unused when the
+ * member associated with it has left distribution system.
+ */
+ ArrayList threadMarkedForUse = new ArrayList();
+
+ DistributionStats stats;
+ ThreadGroup threadGroup;
+
+ final boolean throttlingDisabled;
+
+ /**
+ * Constructor.
+ * @param group thread group to which the threads will belog to.
+ * @param stats
+ */
+ SerialQueuedExecutorPool(ThreadGroup group, DistributionStats stats, boolean throttlingDisabled) {
+ this.threadGroup = group;
+ this.stats = stats;
+ this.throttlingDisabled = throttlingDisabled;
+ }
+
+ /*
+ * Returns an id of the thread in serialQueuedExecutorMap, thats mapped to the
+ * given seder.
+ *
+ * @param sender
+ * @param createNew boolean flag to indicate whether to create a new id, if id
+ * doesnot exists.
+ */
+ private Integer getQueueId(InternalDistributedMember sender, boolean createNew) {
+ // Create a new Id.
+ Integer queueId;
+
+ synchronized (senderToSerialQueueIdMap)
+ {
+ // Check if there is a executor associated with this sender.
+ queueId = (Integer)senderToSerialQueueIdMap.get(sender);
+
+ if (!createNew || queueId != null){
+ return queueId;
+ }
+
+ // Create new.
+ // Check if any threads are availabe that is marked for Use.
+ if (!threadMarkedForUse.isEmpty()){
+ queueId = (Integer)threadMarkedForUse.remove(0);
+ }
+ // If Map is full, use the threads in round-robin fashion.
+ if (queueId == null){
+ queueId = Integer.valueOf((serialQueuedExecutorMap.size() + 1) % MAX_SERIAL_QUEUE_THREAD);
+ }
+ senderToSerialQueueIdMap.put(sender, queueId);
+ }
+ return queueId;
+ }
+
+ /*
+ * Returns the queue associated with this sender.
+ * Used in FlowControl for throttling (based on queue size).
+ */
+ public OverflowQueueWithDMStats getSerialQueue(InternalDistributedMember sender) {
+ Integer queueId = getQueueId(sender, false);
+ if (queueId == null){
+ return null;
+ }
+ return (OverflowQueueWithDMStats)serialQueuedMap.get(queueId);
+ }
+
+ /*
+ * Returns the serial queue executor, before returning the thread this
+ * applies throttling, based on the total serial queue size (total - sum
+ * of all the serial queue size).
+ * The throttling is applied during put event, this doesnt block the extract
+ * operation on the queue.
+ *
+ */
+ public SerialQueuedExecutorWithDMStats getThrottledSerialExecutor(InternalDistributedMember sender) {
+ SerialQueuedExecutorWithDMStats executor = getSerialExecutor(sender);
+
+ // Get the total serial queue size.
+ int totalSerialQueueMemSize = stats.getSerialQueueBytes();
+
+ // for tcp socket reader threads, this code throttles the thread
+ // to keep the sender-side from overwhelming the receiver.
+ // UDP readers are throttled in the FC protocol, which queries
+ // the queue to see if it should throttle
+ if (stats.getSerialQueueBytes() > TOTAL_SERIAL_QUEUE_THROTTLE &&
+ !DistributionMessage.isPreciousThread())
+ {
+ do {
+ boolean interrupted = Thread.interrupted();
+ try {
+ float throttlePercent = (float)(totalSerialQueueMemSize - TOTAL_SERIAL_QUEUE_THROTTLE) / (float)(TOTAL_SERIAL_QUEUE_BYTE_LIMIT - TOTAL_SERIAL_QUEUE_THROTTLE);
+ int sleep = (int)(100.0 * throttlePercent);
+ sleep = Math.max(sleep, 1);
+ Thread.sleep(sleep);
+ } catch (InterruptedException ex) {
+ interrupted = true;
+ // FIXME-InterruptedException
+ // Perhaps we should return null here?
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ this.stats.getSerialQueueHelper().incThrottleCount();
+ } while (stats.getSerialQueueBytes() >= TOTAL_SERIAL_QUEUE_BYTE_LIMIT);
+ }
+ return executor;
+ }
+
+ /*
+ * Returns the serial queue executor for the given sender.
+ */
+ public SerialQueuedExecutorWithDMStats getSerialExecutor(InternalDistributedMember sender) {
+ SerialQueuedExecutorWithDMStats executor = null;
+ Integer queueId = getQueueId(sender, true);
+ if ((executor = (SerialQueuedExecutorWithDMStats)serialQueuedExecutorMap.get(queueId)) != null){
+ return executor;
+ }
+ // If executor doesn't exists for this sender, create one.
+ executor = createSerialExecutor(queueId);
+
+ serialQueuedExecutorMap.put(queueId, executor);
+
+ if (logger.isDebugEnabled()){
+ logger.debug("Created Serial Queued Executor With queueId {}. Total number of live Serial Threads :{}",
+ queueId, serialQueuedExecutorMap.size());
+ }
+ stats.incSerialPooledThread();
+ return executor;
+ }
+
+ /*
+ * Creates a serial queue executor.
+ */
+ private SerialQueuedExecutorWithDMStats createSerialExecutor(final Integer id) {
+
+ BlockingQueue poolQueue;
+
+ if (SERIAL_QUEUE_BYTE_LIMIT == 0 || this.throttlingDisabled) {
+ poolQueue = new OverflowQueueWithDMStats(stats.getSerialQueueHelper());
+ } else {
+ poolQueue = new ThrottlingMemLinkedQueueWithDMStats(SERIAL_QUEUE_BYTE_LIMIT, SERIAL_QUEUE_THROTTLE, SERIAL_QUEUE_SIZE_LIMIT, SERIAL_QUEUE_SIZE_THROTTLE, this.stats.getSerialQueueHelper());
+ }
+
+ serialQueuedMap.put(id, poolQueue);
+
+ ThreadFactory tf = new ThreadFactory() {
+ public Thread newThread(final Runnable command) {
+ SerialQueuedExecutorPool.this.stats.incSerialPooledThreadStarts();
+ final Runnable r = new Runnable() {
+ public void run() {
+ ConnectionTable.threadWantsSharedResources();
+ Connection.makeReaderThread();
+ try {
+ command.run();
+ } finally {
+ ConnectionTable.releaseThreadsSockets();
+ }
+ }
+ };
+
+ Thread thread = new Thread(threadGroup, r, "Pooled Serial Message Processor " + id);
+ thread.setDaemon(true);
+ return thread;
+ }
+ };
+ return new SerialQueuedExecutorWithDMStats(poolQueue, this.stats.getSerialPooledProcessorHelper(), tf);
+ }
+
+ /*
+ * Does cleanup relating to this member. And marks the serial executor associated
+ * with this member for re-use.
+ */
+ public void handleMemberDeparture(InternalDistributedMember member)
+ {
+ Integer queueId = getQueueId(member, false);
+ if (queueId == null){
+ return;
+ }
+
+ boolean isUsed = false;
+
+ synchronized (senderToSerialQueueIdMap)
+ {
+ senderToSerialQueueIdMap.remove(member);
+
+ // Check if any other members are using the same executor.
+ for (Iterator iter = senderToSerialQueueIdMap.values().iterator(); iter.hasNext();) {
+ Integer value = (Integer)iter.next();
+ if (value.equals(queueId))
+ {
+ isUsed = true;
+ break;
+ }
+ }
+
+ // If not used mark this as unused.
+ if (!isUsed)
+ {
+ if (logger.isInfoEnabled(LogMarker.DM))
+ logger.info(LogMarker.DM, LocalizedMessage.create(
+ LocalizedStrings.DistributionManager_MARKING_THE_SERIALQUEUEDEXECUTOR_WITH_ID__0__USED_BY_THE_MEMBER__1__TO_BE_UNUSED,
+ new Object[] {queueId, member}));
+
+ threadMarkedForUse.add(queueId);
+ }
+ }
+ }
+
+ public void awaitTermination(long time, TimeUnit unit) throws InterruptedException {
+ long timeNanos = unit.toNanos(time);
+ long remainingNanos = timeNanos;
+ long start = System.nanoTime();
+ for (Iterator iter = serialQueuedExecutorMap.values().iterator(); iter.hasNext();) {
+ ExecutorService executor = (ExecutorService)iter.next();
+ executor.awaitTermination(remainingNanos, TimeUnit.NANOSECONDS);
+ remainingNanos = timeNanos = (System.nanoTime() - start);
+ if(remainingNanos <= 0) {
+ return;
+ }
+ }
+ }
+
+ protected void shutdown(){
+ for (Iterator iter = serialQueuedExecutorMap.values().iterator(); iter.hasNext();) {
+ ExecutorService executor = (ExecutorService)iter.next();
+ executor.shutdown();
+ }
+ }
+ }
+
+ /**
+ * A simple class used for locking the list of members of the
+ * distributed system. We give this lock its own class so that it
+ * shows up nicely in stack traces.
+ */
+ private static final class MembersLock {
+ protected MembersLock() {
+
+ }
+ }
+
+ /**
+ * A simple class used for locking the list of membership listeners.
+ * We give this lock its own class so that it shows up nicely in
+ * stack traces.
+ */
+ private static final class MembershipListenersLock {
+ protected MembershipListenersLock() {
+ }
+ }
+
+ /**
+ * This is the listener implementation for responding from events from
+ * the Membership Manager.
+ *
+ */
+ private final class MyListener implements DistributedMembershipListener {
+ DistributionManager dm;
+
+ public MyListener(DistributionManager dm) {
+ this.dm = dm;
+ }
+
+ public boolean isShutdownMsgSent() {
+ return shutdownMsgSent;
+ }
+
+ public void membershipFailure(String reason, Throwable t) {
+ exceptionInThreads = true;
+ DistributionManager.this.rootCause = t;
+ getSystem().disconnect(reason, t, true);
+ }
+
+ public void messageReceived(DistributionMessage message) {
+ handleIncomingDMsg(message);
+ }
+
+ public void newMemberConnected(InternalDistributedMember member) {
+ // Do not elect the elder here as surprise members invoke this callback
+ // without holding the view lock. That can cause a race condition and
+ // subsequent deadlock (#45566). Elder selection is now done when a view
+ // is installed.
+ dm.addNewMember(member);
+ }
+
+ public void memberDeparted(InternalDistributedMember theId, boolean crashed, String reason) {
+ boolean wasAdmin = getAdminMemberSet().contains(theId);
+ if (wasAdmin) {
+ // Pretend we received an AdminConsoleDisconnectMessage from the console that
+ // is no longer in the JavaGroup view.
+ // He must have died without sending a ShutdownMessage.
+ // This fixes bug 28454.
+ AdminConsoleDisconnectMessage message = new AdminConsoleDisconnectMessage();
+ message.setSender(theId);
+ message.setCrashed(crashed);
+ message.setAlertListenerExpected(true);
+ message.setIgnoreAlertListenerRemovalFailure(true); // we don't know if it was a listener so don't issue a warning
+ message.setRecipient(myid);
+ message.setReason(reason); //added for #37950
+ handleIncomingDMsg(message);
+ }
+ dm.handleManagerDeparture(theId, crashed, reason);
+ }
+
+ public void memberSuspect(InternalDistributedMember suspect, InternalDistributedMember whoSuspected, String reason) {
+ dm.handleManagerSuspect(suspect, whoSuspected, reason);
+ }
+
+ public void viewInstalled(NetView view) {
+ processElderSelection();
+ dm.handleViewInstalled(view);
+ }
+
+ /** this is invoked when quorum is being lost, before the view has been installed */
+ public void quorumLost(Set failures, List remaining) {
+ dm.handleQuorumLost(failures, remaining);
+ }
+
+ public DistributionManager getDM()
+ {
+ return dm;
+ }
+
+ private void processElderSelection() {
+ // If we currently had no elder, this member might be the elder;
+ // go through the selection process and decide now.
+ try {
+ dm.selectElder();
+ }
+ catch (DistributedSystemDisconnectedException e) {
+ // ignore
+ }
+ }
+ }
+
+
+ private static abstract class MemberEvent {
+
+ private InternalDistributedMember id;
+ MemberEvent(InternalDistributedMember id) {
+ this.id = id;
+ }
+ public InternalDistributedMember getId() {
+ return this.id;
+ }
+
+ public void handleEvent(DistributionManager manager) {
+ handleEvent(manager, manager.membershipListeners.keySet());
+ handleEvent(manager, manager.allMembershipListeners);
+ }
+
+ protected abstract void handleEvent(MembershipListener listener);
+
+ protected void handleEvent(DistributionManager manager, Set membershipListeners) {
+ for (MembershipListener listener : membershipListeners) {
+ try {
+ handleEvent(listener);
+ } catch (CancelException e) {
+ if (manager.isCloseInProgress()) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("MemberEventInvoker: cancelled");
+ }
+ }
+ else {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_UNEXPECTED_CANCELLATION), e);
+ }
+ break;
+ } catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ } catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributionManager_EXCEPTION_WHILE_CALLING_MEMBERSHIP_LISTENER_FOR_EVENT__0, this), t);
+ }
+ }
+ }
+}
+
+ /**
+ * This is an event reflecting that a InternalDistributedMember has joined
+ * the system.
+ *
+ *
+ */
+ private static final class MemberJoinedEvent extends MemberEvent {
+ MemberJoinedEvent(InternalDistributedMember id) {
+ super(id);
+ }
+ @Override
+ public String toString() {
+ return "member " + getId() + " joined";
+ }
+ @Override
+ protected void handleEvent(MembershipListener listener) {
+ listener.memberJoined(getId());
+ }
+ }
+
+ /**
+ * This is an event reflecting that a InternalDistributedMember has left the system.
+ *
+ */
+ private static final class MemberDepartedEvent extends MemberEvent {
+ String reason;
+
+ MemberDepartedEvent(InternalDistributedMember id, String r) {
+ super(id);
+ reason = r;
+ }
+ @Override
+ public String toString() {
+ return "member " + getId() + " departed (" + reason + ")";
+ }
+ @Override
+ protected void handleEvent(MembershipListener listener) {
+ listener.memberDeparted(getId(), false);
+ }
+ }
+
+ /**
+ * This is an event reflecting that a InternalDistributedMember has left the
+ * system in an unexpected way.
+ *
+ *
+ */
+ private static final class MemberCrashedEvent extends MemberEvent {
+ String reason;
+
+ MemberCrashedEvent(InternalDistributedMember id, String r) {
+ super(id);
+ reason = r;
+ }
+ @Override
+ public String toString() {
+ return "member " + getId() + " crashed: " + reason;
+ }
+ @Override
+ protected void handleEvent(MembershipListener listener) {
+ listener.memberDeparted(getId(), true/*crashed*/);
+ }
+ }
+
+ /**
+ * This is an event reflecting that a InternalDistributedMember may be missing
+ * but has not yet left the system.
+ */
+ private static final class MemberSuspectEvent extends MemberEvent {
+ InternalDistributedMember whoSuspected;
+ String reason;
+ MemberSuspectEvent(InternalDistributedMember suspect, InternalDistributedMember whoSuspected, String reason) {
+ super(suspect);
+ this.whoSuspected = whoSuspected;
+ this.reason = reason;
+ }
+ public InternalDistributedMember whoSuspected() {
+ return this.whoSuspected;
+ }
+
+ public String getReason() {
+ return this.reason;
+ }
+
+ @Override
+ public String toString() {
+ return "member " + getId() + " suspected by: " + this.whoSuspected + " reason: " + reason;
+ }
+ @Override
+ protected void handleEvent(MembershipListener listener) {
+ listener.memberSuspect(getId(), whoSuspected(), reason);
+ }
+ }
+
+ private static final class ViewInstalledEvent extends MemberEvent {
+ NetView view;
+ ViewInstalledEvent(NetView view) {
+ super(null);
+ this.view = view;
+ }
+ public long getViewId() {
+ return view.getViewId();
+ }
+ @Override
+ public String toString() {
+ return "view installed: " + this.view;
+ }
+ @Override
+ public void handleEvent(DistributionManager manager) {
+ manager.handleViewInstalledEvent(this);
+ }
+ @Override
+ protected void handleEvent(MembershipListener listener) {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ private static final class QuorumLostEvent extends MemberEvent {
+ Set failures;
+ List remaining;
+
+ QuorumLostEvent(Set failures, List remaining) {
+ super(null);
+ this.failures = failures;
+ this.remaining = remaining;
+ }
+ public Set getFailures() {
+ return this.failures;
+ }
+ public List getRemaining() {
+ return this.remaining;
+ }
+ @Override
+ public String toString() {
+ return "quorum lost. failures=" + failures + "; remaining=" + remaining;
+ }
+ @Override
+ protected void handleEvent(MembershipListener listener) {
+ listener.quorumLost(getFailures(), getRemaining());
+ }
+ }
+
+
+ /* (non-Javadoc)
+ * @see com.gemstone.gemfire.distributed.internal.DM#getRootCause()
+ */
+ public Throwable getRootCause() {
+ return this.rootCause;
+ }
+
+ /* (non-Javadoc)
+ * @see com.gemstone.gemfire.distributed.internal.DM#setRootCause(java.lang.Throwable)
+ */
+ public void setRootCause(Throwable t) {
+ this.rootCause = t;
+ }
+
+ /* (non-Javadoc)
+ * @see com.gemstone.gemfire.distributed.internal.DM#getMembersOnThisHost()
+ * @since gemfire59poc
+ */
+ public Set getMembersInThisZone() {
+ return getMembersInSameZone(getDistributionManagerId());
+ }
+
+ public Set getMembersInSameZone(InternalDistributedMember targetMember) {
+ Set buddyMembers = new HashSet();
+ if(!redundancyZones.isEmpty()) {
+ synchronized(redundancyZones) {
+ String targetZone = redundancyZones.get(targetMember);
+ for(Map.Entry entry : redundancyZones.entrySet()) {
+ if(entry.getValue().equals(targetZone)) {
+ buddyMembers.add(entry.getKey());
+ }
+ }
+ }
+ } else {
+ buddyMembers.add(targetMember);
+ Set targetAddrs = getEquivalents(targetMember.getInetAddress());
+ for (Iterator i = getDistributionManagerIds().iterator(); i.hasNext();) {
+ InternalDistributedMember o = (InternalDistributedMember)i.next();
+ if (SetUtils.intersectsWith(targetAddrs, getEquivalents(o.getInetAddress()))) {
+ buddyMembers.add(o);
+ }
+ }
+ }
+ return buddyMembers;
+ }
+
+ public boolean areInSameZone(InternalDistributedMember member1,
+ InternalDistributedMember member2) {
+
+ if(!redundancyZones.isEmpty()) {
+ String zone1 = redundancyZones.get(member1);
+ String zone2 = redundancyZones.get(member2);
+ return zone1 != null && zone1.equals(zone2);
+ } else {
+ return areOnEquivalentHost(member1, member2);
+ }
+ }
+
+ public void acquireGIIPermitUninterruptibly() {
+ this.parallelGIIs.acquireUninterruptibly();
+ this.stats.incInitialImageRequestsInProgress(1);
+ }
+
+ public void releaseGIIPermit() {
+ this.stats.incInitialImageRequestsInProgress(-1);
+ this.parallelGIIs.release();
+ }
+
+ public void setDistributedSystemId(int distributedSystemId) {
+ if (distributedSystemId != -1) {
+ this.distributedSystemId = distributedSystemId;
+ }
+ }
+
+ public int getDistributedSystemId() {
+ return this.distributedSystemId;
+ }
+
+ /**
+ * this causes all members in the system to log thread dumps
+ * If useNative is true we attempt to use OSProcess native code
+ * for the dumps. This goes to stdout instead of the system.log files.
+ */
+ public void printDistributedSystemStacks(boolean useNative) {
+ printStacks(new HashSet(getDistributionManagerIds()), useNative);
+ }
+
+ /**
+ * this causes the given InternalDistributedMembers to log thread dumps.
+ * If useNative is true we attempt to use OSProcess native code
+ * for the dumps. This goes to stdout instead of the system.log files.
+ */
+ public void printStacks(Collection ids, boolean useNative) {
+ Set requiresMessage = new HashSet();
+ if (ids.contains(myid)) {
+ OSProcess.printStacks(0, useNative);
+ }
+ if (useNative) {
+ requiresMessage.addAll(ids);
+ ids.remove(myid);
+ } else {
+ for (Iterator it=ids.iterator(); it.hasNext(); ) {
+ InternalDistributedMember mbr = (InternalDistributedMember)it.next();
+ if (mbr.getProcessId() > 0 && mbr.getInetAddress().equals(this.myid.getInetAddress())) {
+ if (!mbr.equals(myid)) {
+ if (!OSProcess.printStacks(mbr.getProcessId(), false)) {
+ requiresMessage.add(mbr);
+ }
+ }
+ } else {
+ requiresMessage.add(mbr);
+ }
+ }
+ }
+ if (requiresMessage.size() > 0) {
+ HighPriorityAckedMessage msg = new HighPriorityAckedMessage();
+ msg.dumpStacks(requiresMessage, useNative, false);
+ }
+ }
+
+ public Set getGroupMembers(String group) {
+ HashSet result = null;
+ for (DistributedMember m: (Set)getDistributionManagerIdsIncludingAdmin()) {
+ if (m.getGroups().contains(group)) {
+ if (result == null) {
+ result = new HashSet();
+ }
+ result.add(m);
+ }
+ }
+ if (result == null) {
+ return Collections.emptySet();
+ } else {
+ return result;
+ }
+ }
+
+ @Override
+ public Set getNormalDistributionManagerIds() {
+ // access to members synchronized under membersLock in order to
+ // ensure serialization
+ synchronized (this.membersLock) {
+ HashSet result = new HashSet();
+ for (InternalDistributedMember m: this.members.keySet()) {
+ if (m.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
+ result.add(m);
+ }
+ }
+ return result;
+ }
+ }
+
+ public Set getLocatorDistributionManagerIds() {
+ // access to members synchronized under membersLock in order to
+ // ensure serialization
+ synchronized (this.membersLock) {
+ HashSet result = new HashSet();
+ for (InternalDistributedMember m: this.members.keySet()) {
+ if (m.getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+ result.add(m);
+ }
+ }
+ return result;
+ }
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
new file mode 100755
index 000000000000..b3e2f2d554ee
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -0,0 +1,2654 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal.membership.gms.mgr;
+
+import java.io.IOException;
+import java.io.NotSerializableException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.ForcedDisconnectException;
+import com.gemstone.gemfire.GemFireConfigException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.SystemConnectException;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.ToDataException;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.AdminMessageType;
+import com.gemstone.gemfire.distributed.internal.DMStats;
+import com.gemstone.gemfire.distributed.internal.DSClock;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionException;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionMessage;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.OverflowQueueWithDMStats;
+import com.gemstone.gemfire.distributed.internal.SizeableRunnable;
+import com.gemstone.gemfire.distributed.internal.StartupMessage;
+import com.gemstone.gemfire.distributed.internal.direct.DirectChannel;
+import com.gemstone.gemfire.distributed.internal.direct.DirectChannelListener;
+import com.gemstone.gemfire.distributed.internal.direct.ShunnedMemberException;
+import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
+import com.gemstone.gemfire.distributed.internal.membership.MembershipTestHook;
+import com.gemstone.gemfire.distributed.internal.membership.NetView;
+import com.gemstone.gemfire.distributed.internal.membership.QuorumChecker;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
+import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
+import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
+import com.gemstone.gemfire.distributed.internal.membership.gms.fd.GMSHealthMonitor;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.GMSQuorumChecker;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.SystemTimer;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.log4j.AlertAppender;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.shared.StringPrintWriter;
+import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
+import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
+import com.gemstone.gemfire.internal.util.Breadcrumbs;
+
+public class GMSMembershipManager implements MembershipManager, Manager
+{
+ private static final Logger logger = Services.getLogger();
+
+ /** product version to use for multicast serialization */
+ volatile boolean disableMulticastForRollingUpgrade;
+
+ /**
+ * set to true if the distributed system that created this manager was
+ * auto-reconnecting when it was created.
+ */
+ boolean wasReconnectingSystem;
+
+ /**
+ * A quorum checker is created during reconnect and is held
+ * here so it is available to the UDP protocol for passing off
+ * the ping-pong responses used in the quorum-checking algorithm.
+ */
+ private volatile QuorumChecker quorumChecker;
+
+ /**
+ * thread-local used to force use of Messenger for communications, usually to
+ * avoid deadlock when conserve-sockets=true. Use of this should be removed
+ * when connection pools are implemented in the direct-channel
+ */
+ private ThreadLocal forceUseUDPMessaging = new ThreadLocal();
+
+ /**
+ * Trick class to make the startup synch more
+ * visible in stack traces
+ *
+ * @see GMSMembershipManager#startupLock
+ */
+ static class EventProcessingLock {
+ public EventProcessingLock() {
+ }
+ }
+
+ static class StartupEvent {
+ static final int SURPRISE_CONNECT = 1;
+ static final int VIEW = 2;
+ static final int MESSAGE = 3;
+
+ /**
+ * indicates whether the event is a departure, a surprise connect
+ * (i.e., before the view message arrived), a view, or a regular
+ * message
+ *
+ * @see #SURPRISE_CONNECT
+ * @see #VIEW
+ * @see #MESSAGE
+ */
+ private int kind;
+
+ // Miscellaneous state depending on the kind of event
+ InternalDistributedMember member;
+ boolean crashed;
+ String reason;
+ DistributionMessage dmsg;
+ NetView gmsView;
+
+ @Override
+ public String toString() {
+ StringBuffer sb = new StringBuffer();
+ sb.append("kind=");
+ switch (kind) {
+ case SURPRISE_CONNECT:
+ sb.append("connect; member = <" + member + ">");
+ break;
+ case VIEW:
+ String text = gmsView.toString();
+ sb.append("view <" + text + ">");
+ break;
+ case MESSAGE:
+ sb.append("message <" + dmsg + ">");
+ break;
+ default:
+ sb.append("unknown=<" + kind + ">");
+ break;
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Create a surprise connect event
+ * @param member the member connecting
+ */
+ StartupEvent(final InternalDistributedMember member) {
+ this.kind = SURPRISE_CONNECT;
+ this.member = member;
+ }
+ /**
+ * Indicate if this is a surprise connect event
+ * @return true if this is a connect event
+ */
+ boolean isSurpriseConnect() {
+ return this.kind == SURPRISE_CONNECT;
+ }
+
+ /**
+ * Create a view event
+ * @param v the new view
+ */
+ StartupEvent(NetView v) {
+ this.kind = VIEW;
+ this.gmsView = v;
+ }
+
+ /**
+ * Indicate if this is a view event
+ * @return true if this is a view event
+ */
+ boolean isGmsView() {
+ return this.kind == VIEW;
+ }
+
+ /**
+ * Create a message event
+ * @param d the message
+ */
+ StartupEvent(DistributionMessage d) {
+ this.kind = MESSAGE;
+ this.dmsg = d;
+ }
+ /**
+ * Indicate if this is a message event
+ * @return true if this is a message event
+ */
+ boolean isDistributionMessage() {
+ return this.kind == MESSAGE;
+ }
+ }
+
+ private int membershipCheckTimeout = DistributionConfig.DEFAULT_SECURITY_PEER_VERIFYMEMBER_TIMEOUT;
+
+ /**
+ * This object synchronizes threads waiting for
+ * startup to finish. Updates to {@link #startupMessages}
+ * are synchronized through this object.
+ */
+ protected final EventProcessingLock startupLock = new EventProcessingLock();
+
+ /**
+ * This is the latest view (ordered list of DistributedMembers)
+ * that has been installed
+ *
+ * All accesses to this object are protected via {@link #latestViewLock}
+ */
+ protected NetView latestView = new NetView();
+
+ /**
+ * This is the lock for protecting access to latestView
+ *
+ * @see #latestView
+ */
+ protected ReadWriteLock latestViewLock = new ReentrantReadWriteLock();
+
+ /**
+ * This is the listener that accepts our membership events
+ */
+ protected com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener listener;
+
+ /**
+ * Membership failure listeners - for testing
+ */
+ List membershipTestHooks;
+
+ /**
+ * This is a representation of the local member (ourself)
+ */
+ protected InternalDistributedMember address = null; // new DistributedMember(-1);
+
+ protected DirectChannel directChannel;
+
+ protected MyDCReceiver dcReceiver;
+
+ volatile boolean isJoining;
+
+ /** have we joined successfully? */
+ volatile boolean hasJoined;
+
+ /**
+ * Members of the distributed system that we believe have shut down.
+ * Keys are instances of {@link InternalDistributedMember}, values are
+ * Longs indicating the time this member was shunned.
+ *
+ * Members are removed after {@link #SHUNNED_SUNSET} seconds have
+ * passed.
+ *
+ * Accesses to this list needs to be under the read or write lock of {@link #latestViewLock}
+ *
+ * @see System#currentTimeMillis()
+ */
+// protected final Set shunnedMembers = Collections.synchronizedSet(new HashSet());
+ protected final Map shunnedMembers = new ConcurrentHashMap();
+
+ /**
+ * Members that have sent a shutdown message. This is used to suppress
+ * suspect processing that otherwise becomes pretty aggressive
+ * when a member is shutting down.
+ */
+ private final Map shutdownMembers = new BoundedLinkedHashMap(1000);
+
+ /**
+ * per bug 39552, keep a list of members that have been shunned and for
+ * which a message is printed. Contents of this list are cleared at the
+ * same time they are removed from {@link #shunnedMembers}.
+ *
+ * Accesses to this list needs to be under the read or write lock of {@link #latestViewLock}
+ */
+ protected final HashSet shunnedAndWarnedMembers = new HashSet();
+ /**
+ * The identities and birth-times of others that we have allowed into
+ * membership at the distributed system level, but have not yet appeared
+ * in a view.
+ *
+ * Keys are instances of {@link InternalDistributedMember}, values are
+ * Longs indicating the time this member was shunned.
+ *
+ * Members are removed when a view containing them is processed. If,
+ * after {@link #surpriseMemberTimeout} milliseconds have passed, a view
+ * containing the member has not arrived, the member is removed from
+ * membership and member-left notification is performed.
+ *
>
+ * Accesses to this list needs to be under the read or write lock of {@link #latestViewLock}
+ *
+ * @see System#currentTimeMillis()
+ */
+ protected final Map surpriseMembers = new ConcurrentHashMap();
+
+ /**
+ * the timeout interval for surprise members. This is calculated from
+ * the member-timeout setting
+ */
+ protected int surpriseMemberTimeout;
+
+ /**
+ * javagroups can skip views and omit telling us about a crashed member.
+ * This map holds a history of suspected members that we use to detect
+ * crashes.
+ */
+ private final Map suspectedMembers = new ConcurrentHashMap();
+
+ /**
+ * the timeout interval for suspected members
+ */
+ private final long suspectMemberTimeout = 180000;
+
+ /**
+ * Length of time, in seconds, that a member is retained in the zombie set
+ *
+ * @see #shunnedMembers
+ */
+ static private final int SHUNNED_SUNSET = Integer.getInteger(
+ "gemfire.shunned-member-timeout", 300).intValue();
+
+ /**
+ * Set to true when the service should stop.
+ */
+ protected volatile boolean shutdownInProgress = false;
+
+ /**
+ * Set to true when upcalls should be generated for
+ * events.
+ */
+ protected volatile boolean processingEvents = false;
+
+ /**
+ * This is the latest viewId installed
+ */
+ long latestViewId = -1;
+
+ /** distribution manager statistics */
+ DMStats stats;
+
+ /**
+ A list of messages received during channel startup that couldn't be processed yet.
+ Additions or removals of this list must be synchronized
+ via {@link #startupLock}.
+ @since 5.0
+ */
+ protected LinkedList startupMessages = new LinkedList();
+
+ /**
+ * ARB: the map of latches is used to block peer handshakes till
+ * authentication is confirmed.
+ */
+ final private HashMap memberLatch = new HashMap();
+
+ /**
+ * Insert our own MessageReceiver between us and the direct channel, in order
+ * to correctly filter membership events.
+ *
+ *
+ */
+ class MyDCReceiver implements DirectChannelListener
+ {
+
+ DirectChannelListener upCall;
+
+ /**
+ * Don't provide events until the caller has told us we are ready.
+ *
+ * Synchronization provided via GroupMembershipService.class.
+ *
+ * Note that in practice we only need to delay accepting the first
+ * client; we don't need to put this check before every call...
+ *
+ */
+ MyDCReceiver(DirectChannelListener up) {
+ upCall = up;
+ }
+
+ public void messageReceived(DistributionMessage msg) {
+ // bug 36851 - notify failure detection that we've had contact from a member
+ services.getHealthMonitor().contactedBy(msg.getSender());
+ handleOrDeferMessage(msg);
+ }
+
+ public DistributionManager getDM() {
+ return upCall.getDM();
+ }
+
+ }
+
+
+ /** if we connect to a locator that has NPD enabled then we enable it in this VM */
+ public void enableNetworkPartitionDetection() {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Network partition detection is being enabled");
+ }
+ this.services.getConfig().getDistributionConfig().setEnableNetworkPartitionDetection(true);
+ this.services.getConfig().setNetworkPartitionDetectionEnabled(true);
+ }
+
+ /**
+ * Analyze a given view object, generate events as appropriate
+ *
+ * @param newView
+ */
+ protected void processView(long newViewId, NetView newView)
+ {
+ // Sanity check...
+ if (logger.isDebugEnabled()) {
+ StringBuffer msg = new StringBuffer(200);
+ msg.append("Membership: Processing view ");
+ msg.append(newView);
+ msg.append("} on " + address.toString());
+ if (!newView.contains(address)) {
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_THE_MEMBER_WITH_ID_0_IS_NO_LONGER_IN_MY_OWN_VIEW_1,
+ new Object[] {address, newView}));
+ }
+ }
+
+// if (newView.getCrashedMembers().size() > 0) {
+// // dump stack for debugging #39827
+// OSProcess.printStacks(0);
+// }
+ // We perform the update under a global lock so that other
+ // incoming events will not be lost in terms of our global view.
+ latestViewLock.writeLock().lock();
+ try {
+ // first determine the version for multicast message serialization
+ Version version = Version.CURRENT;
+ for (Iterator> it=surpriseMembers.entrySet().iterator(); it.hasNext(); ) {
+ InternalDistributedMember mbr = it.next().getKey();
+ Version itsVersion = mbr.getVersionObject();
+ if (itsVersion != null && version.compareTo(itsVersion) < 0) {
+ version = itsVersion;
+ }
+ }
+ for (InternalDistributedMember mbr: newView.getMembers()) {
+ Version itsVersion = mbr.getVersionObject();
+ if (itsVersion != null && itsVersion.compareTo(version) < 0) {
+ version = mbr.getVersionObject();
+ }
+ }
+ disableMulticastForRollingUpgrade = !version.equals(Version.CURRENT);
+
+ if (newViewId < latestViewId) {
+ // ignore this view since it is old news
+ return;
+ }
+
+ // Save previous view, for delta analysis
+ NetView priorView = latestView;
+
+ // update the view to reflect our changes, so that
+ // callbacks will see the new (updated) view.
+ latestViewId = newViewId;
+ latestView = new NetView(newView, newView.getViewId());
+
+ // look for additions
+ for (int i = 0; i < newView.getMembers().size(); i++) { // additions
+ InternalDistributedMember m = (InternalDistributedMember)newView.getMembers().get(i);
+
+ // Once a member has been seen via a view, remove them from the
+ // newborn set
+ boolean wasSurprise = surpriseMembers.remove(m) != null;
+
+ // bug #45155 - membership view processing was slow, causing a member to connect as "surprise"
+ // and the surprise timeout removed the member and shunned it, keeping it from being
+ // recognized as a valid member when it was finally seen in a view
+// if (isShunned(m)) {
+// warnShuns.add(m);
+// continue;
+// }
+
+ // if it's in a view, it's no longer suspect
+ suspectedMembers.remove(m);
+
+ if (priorView.contains(m) || wasSurprise) {
+ continue; // already seen
+ }
+
+ // ARB: unblock any waiters for this particular member.
+ // i.e. signal any waiting threads in tcpconduit.
+ String authInit = this.services.getConfig().getDistributionConfig().getSecurityPeerAuthInit();
+ boolean isSecure = authInit != null && authInit.length() != 0;
+
+ if (isSecure) {
+ CountDownLatch currentLatch;
+ if ((currentLatch = (CountDownLatch)memberLatch.get(m)) != null) {
+ currentLatch.countDown();
+ }
+ }
+
+ if (shutdownInProgress()) {
+ addShunnedMember(m);
+ continue; // no additions processed after shutdown begins
+ } else {
+ boolean wasShunned = endShun(m); // bug #45158 - no longer shun a process that is now in view
+ if (wasShunned && logger.isDebugEnabled()) {
+ logger.debug("No longer shunning {} as it is in the current membership view", m);
+ }
+ }
+
+ logger.info(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_MEMBERSHIP_PROCESSING_ADDITION__0_, m));
+
+ try {
+ listener.newMemberConnected(m);
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (DistributedSystemDisconnectedException e) {
+ // don't log shutdown exceptions
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_FAULT_WHILE_PROCESSING_VIEW_ADDITION_OF__0, m), t);
+ }
+ } // additions
+
+ // look for departures
+ for (int i = 0; i < priorView.getMembers().size(); i++) { // departures
+ InternalDistributedMember m = (InternalDistributedMember)priorView.getMembers().get(i);
+ if (newView.contains(m)) {
+ continue; // still alive
+ }
+
+ if (surpriseMembers.containsKey(m)) {
+ continue; // member has not yet appeared in a view
+ }
+
+ try {
+ removeWithViewLock(m,
+ newView.getCrashedMembers().contains(m) || suspectedMembers.containsKey(m)
+ , "departed membership view");
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_FAULT_WHILE_PROCESSING_VIEW_REMOVAL_OF__0, m), t);
+ }
+ } // departures
+
+ // expire surprise members, add others to view
+ long oldestAllowed = System.currentTimeMillis() - this.surpriseMemberTimeout;
+ for (Iterator> it=surpriseMembers.entrySet().iterator(); it.hasNext(); ) {
+ Map.Entry entry = it.next();
+ Long birthtime = (Long)entry.getValue();
+ if (birthtime.longValue() < oldestAllowed) {
+ it.remove();
+ InternalDistributedMember m = entry.getKey();
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_EXPIRING_MEMBERSHIP_OF_SURPRISE_MEMBER_0, m));
+ removeWithViewLock(m, true, "not seen in membership view in "
+ + this.surpriseMemberTimeout + "ms");
+ }
+ else {
+ if (!latestView.contains(entry.getKey())) {
+ latestView.add(entry.getKey());
+ }
+ }
+ }
+ // expire suspected members
+ oldestAllowed = System.currentTimeMillis() - this.suspectMemberTimeout;
+ for (Iterator it=suspectedMembers.entrySet().iterator(); it.hasNext(); ) {
+ Map.Entry entry = (Map.Entry)it.next();
+ Long birthtime = (Long)entry.getValue();
+ if (birthtime.longValue() < oldestAllowed) {
+ InternalDistributedMember m = (InternalDistributedMember)entry.getKey();
+ it.remove();
+ }
+ }
+ try {
+ listener.viewInstalled(latestView);
+ startCleanupTimer();
+ }
+ catch (DistributedSystemDisconnectedException se) {
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ /**
+ * the timer used to perform periodic tasks
+ *
+ * Concurrency: protected by {@link #latestViewLock} ReentrantReadWriteLock
+ */
+ private SystemTimer cleanupTimer;
+
+ private Services services;
+
+ private boolean mcastEnabled;
+
+ private boolean tcpDisabled;
+
+
+ @Override
+ public boolean isMulticastAllowed() {
+ return !disableMulticastForRollingUpgrade;
+ }
+
+ /**
+ * Joins the distributed system
+ *
+ * @throws GemFireConfigException - configuration error
+ * @throws SystemConnectException - problem joining
+ */
+ private void join() {
+ services.setShutdownCause(null);
+ services.getCancelCriterion().cancel(null);
+
+ latestViewLock.writeLock().lock();
+ try {
+ try {
+ this.isJoining = true; // added for bug #44373
+
+ // connect
+ long start = System.currentTimeMillis();
+
+ boolean ok = services.getJoinLeave().join();
+
+ if (!ok) {
+ throw new GemFireConfigException("Unable to join the distributed system. " +
+ "Operation either timed out, was stopped or Locator does not exist.");
+ }
+
+ long delta = System.currentTimeMillis() - start;
+
+ logger.info(LogMarker.DISTRIBUTION, LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_JOINED_TOOK__0__MS, delta));
+
+ NetView initialView = services.getJoinLeave().getView();
+ latestView = new NetView(initialView, initialView.getViewId());
+ listener.viewInstalled(latestView);
+
+ } catch (RuntimeException ex) {
+ throw ex;
+ }
+ catch (Exception ex) {
+ if (ex.getCause() != null && ex.getCause().getCause() instanceof SystemConnectException) {
+ throw (SystemConnectException)(ex.getCause().getCause());
+ }
+ throw new DistributionException(LocalizedStrings.GroupMembershipService_AN_EXCEPTION_WAS_THROWN_WHILE_JOINING.toLocalizedString(), ex);
+ }
+ finally {
+ this.isJoining = false;
+ }
+ }
+ finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+
+ public GMSMembershipManager(DistributedMembershipListener listener) {
+ Assert.assertTrue(listener != null);
+ this.listener = listener;
+ }
+
+ @Override
+ public void init(Services services) {
+ this.services = services;
+
+ Assert.assertTrue(services != null);
+
+ this.stats = services.getStatistics();
+ DistributionConfig config = services.getConfig().getDistributionConfig();
+ RemoteTransportConfig transport = services.getConfig().getTransport();
+
+ this.membershipCheckTimeout = config.getSecurityPeerMembershipTimeout();
+ this.wasReconnectingSystem = transport.getIsReconnectingDS();
+
+ // cache these settings for use in send()
+ this.mcastEnabled = transport.isMcastEnabled();
+ this.tcpDisabled = transport.isTcpDisabled();
+
+ if (!this.tcpDisabled) {
+ dcReceiver = new MyDCReceiver(listener);
+ }
+
+ surpriseMemberTimeout = Math.max(20 * DistributionConfig.DEFAULT_MEMBER_TIMEOUT,
+ 20 * config.getMemberTimeout());
+ surpriseMemberTimeout = Integer.getInteger("gemfire.surprise-member-timeout", surpriseMemberTimeout).intValue();
+
+ }
+
+ @Override
+ public void start() {
+ DistributionConfig config = services.getConfig().getDistributionConfig();
+ RemoteTransportConfig transport = services.getConfig().getTransport();
+
+ int dcPort = 0;
+ if (!tcpDisabled) {
+ directChannel = new DirectChannel(this, dcReceiver, config);
+ dcPort = directChannel.getPort();
+ }
+
+
+ services.getMessenger().getMemberID().setDirectChannelPort(dcPort);
+
+ }
+
+
+ @Override
+ public void joinDistributedSystem() {
+ long startTime = System.currentTimeMillis();
+
+ try {
+ join();
+ }
+ catch (RuntimeException e) {
+ if (directChannel != null) {
+ directChannel.disconnect(e);
+ }
+ throw e;
+ }
+
+ this.address = services.getMessenger().getMemberID();
+
+ int dcPort = 0;
+ if (directChannel != null) {
+ dcPort = directChannel.getPort();
+ }
+
+ if (directChannel != null) {
+ directChannel.setLocalAddr(address);
+ }
+
+ this.hasJoined = true;
+
+ // in order to debug startup issues we need to announce the membership
+ // ID as soon as we know it
+ logger.info(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_entered_into_membership_in_group_0_with_id_1,
+ new Object[]{""+(System.currentTimeMillis()-startTime)}));
+
+ }
+
+ @Override
+ public void started() {
+ }
+
+
+ /** this is invoked by JoinLeave when there is a loss of quorum in the membership system */
+ public void quorumLost(Collection failures, NetView view) {
+ // notify of quorum loss if split-brain detection is enabled (meaning we'll shut down) or
+ // if the loss is more than one member
+
+ boolean notify = failures.size() > 1;
+ if (!notify) {
+ notify = services.getConfig().isNetworkPartitionDetectionEnabled();
+ }
+
+ if (notify) {
+ List remaining = new ArrayList(view.getMembers());
+ remaining.removeAll(failures);
+
+ if (inhibitForceDisconnectLogging) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Possible loss of quorum ");
+ }
+ }
+ logger.fatal(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_POSSIBLE_LOSS_OF_QUORUM_DETECTED, new Object[] {failures.size(), failures}));
+ if (inhibitForceDisconnectLogging) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Possible loss of quorum ");
+ }
+ }
+
+
+ try {
+ this.listener.quorumLost(new HashSet(failures),
+ remaining);
+ } catch (CancelException e) {
+ // safe to ignore - a forced disconnect probably occurred
+ }
+ }
+ }
+
+
+ @Override
+ public boolean testMulticast() {
+ try {
+ return services.getMessenger().testMulticast(services.getConfig().getMemberTimeout());
+ } catch (InterruptedException e) {
+ services.getCancelCriterion().checkCancelInProgress(e);
+ Thread.currentThread().interrupt();
+ return false;
+ }
+ }
+
+ /**
+ * Remove a member. {@link #latestViewLock} must be held
+ * before this method is called. If member is not already shunned,
+ * the uplevel event handler is invoked.
+ *
+ * @param dm
+ * @param crashed
+ * @param reason
+ */
+ protected void removeWithViewLock(InternalDistributedMember dm,
+ boolean crashed, String reason) {
+ boolean wasShunned = isShunned(dm);
+
+ // Delete resources
+ destroyMember(dm, crashed, reason);
+
+ if (wasShunned) {
+ return; // Explicit deletion, no upcall.
+ }
+
+ try {
+ listener.memberDeparted(dm, crashed, reason);
+ }
+ catch (DistributedSystemDisconnectedException se) {
+ // let's not get huffy about it
+ }
+ }
+
+ /**
+ * Process a surprise connect event, or place it on the startup queue.
+ * @param member the member
+ */
+ protected void handleOrDeferSurpriseConnect(InternalDistributedMember member) {
+ synchronized (startupLock) {
+ if (!processingEvents) {
+ startupMessages.add(new StartupEvent(member));
+ return;
+ }
+ }
+ processSurpriseConnect(member);
+ }
+
+ public void startupMessageFailed(DistributedMember mbr, String failureMessage) {
+ // fix for bug #40666
+ addShunnedMember((InternalDistributedMember)mbr);
+ // fix for bug #41329, hang waiting for replies
+ try {
+ listener.memberDeparted((InternalDistributedMember)mbr, true, "failed to pass startup checks");
+ }
+ catch (DistributedSystemDisconnectedException se) {
+ // let's not get huffy about it
+ }
+ }
+
+
+ /**
+ * Logic for handling a direct connection event (message received
+ * from a member not in the view). Does not employ the
+ * startup queue.
+ *
+ * Must be called with {@link #latestViewLock} held. Waits
+ * until there is a stable view. If the member has already
+ * been added, simply returns; else adds the member.
+ *
+ * @param dm the member joining
+ */
+ public boolean addSurpriseMember(DistributedMember dm) {
+ final InternalDistributedMember member = (InternalDistributedMember)dm;
+ boolean warn = false;
+
+ latestViewLock.writeLock().lock();
+ try {
+ // At this point, the join may have been discovered by
+ // other means.
+ if (latestView.contains(member)) {
+ return true;
+ }
+ if (surpriseMembers.containsKey(member)) {
+ return true;
+ }
+ if (member.getVmViewId() < 0) {
+ logger.warn("adding a surprise member that has not yet joined the distributed system: " + member, new Exception("stack trace"));
+ }
+ if (latestView.getViewId() > member.getVmViewId()) {
+ // tell the process that it should shut down distribution.
+ // Run in a separate thread so we don't hold the view lock during the request. Bug #44995
+ new Thread(Thread.currentThread().getThreadGroup(),
+ "Removing shunned GemFire node " + member) {
+ @Override
+ public void run() {
+ // fix for bug #42548
+ // this is an old member that shouldn't be added
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_Invalid_Surprise_Member, new Object[]{member, latestView}));
+ requestMemberRemoval(member, "this member is no longer in the view but is initiating connections");
+ }
+ }.start();
+ addShunnedMember(member);
+ return false;
+ }
+
+ // Adding him to this set ensures we won't remove him if a new
+ // view comes in and he's still not visible.
+ surpriseMembers.put(member, Long.valueOf(System.currentTimeMillis()));
+
+ if (shutdownInProgress()) {
+ // Force disconnect, esp. the TCPConduit
+ String msg = LocalizedStrings.GroupMembershipService_THIS_DISTRIBUTED_SYSTEM_IS_SHUTTING_DOWN.toLocalizedString();
+ if (directChannel != null) {
+ try {
+ directChannel.closeEndpoint(member, msg);
+ } catch (DistributedSystemDisconnectedException e) {
+ // ignore - happens during shutdown
+ }
+ }
+ destroyMember(member, false, msg); // for good luck
+ return true; // allow during shutdown
+ }
+
+ if (isShunned(member)) {
+ warn = true;
+ surpriseMembers.remove(member);
+ } else {
+
+ // Now that we're sure the member is new, add them.
+ // make sure the surprise-member cleanup task is running
+ if (this.cleanupTimer == null) {
+ startCleanupTimer();
+ } // cleanupTimer == null
+
+ // Ensure that the member is accounted for in the view
+ // Conjure up a new view including the new member. This is necessary
+ // because we are about to tell the listener about a new member, so
+ // the listener should rightfully expect that the member is in our
+ // membership view.
+
+ // However, we put the new member at the end of the list. This
+ // should ensure he's not chosen as an elder.
+ // This will get corrected when he finally shows up in the
+ // view.
+ NetView newMembers = new NetView(latestView, latestView.getViewId());
+ newMembers.add(member);
+ latestView = newMembers;
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ if (warn) { // fix for bug #41538 - deadlock while alerting
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_IGNORING_SURPRISE_CONNECT_FROM_SHUNNED_MEMBER_0, member));
+ } else {
+ listener.newMemberConnected(member);
+ }
+ return !warn;
+ }
+
+
+ /** starts periodic task to perform cleanup chores such as expire surprise members */
+ private void startCleanupTimer() {
+ latestViewLock.writeLock().lock();
+ try {
+ if (this.cleanupTimer != null) {
+ return;
+ }
+ DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
+ if (ds != null && ds.isConnected()) {
+ this.cleanupTimer = new SystemTimer(ds, true);
+ SystemTimer.SystemTimerTask st = new SystemTimer.SystemTimerTask() {
+ @Override
+ public void run2() {
+ latestViewLock.writeLock().lock();
+ try {
+ long oldestAllowed = System.currentTimeMillis() - surpriseMemberTimeout;
+ for (Iterator it=surpriseMembers.entrySet().iterator(); it.hasNext(); ) {
+ Map.Entry entry = (Map.Entry)it.next();
+ Long birthtime = (Long)entry.getValue();
+ if (birthtime.longValue() < oldestAllowed) {
+ it.remove();
+ InternalDistributedMember m = (InternalDistributedMember)entry.getKey();
+ logger.info(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_EXPIRING_MEMBERSHIP_OF_SURPRISE_MEMBER_0, m));
+ removeWithViewLock(m, true, "not seen in membership view in "
+ + surpriseMemberTimeout + "ms");
+ }
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+ };
+ this.cleanupTimer.scheduleAtFixedRate(st, surpriseMemberTimeout, surpriseMemberTimeout/3);
+ } // ds != null && ds.isConnected()
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+ /**
+ * Dispatch the distribution message, or place it on the startup queue.
+ *
+ * @param msg the message to process
+ */
+ protected void handleOrDeferMessage(DistributionMessage msg) {
+ synchronized(startupLock) {
+ if (!processingEvents) {
+ startupMessages.add(new StartupEvent(msg));
+ return;
+ }
+ }
+ dispatchMessage(msg);
+ }
+
+ public void warnShun(DistributedMember m) {
+ latestViewLock.writeLock().lock();
+ try {
+ if (!shunnedMembers.containsKey(m)) {
+ return; // not shunned
+ }
+ if (shunnedAndWarnedMembers.contains(m)) {
+ return; // already warned
+ }
+ shunnedAndWarnedMembers.add(m);
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ // issue warning outside of sync since it may cause messaging and we don't
+ // want to hold the view lock while doing that
+ logger.warn(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_MEMBERSHIP_DISREGARDING_SHUNNED_MEMBER_0, m));
+ }
+
+ @Override
+ public void processMessage(DistributionMessage msg) {
+ handleOrDeferMessage(msg);
+ }
+
+ /**
+ * Logic for processing a distribution message.
+ *
+ * It is possible to receive messages not consistent with our view.
+ * We handle this here, and generate an uplevel event if necessary
+ * @param msg the message
+ */
+ public void dispatchMessage(DistributionMessage msg) {
+ boolean isNew = false;
+ InternalDistributedMember m = msg.getSender();
+ boolean shunned = false;
+
+ // First grab the lock: check the sender against our stabilized view.
+ latestViewLock.writeLock().lock();
+ try {
+ if (isShunned(m)) {
+ if (msg instanceof StartupMessage) {
+ endShun(m);
+ }
+ else {
+ // fix for bug 41538 - sick alert listener causes deadlock
+ // due to view lock being held during messaging
+ shunned = true;
+ }
+ } // isShunned
+
+ if (!shunned) {
+ isNew = !latestView.contains(m) && !surpriseMembers.containsKey(m);
+
+ // If it's a new sender, wait our turn, generate the event
+ if (isNew) {
+ shunned = !addSurpriseMember(m);
+ } // isNew
+ }
+
+ // Latch the view before we unlock
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+
+ if (shunned) { // bug #41538 - shun notification must be outside synchronization to avoid hanging
+ warnShun(m);
+ logger.info("Membership: Ignoring message from shunned member <{}>:{}", m, msg);
+ throw new MemberShunnedException(m);
+ }
+
+ listener.messageReceived(msg);
+ }
+
+ /**
+ * Process a new view object, or place on the startup queue
+ * @param viewArg the new view
+ */
+ protected void handleOrDeferViewEvent(NetView viewArg) {
+ if (this.isJoining) {
+ // bug #44373 - queue all view messages while joining.
+ // This is done under the latestViewLock, but we can't block here because
+ // we're sitting in the UDP reader thread.
+ synchronized(startupLock) {
+ startupMessages.add(new StartupEvent(viewArg));
+ return;
+ }
+ }
+ latestViewLock.writeLock().lock();
+ try {
+ synchronized(startupLock) {
+ if (!processingEvents) {
+ startupMessages.add(new StartupEvent(viewArg));
+ return;
+ }
+ }
+ // view processing can take a while, so we use a separate thread
+ // to avoid blocking a reader thread
+ NetView newView = viewArg;
+ long newId = viewArg.getViewId();
+ LocalViewMessage v = new LocalViewMessage(address, newId, newView,
+ GMSMembershipManager.this);
+
+ listener.messageReceived(v);
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ @Override
+ public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
+ SuspectMember s = new SuspectMember(initiator, suspect, reason);
+ handleOrDeferSuspect(s);
+ }
+
+ /**
+ * Process a new view object, or place on the startup queue
+ * @param suspectInfo the suspectee and suspector
+ */
+ protected void handleOrDeferSuspect(SuspectMember suspectInfo) {
+ latestViewLock.writeLock().lock();
+ try {
+ synchronized(startupLock) {
+ if (!processingEvents) {
+ return;
+ }
+ }
+ InternalDistributedMember suspect = suspectInfo.suspectedMember;
+ InternalDistributedMember who = suspectInfo.whoSuspected;
+ this.suspectedMembers.put(suspect, Long.valueOf(System.currentTimeMillis()));
+ try {
+ listener.memberSuspect(suspect, who, suspectInfo.reason);
+ }
+ catch (DistributedSystemDisconnectedException se) {
+ // let's not get huffy about it
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ /**
+ * Process a potential direct connect. Does not use
+ * the startup queue. It grabs the {@link #latestViewLock}
+ * and then processes the event.
+ *
+ * It is a potential event, because we don't know until we've
+ * grabbed a stable view if this is really a new member.
+ *
+ * @param member
+ */
+ private void processSurpriseConnect(
+ InternalDistributedMember member)
+ {
+ addSurpriseMember(member);
+ }
+
+ /**
+ * Dispatch routine for processing a single startup event
+ * @param o the startup event to handle
+ */
+ private void processStartupEvent(StartupEvent o) {
+ // Most common events first
+
+ if (o.isDistributionMessage()) { // normal message
+ try {
+ dispatchMessage(o.dmsg);
+ }
+ catch (MemberShunnedException e) {
+ // message from non-member - ignore
+ }
+ }
+ else if (o.isGmsView()) { // view event
+ processView(o.gmsView.getViewId(), o.gmsView);
+ }
+ else if (o.isSurpriseConnect()) { // connect
+ processSurpriseConnect(o.member);
+ }
+
+ else // sanity
+ throw new InternalGemFireError(LocalizedStrings.GroupMembershipService_UNKNOWN_STARTUP_EVENT_0.toLocalizedString(o));
+ }
+
+ /**
+ * Special mutex to create a critical section for
+ * {@link #startEventProcessing()}
+ */
+ private final Object startupMutex = new Object();
+
+
+ public void startEventProcessing()
+ {
+ // Only allow one thread to perform the work
+ synchronized (startupMutex) {
+ if (logger.isDebugEnabled())
+ logger.debug("Membership: draining startup events.");
+ // Remove the backqueue of messages, but allow
+ // additional messages to be added.
+ for (;;) {
+ StartupEvent ev;
+ // Only grab the mutex while reading the queue.
+ // Other events may arrive while we're attempting to
+ // drain the queue. This is OK, we'll just keep processing
+ // events here until we've caught up.
+ synchronized (startupLock) {
+ int remaining = startupMessages.size();
+ if (remaining == 0) {
+ // While holding the lock, flip the bit so that
+ // no more events get put into startupMessages, and
+ // notify all waiters to proceed.
+ processingEvents = true;
+ startupLock.notifyAll();
+ break; // ...and we're done.
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: {} remaining startup message(s)", remaining);
+ }
+ ev = (StartupEvent)startupMessages.removeFirst();
+ } // startupLock
+ try {
+ processStartupEvent(ev);
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.warn(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_MEMBERSHIP_ERROR_HANDLING_STARTUP_EVENT), t);
+ }
+
+ } // for
+ if (logger.isDebugEnabled())
+ logger.debug("Membership: finished processing startup events.");
+ } // startupMutex
+ }
+
+
+ public void waitForEventProcessing() throws InterruptedException {
+ // First check outside of a synchronized block. Cheaper and sufficient.
+ if (Thread.interrupted()) throw new InterruptedException();
+ if (processingEvents)
+ return;
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: waiting until the system is ready for events");
+ }
+ for (;;) {
+ directChannel.getCancelCriterion().checkCancelInProgress(null);
+ synchronized (startupLock) {
+ // Now check using a memory fence and synchronization.
+ if (processingEvents)
+ break;
+ boolean interrupted = Thread.interrupted();
+ try {
+ startupLock.wait();
+ }
+ catch (InterruptedException e) {
+ interrupted = true;
+ directChannel.getCancelCriterion().checkCancelInProgress(e);
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } // synchronized
+ } // for
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: continuing");
+ }
+ }
+
+ /**
+ * for testing we need to validate the startup event list
+ */
+ public List getStartupEvents() {
+ return this.startupMessages;
+ }
+
+ public ReadWriteLock getViewLock() {
+ return this.latestViewLock;
+ }
+
+ /**
+ * Returns a copy (possibly not current) of the current
+ * view (a list of {@link DistributedMember}s)
+ */
+ public NetView getView()
+ {
+ // Grab the latest view under a mutex...
+ NetView v;
+
+ latestViewLock.readLock().lock();
+ v = latestView;
+ latestViewLock.readLock().unlock();
+
+ NetView result = new NetView(v, v.getViewId());
+
+ for (InternalDistributedMember m: v.getMembers()) {
+ if (isShunned(m)) {
+ result.remove(m);
+ }
+ }
+
+ return result;
+ }
+
+ /**
+ * test hook
+ * The lead member is the eldest member with partition detection enabled.
+ * If no members have partition detection enabled, there will be no
+ * lead member and this method will return null.
+ * @return the lead member associated with the latest view
+ */
+ public DistributedMember getLeadMember() {
+ latestViewLock.readLock().lock();
+ try {
+ return latestView == null? null : latestView.getLeadMember();
+ } finally {
+ latestViewLock.readLock().unlock();
+ }
+ }
+
+ protected boolean isJoining() {
+ return this.isJoining;
+ }
+
+ /**
+ * test hook
+ * @return the current membership view coordinator
+ */
+ public DistributedMember getCoordinator() {
+ // note - we go straight to JoinLeave because the
+ // DistributionManager queues view changes in a serial executor, where
+ // they're asynchronously installed. The DS may still see the old coordinator
+ latestViewLock.readLock().lock();
+ try {
+ return latestView == null? null : latestView.getCoordinator();
+ } finally {
+ latestViewLock.readLock().unlock();
+ }
+ }
+
+ public boolean memberExists(DistributedMember m) {
+ latestViewLock.readLock().lock();
+ NetView v = latestView;
+ latestViewLock.readLock().unlock();
+ return v.getMembers().contains(m);
+ }
+
+ /**
+ * Returns the identity associated with this member. WARNING: this value will
+ * be returned after the channel is closed, but in that case it is good for
+ * logging purposes only. :-)
+ */
+ public InternalDistributedMember getLocalMember()
+ {
+ return address;
+ }
+
+ public Services getServices() {
+ return services;
+ }
+
+ public void postConnect() {
+ }
+
+ /**
+ * @see SystemFailure#loadEmergencyClasses()
+ /**
+ * break any potential circularity in {@link #loadEmergencyClasses()}
+ */
+ private static volatile boolean emergencyClassesLoaded = false;
+
+ /**
+ * inhibits logging of ForcedDisconnectException to keep dunit logs clean
+ * while testing this feature
+ */
+ protected static volatile boolean inhibitForceDisconnectLogging;
+
+ /**
+ * Ensure that the critical classes from components
+ * get loaded.
+ *
+ * @see SystemFailure#loadEmergencyClasses()
+ */
+ public static void loadEmergencyClasses() {
+ if (emergencyClassesLoaded) return;
+ emergencyClassesLoaded = true;
+ DirectChannel.loadEmergencyClasses();
+ GMSJoinLeave.loadEmergencyClasses();
+ GMSHealthMonitor.loadEmergencyClasses();
+ }
+ /**
+ * Close the receiver, avoiding all potential deadlocks and
+ * eschewing any attempts at being graceful.
+ *
+ * @see SystemFailure#emergencyClose()
+ */
+ public void emergencyClose() {
+ final boolean DEBUG = SystemFailure.TRACE_CLOSE;
+
+ setShutdown();
+
+ // We can't call close() because they will allocate objects. Attempt
+ // a surgical strike and touch the important protocols.
+
+ // MOST important, kill the FD protocols...
+ services.emergencyClose();
+
+ // Close the TCPConduit sockets...
+ if (directChannel != null) {
+ if (DEBUG) {
+ System.err.println("DEBUG: emergency close of DirectChannel");
+ }
+ directChannel.emergencyClose();
+ }
+
+ if (DEBUG) {
+ System.err.println("DEBUG: done closing GroupMembershipService");
+ }
+ }
+
+
+ /**
+ * in order to avoid split-brain occurring when a member is shutting down due to
+ * race conditions in view management we add it as a shutdown member when we receive
+ * a shutdown message. This is not the same as a SHUNNED member.
+ */
+ public void shutdownMessageReceived(InternalDistributedMember id, String reason) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: recording shutdown status of {}", id);
+ }
+ synchronized(this.shutdownMembers) {
+ this.shutdownMembers.put(id, id);
+ services.getHealthMonitor().memberShutdown(id, reason);
+ services.getJoinLeave().memberShutdown(id, reason);
+ }
+ }
+
+ /**
+ * returns true if a shutdown message has been received from the given address but
+ * that member is still in the membership view or is a surprise member.
+ */
+ public boolean isShuttingDown(InternalDistributedMember mbr) {
+ synchronized(shutdownMembers) {
+ return shutdownMembers.containsKey(mbr);
+ }
+ }
+
+
+ public void shutdown() {
+ setShutdown();
+ services.stop();
+ }
+
+ @Override
+ public void stop() {
+
+ // [bruce] Do not null out the channel w/o adding appropriate synchronization
+
+ logger.debug("MembershipManager closing");
+
+ if (directChannel != null) {
+ directChannel.disconnect(null);
+
+ if (address != null) {
+ // Make sure that channel information is consistent
+ // Probably not important in this particular case, but just
+ // to be consistent...
+ latestViewLock.writeLock().lock();
+ try {
+ destroyMember(address, false, "orderly shutdown");
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+ }
+
+ if (cleanupTimer != null) {
+ cleanupTimer.cancel();
+ }
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: channel closed");
+ }
+ }
+
+ public void uncleanShutdown(String reason, final Exception e) {
+ inhibitForcedDisconnectLogging(false);
+
+ if (services.getShutdownCause() == null) {
+ services.setShutdownCause(e);
+ }
+
+ if (this.directChannel != null) {
+ this.directChannel.disconnect(e);
+ }
+
+ // first shut down communication so we don't do any more harm to other
+ // members
+ services.emergencyClose();
+
+ if (e != null) {
+ try {
+ if (membershipTestHooks != null) {
+ List l = membershipTestHooks;
+ for (Iterator it=l.iterator(); it.hasNext(); ) {
+ MembershipTestHook dml = (MembershipTestHook)it.next();
+ dml.beforeMembershipFailure(reason, e);
+ }
+ }
+ listener.membershipFailure(reason, e);
+ if (membershipTestHooks != null) {
+ List l = membershipTestHooks;
+ for (Iterator it=l.iterator(); it.hasNext(); ) {
+ MembershipTestHook dml = (MembershipTestHook)it.next();
+ dml.afterMembershipFailure(reason, e);
+ }
+ }
+ }
+ catch (RuntimeException re) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_EXCEPTION_CAUGHT_WHILE_SHUTTING_DOWN), re);
+ }
+ }
+ }
+
+ /** generate XML for the cache before shutting down due to forced disconnect */
+ public void saveCacheXmlForReconnect() {
+ // there are two versions of this method so it can be unit-tested
+ boolean sharedConfigEnabled = services.getConfig().getDistributionConfig().getUseSharedConfiguration();
+ saveCacheXmlForReconnect(sharedConfigEnabled);
+ }
+
+ /** generate XML from the cache before shutting down due to forced disconnect */
+ public void saveCacheXmlForReconnect(boolean sharedConfigEnabled) {
+ // first save the current cache description so reconnect can rebuild the cache
+ GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+ if (cache != null && (cache instanceof Cache)) {
+ if (!Boolean.getBoolean("gemfire.autoReconnect-useCacheXMLFile")
+ && !cache.isSqlfSystem() && !sharedConfigEnabled) {
+ try {
+ logger.info("generating XML to rebuild the cache after reconnect completes");
+ StringPrintWriter pw = new StringPrintWriter();
+ CacheXmlGenerator.generate((Cache)cache, pw, true, false);
+ String cacheXML = pw.toString();
+ cache.getCacheConfig().setCacheXMLDescription(cacheXML);
+ logger.info("XML generation completed: {}", cacheXML);
+ } catch (CancelException e) {
+ logger.info(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_PROBLEM_GENERATING_CACHE_XML), e);
+ }
+ } else if (sharedConfigEnabled && !cache.getCacheServers().isEmpty()) {
+ // we need to retain a cache-server description if this JVM was started by gfsh
+ List list = new ArrayList(cache.getCacheServers().size());
+ for (Iterator it = cache.getCacheServers().iterator(); it.hasNext(); ) {
+ CacheServerImpl cs = (CacheServerImpl)it.next();
+ if (cs.isDefaultServer()) {
+ CacheServerCreation bsc = new CacheServerCreation(cache, cs);
+ list.add(bsc);
+ }
+ }
+ cache.getCacheConfig().setCacheServerCreation(list);
+ logger.info("CacheServer configuration saved");
+ }
+ }
+ }
+
+ public boolean requestMemberRemoval(DistributedMember mbr, String reason) {
+ if (mbr.equals(this.address)) {
+ return false;
+ }
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_REQUESTING_REMOVAL_OF_0_REASON_1,
+ new Object[] {mbr, reason}));
+ try {
+ services.getJoinLeave().remove((InternalDistributedMember)mbr, reason);
+ }
+ catch (RuntimeException e) {
+ Throwable problem = e;
+ if (services.getShutdownCause() != null) {
+ Throwable cause = services.getShutdownCause();
+ // If ForcedDisconnectException occurred then report it as actual
+ // problem.
+ if (cause instanceof ForcedDisconnectException) {
+ problem = (Exception) cause;
+ } else {
+ Throwable ne = problem;
+ while (ne.getCause() != null) {
+ ne = ne.getCause();
+ }
+ try {
+ ne.initCause(services.getShutdownCause());
+ }
+ catch (IllegalArgumentException selfCausation) {
+ // fix for bug 38895 - the cause is already in place
+ }
+ }
+ }
+ if (!services.getConfig().getDistributionConfig().getDisableAutoReconnect()) {
+ saveCacheXmlForReconnect();
+ }
+ listener.membershipFailure("Channel closed", problem);
+ throw new DistributedSystemDisconnectedException("Channel closed", problem);
+ }
+ return true;
+ }
+
+ public void suspectMembers(Set members, String reason) {
+ for (Iterator it=members.iterator(); it.hasNext(); ) {
+ verifyMember((DistributedMember)it.next(), reason);
+ }
+ }
+
+ public void suspectMember(DistributedMember mbr, String reason) {
+ if (!this.shutdownInProgress && !this.shutdownMembers.containsKey(mbr)) {
+ verifyMember(mbr, reason);
+ }
+ }
+
+ /* like memberExists() this checks to see if the given ID is in the current
+ * membership view. If it is in the view though we try to contact it
+ * to see if it's still around. If we can't contact it then
+ * suspect messages are sent to initiate final checks
+ * @param mbr the member to verify
+ * @param reason why the check is being done (must not be blank/null)
+ * @return true if the member checks out
+ */
+ public boolean verifyMember(DistributedMember mbr, String reason) {
+ if (mbr != null && memberExists((InternalDistributedMember)mbr)) {
+ return this.services.getHealthMonitor().checkIfAvailable(mbr, reason, true);
+ }
+ return false;
+ }
+
+ /**
+ * Perform the grossness associated with sending a message over
+ * a DirectChannel
+ *
+ * @param destinations the list of destinations
+ * @param content the message
+ * @param theStats the statistics object to update
+ * @return all recipients who did not receive the message (null if
+ * all received it)
+ * @throws NotSerializableException if the message is not serializable
+ */
+ protected Set directChannelSend(InternalDistributedMember[] destinations,
+ DistributionMessage content,
+ DMStats theStats)
+ throws NotSerializableException
+ {
+ boolean allDestinations;
+ InternalDistributedMember[] keys;
+ if (content.forAll()) {
+ allDestinations = true;
+ latestViewLock.writeLock().lock();
+ try {
+ List keySet = latestView.getMembers();
+ keys = new InternalDistributedMember[keySet.size()];
+ keys = (InternalDistributedMember[])keySet.toArray(keys);
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+ else {
+ allDestinations = false;
+ keys = destinations;
+ }
+
+ int sentBytes = 0;
+ try {
+ sentBytes = directChannel.send(this, keys, content,
+ this.services.getConfig().getDistributionConfig().getAckWaitThreshold(),
+ this.services.getConfig().getDistributionConfig().getAckSevereAlertThreshold());
+
+ if (theStats != null) {
+ theStats.incSentBytes(sentBytes);
+ }
+
+ if (sentBytes == 0) {
+ if (services.getCancelCriterion().cancelInProgress() != null) {
+ throw new DistributedSystemDisconnectedException();
+ }
+ }
+ }
+ catch (DistributedSystemDisconnectedException ex) {
+ if (services.getShutdownCause() != null) {
+ throw new DistributedSystemDisconnectedException("DistributedSystem is shutting down", services.getShutdownCause());
+ } else {
+ throw ex; // see bug 41416
+ }
+ }
+ catch (ConnectExceptions ex) {
+ if (allDestinations)
+ return null;
+
+ List members = ex.getMembers(); // We need to return this list of failures
+
+ // SANITY CHECK: If we fail to send a message to an existing member
+ // of the view, we have a serious error (bug36202).
+ NetView view = services.getJoinLeave().getView(); // grab a recent view, excluding shunned members
+
+ // Iterate through members and causes in tandem :-(
+ Iterator it_mem = members.iterator();
+ Iterator it_causes = ex.getCauses().iterator();
+ while (it_mem.hasNext()) {
+ InternalDistributedMember member = (InternalDistributedMember)it_mem.next();
+ Throwable th = (Throwable)it_causes.next();
+
+ if (!view.contains(member) || (th instanceof ShunnedMemberException)) {
+ continue;
+ }
+ logger.fatal(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_FAILED_TO_SEND_MESSAGE_0_TO_MEMBER_1_VIEW_2,
+ new Object[] {content, member, view}), th);
+// Assert.assertTrue(false, "messaging contract failure");
+ }
+ return new HashSet(members);
+ } // catch ConnectionExceptions
+ catch (ToDataException | CancelException e) {
+ throw e;
+ }
+ catch (IOException e) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: directChannelSend caught exception: {}", e.getMessage(), e);
+ }
+ if (e instanceof NotSerializableException) {
+ throw (NotSerializableException)e;
+ }
+ }
+ catch (RuntimeException e) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: directChannelSend caught exception: {}", e.getMessage(), e);
+ }
+ throw e;
+ }
+ catch (Error e) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: directChannelSend caught exception: {}", e.getMessage(), e);
+ }
+ throw e;
+ }
+ return null;
+ }
+
+ /*
+ * (non-Javadoc)
+ * @see com.gemstone.gemfire.distributed.internal.membership.MembershipManager#isConnected()
+ */
+ public boolean isConnected() {
+ return (this.hasJoined && !this.shutdownInProgress);
+ }
+
+ /**
+ * Returns true if the distributed system is in the process of auto-reconnecting.
+ * Otherwise returns false.
+ */
+ public boolean isReconnectingDS() {
+ if (this.hasJoined) {
+ return false;
+ } else {
+ return this.wasReconnectingSystem;
+ }
+ }
+
+ @Override
+ public QuorumChecker getQuorumChecker() {
+ if ( ! (services.isShutdownDueToForcedDisconnect()) ) {
+ return null;
+ }
+ if (this.quorumChecker != null) {
+ return this.quorumChecker;
+ }
+
+ QuorumChecker impl = services.getMessenger().getQuorumChecker();
+ this.quorumChecker = impl;
+ return impl;
+ }
+
+ @Override
+ public void releaseQuorumChecker(QuorumChecker checker) {
+ ((GMSQuorumChecker)checker).suspend();
+ InternalDistributedSystem system = InternalDistributedSystem.getAnyInstance();
+ if (system == null || !system.isConnected()) {
+ checker.close();
+ }
+ }
+
+ public Set send(InternalDistributedMember dest, DistributionMessage msg)
+ throws NotSerializableException {
+
+ InternalDistributedMember dests[] = new InternalDistributedMember[] { dest };
+ return send (dests, msg, null);
+ }
+
+ public Set send(InternalDistributedMember[] destinations,
+ DistributionMessage msg,
+ DMStats theStats)
+ throws NotSerializableException
+ {
+ Set result = null;
+ boolean allDestinations = msg.forAll();
+
+ if (services.getCancelCriterion().cancelInProgress() != null) {
+ throw new DistributedSystemDisconnectedException("Distributed System is shutting down",
+ services.getCancelCriterion().generateCancelledException(null));
+ }
+
+ if (playingDead) { // wellness test hook
+ while (playingDead && !shutdownInProgress) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+
+ if (isJoining()) {
+ // If we get here, we are starting up, so just report a failure.
+ if (allDestinations)
+ return null;
+ else {
+ result = new HashSet();
+ for (int i = 0; i < destinations.length; i ++)
+ result.add(destinations[i]);
+ return result;
+ }
+ }
+
+ if (msg instanceof AdminMessageType
+ && this.shutdownInProgress) {
+ // no admin messages while shutting down - this can cause threads to hang
+ return new HashSet(Arrays.asList(msg.getRecipients()));
+ }
+
+ // Handle trivial cases
+ if (destinations == null) {
+ if (logger.isTraceEnabled())
+ logger.trace("Membership: Message send: returning early because null set passed in: '{}'", msg);
+ return null; // trivially: all recipients received the message
+ }
+ if (destinations.length == 0) {
+ if (logger.isTraceEnabled())
+ logger.trace("Membership: Message send: returning early because empty destination list passed in: '{}'", msg);
+ return null; // trivially: all recipients received the message
+ }
+
+ msg.setSender(address);
+
+ msg.setBreadcrumbsInSender();
+ Breadcrumbs.setProblem(null);
+
+ boolean useMcast = false;
+ if (mcastEnabled) {
+ useMcast = (msg.getMulticast() || allDestinations);
+ }
+
+ boolean sendViaMessenger = isForceUDPCommunications(); // enable when bug #46438 is fixed: || msg.sendViaUDP();
+
+ if (useMcast || tcpDisabled || sendViaMessenger) {
+ checkAddressesForUUIDs(destinations);
+ result = services.getMessenger().send(msg);
+ }
+ else {
+ result = directChannelSend(destinations, msg, theStats);
+ }
+
+ // If the message was a broadcast, don't enumerate failures.
+ if (allDestinations)
+ return null;
+ else {
+ return result;
+ }
+ }
+
+ // MembershipManager method
+ @Override
+ public void forceUDPMessagingForCurrentThread() {
+ forceUseUDPMessaging.set(null);
+ }
+
+ void checkAddressesForUUIDs(InternalDistributedMember[] addresses) {
+ for (int i=0; i now)
+ return true;
+
+ // Oh, it _is_ stale. Remove it while we're here.
+ endShun(m);
+ return false;
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ /**
+ * Indicate whether the given member is in the surprise member list
+ *
+ * Unlike isShunned, this method will not cause expiry of a surprise member.
+ * That must be done during view processing.
+ *
+ * Like isShunned, this method holds the view lock while executing
+ *
+ * Concurrency: protected by {@link #latestViewLock} ReentrantReadWriteLock
+ *
+ * @param m the member in question
+ * @return true if the given member is a surprise member
+ */
+ public boolean isSurpriseMember(DistributedMember m) {
+ latestViewLock.readLock().lock();
+ try {
+ if (surpriseMembers.containsKey(m)) {
+ long birthTime = ((Long)surpriseMembers.get(m)).longValue();
+ long now = System.currentTimeMillis();
+ return (birthTime >= (now - this.surpriseMemberTimeout));
+ }
+ return false;
+ } finally {
+ latestViewLock.readLock().unlock();
+ }
+ }
+
+ /**
+ * for testing we need to be able to inject surprise members into
+ * the view to ensure that sunsetting works properly
+ * @param m the member ID to add
+ * @param birthTime the millisecond clock time that the member was first seen
+ */
+ public void addSurpriseMemberForTesting(DistributedMember m, long birthTime) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("test hook is adding surprise member {} birthTime={}", m, birthTime);
+ }
+ latestViewLock.writeLock().lock();
+ try {
+ surpriseMembers.put((InternalDistributedMember)m, Long.valueOf(birthTime));
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ /**
+ * returns the surpriseMemberTimeout interval, in milliseconds
+ */
+ public int getSurpriseMemberTimeout() {
+ return this.surpriseMemberTimeout;
+ }
+
+ private boolean endShun(DistributedMember m) {
+ boolean wasShunned = (shunnedMembers.remove(m) != null);
+ shunnedAndWarnedMembers.remove(m);
+ return wasShunned;
+ }
+
+ /**
+ * Add the given member to the shunned list. Also, purge any shunned
+ * members that are really really old.
+ *
+ * Must be called with {@link #latestViewLock} held and
+ * the view stable.
+ *
+ * @param m the member to add
+ */
+ protected void addShunnedMember(InternalDistributedMember m) {
+ long deathTime = System.currentTimeMillis() - SHUNNED_SUNSET * 1000;
+
+ surpriseMembers.remove(m); // for safety
+
+ // Update the shunned set.
+ if (!isShunned(m)) {
+ shunnedMembers.put(m, Long.valueOf(System.currentTimeMillis()));
+ }
+
+ // Remove really really old shunned members.
+ // First, make a copy of the old set. New arrivals _a priori_ don't matter,
+ // and we're going to be updating the list so we don't want to disturb
+ // the iterator.
+ Set oldMembers = new HashSet(shunnedMembers.entrySet());
+
+ Set removedMembers = new HashSet();
+
+ Iterator it = oldMembers.iterator();
+ while (it.hasNext()) {
+ Map.Entry e = (Map.Entry)it.next();
+
+ // Key is the member. Value is the time to remove it.
+ long ll = ((Long)e.getValue()).longValue();
+ if (ll >= deathTime) {
+ continue; // too new.
+ }
+
+ InternalDistributedMember mm = (InternalDistributedMember)e.getKey();
+
+ if (latestView.contains(mm)) {
+ // Fault tolerance: a shunned member can conceivably linger but never
+ // disconnect.
+ //
+ // We may not delete it at the time that we shun it because the view
+ // isn't necessarily stable. (Note that a well-behaved cache member
+ // will depart on its own accord, but we force the issue here.)
+ destroyMember(mm, true, "shunned but never disconnected");
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("Membership: finally removed shunned member entry <{}>", mm);
+ }
+
+ removedMembers.add(mm);
+ }
+
+ // removed timed-out entries from the shunned-members collections
+ if (removedMembers.size() > 0) {
+ it = removedMembers.iterator();
+ while (it.hasNext()) {
+ InternalDistributedMember idm = (InternalDistributedMember)it.next();
+ endShun(idm);
+ }
+ }
+ }
+
+
+ /**
+ * Retrieve thread-local data for transport to another thread in hydra
+ */
+ public Object getThreadLocalData() {
+ Map result = new HashMap();
+ return result;
+ }
+
+ /**
+ * for testing verification purposes, this return the port for the
+ * direct channel, or zero if there is no direct
+ * channel
+ */
+ public int getDirectChannelPort() {
+ return directChannel == null? 0 : directChannel.getPort();
+ }
+
+ /**
+ * for mock testing this allows insertion of a DirectChannel mock
+ */
+ protected void setDirectChannel(DirectChannel dc) {
+ this.directChannel = dc;
+ this.tcpDisabled = false;
+ }
+
+ /* non-thread-owned serial channels and high priority channels are not
+ * included
+ */
+ public Map getMessageState(DistributedMember member, boolean includeMulticast) {
+ Map result = new HashMap();
+ DirectChannel dc = directChannel;
+ if (dc != null) {
+ dc.getChannelStates(member, result);
+ }
+ services.getMessenger().getMessageState((InternalDistributedMember)member, result, includeMulticast);
+ return result;
+ }
+
+ public void waitForMessageState(DistributedMember otherMember, Map state)
+ throws InterruptedException
+ {
+ if (Thread.interrupted()) throw new InterruptedException();
+ DirectChannel dc = directChannel;
+ if (dc != null) {
+ dc.waitForChannelState(otherMember, state);
+ }
+ services.getMessenger().waitForMessageState((InternalDistributedMember)otherMember, state);
+ }
+
+ /*
+ * (non-Javadoc)
+ * MembershipManager method: wait for the given member to be gone. Throws TimeoutException if
+ * the wait goes too long
+ * @see com.gemstone.gemfire.distributed.internal.membership.MembershipManager#waitForDeparture(com.gemstone.gemfire.distributed.DistributedMember)
+ */
+ public boolean waitForDeparture(DistributedMember mbr) throws TimeoutException, InterruptedException {
+ if (Thread.interrupted()) throw new InterruptedException();
+ boolean result = false;
+ DirectChannel dc = directChannel;
+ InternalDistributedMember idm = (InternalDistributedMember)mbr;
+ int memberTimeout = this.services.getConfig().getDistributionConfig().getMemberTimeout();
+ long pauseTime = (memberTimeout < 1000) ? 100 : memberTimeout / 10;
+ boolean wait;
+ int numWaits = 0;
+ do {
+ wait = false;
+ if (dc != null) {
+ if (dc.hasReceiversFor(idm)) {
+ wait = true;
+ }
+ if (wait && logger.isDebugEnabled()) {
+ logger.info("waiting for receivers for {} to shut down", mbr);
+ }
+ }
+ if (!wait) {
+ latestViewLock.readLock().lock();
+ try {
+ wait = this.latestView.contains(idm);
+ } finally {
+ latestViewLock.readLock().unlock();
+ }
+ if (wait && logger.isDebugEnabled()) {
+ logger.debug("waiting for {} to leave the membership view", mbr);
+ }
+ }
+ if (!wait) {
+ // run a message through the member's serial execution queue to ensure that all of its
+ // current messages have been processed
+ OverflowQueueWithDMStats serialQueue = listener.getDM().getSerialQueue(idm);
+ if (serialQueue != null) {
+ final boolean done[] = new boolean[1];
+ final FlushingMessage msg = new FlushingMessage(done);
+ serialQueue.add(new SizeableRunnable(100) {
+ public void run() {
+ msg.invoke();
+ }
+ public String toString() {
+ return "Processing fake message";
+ }
+ });
+ synchronized(done) {
+ while (done[0] == false) {
+ done.wait(10);
+ }
+ result = true;
+ }
+ }
+ }
+ if (wait) {
+ numWaits++;
+ if (numWaits > 40) {
+ // waited over 4 * memberTimeout ms. Give up at this point
+ throw new TimeoutException("waited too long for " + idm + " to be removed");
+ }
+ Thread.sleep(pauseTime);
+ }
+ } while (wait && (dc != null && dc.isOpen())
+ && services.getCancelCriterion().cancelInProgress()==null );
+ if (logger.isDebugEnabled()) {
+ logger.debug("operations for {} should all be in the cache at this point", mbr);
+ }
+ return result;
+ }
+
+
+ // TODO remove this overly complex method and replace its use with
+ // waitForViewChange using the remote member's view ID
+ public boolean waitForMembershipCheck(InternalDistributedMember remoteId) {
+ boolean foundRemoteId = false;
+ CountDownLatch currentLatch = null;
+ // ARB: preconditions
+ // remoteId != null
+ latestViewLock.writeLock().lock();
+ try {
+ if (latestView == null) {
+ // Not sure how this would happen, but see bug 38460.
+ // No view?? Not found!
+ }
+ else if (latestView.contains(remoteId)) {
+ // ARB: check if remoteId is already in membership view.
+ // If not, then create a latch if needed and wait for the latch to open.
+ foundRemoteId = true;
+ }
+ else if ((currentLatch = (CountDownLatch)this.memberLatch.get(remoteId)) == null) {
+ currentLatch = new CountDownLatch(1);
+ this.memberLatch.put(remoteId, currentLatch);
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+
+ if (!foundRemoteId) {
+ try {
+ if (currentLatch.await(membershipCheckTimeout, TimeUnit.MILLISECONDS)) {
+ foundRemoteId = true;
+ // @todo
+ // ARB: remove latch from memberLatch map if this is the last thread waiting on latch.
+ }
+ }
+ catch (InterruptedException ex) {
+ // ARB: latch attempt was interrupted.
+ Thread.currentThread().interrupt();
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_THE_MEMBERSHIP_CHECK_WAS_TERMINATED_WITH_AN_EXCEPTION));
+ }
+ }
+
+ // ARB: postconditions
+ // (foundRemoteId == true) ==> (currentLatch is non-null ==> currentLatch is open)
+ return foundRemoteId;
+ }
+
+ /* returns the cause of shutdown, if known */
+ public Throwable getShutdownCause() {
+ return services.getShutdownCause();
+ }
+
+// @Override
+// public void membershipFailure(String reason, Exception e) {
+// try {
+// if (this.membershipTestHooks != null) {
+// List l = this.membershipTestHooks;
+// for (Iterator it=l.iterator(); it.hasNext(); ) {
+// MembershipTestHook dml = (MembershipTestHook)it.next();
+// dml.beforeMembershipFailure(reason, e);
+// }
+// }
+// listener.membershipFailure(reason, e);
+// if (this.membershipTestHooks != null) {
+// List l = this.membershipTestHooks;
+// for (Iterator it=l.iterator(); it.hasNext(); ) {
+// MembershipTestHook dml = (MembershipTestHook)it.next();
+// dml.afterMembershipFailure(reason, e);
+// }
+// }
+// }
+// catch (RuntimeException re) {
+// logger.warn(LocalizedMessage.create(LocalizedStrings.GroupMembershipService_EXCEPTION_CAUGHT_WHILE_SHUTTING_DOWN), re);
+// }
+// }
+
+ public void registerTestHook(MembershipTestHook mth) {
+ // lock for additions to avoid races during startup
+ latestViewLock.writeLock().lock();
+ try {
+ if (this.membershipTestHooks == null) {
+ this.membershipTestHooks = Collections.singletonList(mth);
+ }
+ else {
+ List l = new ArrayList(this.membershipTestHooks);
+ l.add(mth);
+ this.membershipTestHooks = l;
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ public void unregisterTestHook(MembershipTestHook mth) {
+ latestViewLock.writeLock().lock();
+ try {
+ if (this.membershipTestHooks != null) {
+ if (this.membershipTestHooks.size() == 1) {
+ this.membershipTestHooks = null;
+ }
+ else {
+ List l = new ArrayList(this.membershipTestHooks);
+ l.remove(mth);
+ }
+ }
+ } finally {
+ latestViewLock.writeLock().unlock();
+ }
+ }
+
+ boolean beingSick;
+ boolean playingDead;
+
+ /**
+ * Test hook - be a sick member
+ */
+ public synchronized void beSick() {
+ if (!beingSick) {
+ beingSick = true;
+ logger.info("GroupMembershipService.beSick invoked for {} - simulating sickness", this.address);
+ services.getJoinLeave().beSick();
+ services.getHealthMonitor().beSick();
+ if (directChannel != null) {
+ directChannel.beSick();
+ }
+ }
+ }
+
+ /**
+ * Test hook - don't answer "are you alive" requests
+ */
+ public synchronized void playDead() {
+ if (!playingDead) {
+ playingDead = true;
+ logger.info("GroupMembershipService.playDead invoked for {}", this.address);
+ services.getJoinLeave().playDead();
+ services.getHealthMonitor().playDead();
+ services.getMessenger().playDead();
+ }
+ }
+
+ /**
+ * Test hook - recover health
+ */
+ public synchronized void beHealthy() {
+ if (beingSick || playingDead) {
+ beingSick = false;
+ playingDead = false;
+ logger.info("GroupMembershipService.beHealthy invoked for {} - recovering health now", this.address);
+ if (directChannel != null) {
+ directChannel.beHealthy();
+ }
+ services.getJoinLeave().beHealthy();
+ services.getHealthMonitor().beHealthy();
+ services.getMessenger().beHealthy();
+ }
+ }
+
+ /**
+ * Test hook
+ */
+ public boolean isBeingSick() {
+ return this.beingSick;
+ }
+
+ /**
+ * Test hook - inhibit ForcedDisconnectException logging to keep dunit logs clean
+ * @param b
+ */
+ public static void inhibitForcedDisconnectLogging(boolean b) {
+ inhibitForceDisconnectLogging = true;
+ }
+
+ /** this is a fake message class that is used to flush the serial execution queue */
+ static class FlushingMessage extends DistributionMessage {
+ boolean[] done;
+ FlushingMessage(boolean[] done) {
+ this.done = done;
+ }
+ public void invoke() {
+ synchronized(done) {
+ done[0] = true;
+ done.notify();
+ }
+ }
+ protected void process(DistributionManager dm) {
+ // not used
+ }
+ public int getDSFID() {
+ return 0;
+ }
+ public int getProcessorType() {
+ return DistributionManager.SERIAL_EXECUTOR;
+ }
+ }
+
+ /**
+ * Sets cache time offset in {@link DistributionManager}.
+ *
+ * @param coord
+ * @param timeOffset
+ * @see InternalDistributedSystem#getClock()
+ * @see DSClock#setCacheTimeOffset(DistributedMember, long, boolean)
+ */
+ public void setCacheTimeOffset(InternalDistributedMember coord, long timeOffset, boolean isJoin) {
+ if (this.listener != null) {
+ DistributionManager dm = this.listener.getDM();
+ dm.getSystem().getClock().setCacheTimeOffset(coord, timeOffset, isJoin);
+ }
+ }
+
+ @Override
+ public void stopped() {
+ }
+
+ @Override
+ public void installView(NetView v) {
+ if (latestViewId < 0 && !isConnected()) {
+ if (this.directChannel != null) {
+ this.directChannel.setMembershipSize(v.getMembers().size());
+ }
+ latestViewId = v.getViewId();
+ latestView = v;
+ logger.debug("MembershipManager: initial view is {}", latestView);
+ } else {
+ handleOrDeferViewEvent(v);
+ }
+ }
+
+ @Override
+ public Set send(DistributionMessage m) throws NotSerializableException {
+ return send(m.getRecipients(), m, this.services.getStatistics());
+ }
+
+ @Override
+ public void forceDisconnect(final String reason) {
+ if (GMSMembershipManager.this.shutdownInProgress || isJoining()) {
+ return; // probably a race condition
+ }
+
+ setShutdown();
+
+ final Exception shutdownCause = new ForcedDisconnectException(reason);
+
+ // cache the exception so it can be appended to ShutdownExceptions
+ services.setShutdownCause(shutdownCause);
+ services.getCancelCriterion().cancel(reason);
+
+ AlertAppender.getInstance().shuttingDown();
+
+ if (!inhibitForceDisconnectLogging) {
+ logger.fatal(LocalizedMessage.create(
+ LocalizedStrings.GroupMembershipService_MEMBERSHIP_SERVICE_FAILURE_0, reason), shutdownCause);
+ }
+
+ if (!services.getConfig().getDistributionConfig().getDisableAutoReconnect()) {
+ saveCacheXmlForReconnect();
+ }
+
+
+ Thread reconnectThread = new Thread (new Runnable() {
+ public void run() {
+ // stop server locators immediately since they may not have correct
+ // information. This has caused client failures in bridge/wan
+ // network-down testing
+ InternalLocator loc = (InternalLocator)Locator.getLocator();
+ if (loc != null) {
+ loc.stop(true, !services.getConfig().getDistributionConfig()
+ .getDisableAutoReconnect(), false);
+ }
+
+ uncleanShutdown(reason, shutdownCause);
+ }
+ });
+
+ reconnectThread.setName("DisconnectThread");
+ reconnectThread.setDaemon(false);
+ reconnectThread.start();
+ }
+
+
+ public void disableDisconnectOnQuorumLossForTesting() {
+ services.getJoinLeave().disableDisconnectOnQuorumLossForTesting();
+ }
+
+
+ /**
+ * Class BoundedLinkedHashMap
is a bounded
+ * LinkedHashMap
. The bound is the maximum
+ * number of entries the BoundedLinkedHashMap
+ * can contain.
+ */
+ static class BoundedLinkedHashMap extends LinkedHashMap
+ {
+ private static final long serialVersionUID = -3419897166186852692L;
+
+ /**
+ * The maximum number of entries allowed in this
+ * BoundedLinkedHashMap
+ */
+ protected int _maximumNumberOfEntries;
+
+ /**
+ * Constructor.
+ *
+ * @param maximumNumberOfEntries The maximum number of allowed entries
+ */
+ public BoundedLinkedHashMap(int maximumNumberOfEntries) {
+ super();
+ this._maximumNumberOfEntries = maximumNumberOfEntries;
+ }
+
+ @Override
+ protected boolean removeEldestEntry(Map.Entry entry) {
+ return size() > this._maximumNumberOfEntries;
+ }
+ }
+
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/MigrationServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/MigrationServer.java
new file mode 100755
index 000000000000..3d6d6bef0750
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/MigrationServer.java
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal;
+
+import com.gemstone.gemfire.admin.internal.InetAddressUtil;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.NetworkInterface;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.SocketException;
+import java.util.Enumeration;
+import java.util.Iterator;
+import java.util.Properties;
+
+/**
+ * MigrationServer creates a cache using a supplied cache.xml and then
+ * opens a server socket that a MigrationClient connects to and requests
+ * the data from a Region. MigrationServer sends the data to
+ * the MigrationClient using normal java serialization in
+ * order to allow migration from incompatible versions of DataSerializer.
+ * Keys and values stored in the cache must serialize and deserialize correctly.
+ *
+ * Command line arguments are
+ * cache-xml-file-name (required)
+ * listen port (defaults to 10553)
+ * bind address (defaults to listing on all interfaces)
+ *
+ * Both the MigrationClient and MigrationServer must be configured to have
+ * the appropriate domain classes in their CLASSPATH, or errors will be
+ * encountered during deserialization.
+ *
+ * Details of the transfers can be viewed by setting the system property
+ * Migration.VERBOSE=true.
+ *
+ * For example,
+ *
+ * java -cp $MYCLASSES:migration.jar:$GEMFIRE/lib/gemfire-core-dependencies.jar \
+ * com.gemstone.gemfire.internal.MigrationServer cacheDescription.xml
+ *
+ * Where the cacheDescription.xml file might look like this:
+ *
+ * <!DOCTYPE cache PUBLIC
+ "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.7//EN"
+ "http://www.gemstone.com/dtd/cache5_7.dtd">
+<cache is-server="false">
+ <region name="root">
+ <region-attributes scope="distributed-no-ack">
+ </region-attributes>
+
+ <region name="Test">
+ <region-attributes data-policy="persistent-replicate">
+
+ <disk-write-attributes>
+ <synchronous-writes/>
+ </disk-write-attributes>
+
+ <disk-dirs>
+ <disk-dir>diskfiles</disk-dir>
+ </disk-dirs>
+
+ <eviction-attributes>
+ <lru-memory-size maximum="100" action="overflow-to-disk"/>
+ </eviction-attributes>
+
+ </region-attributes>
+ </region> <!-- Test region -->
+ </region> <!-- root region -->
+</cache>
+
+ *
+ * The client is then run with a different cache description having different
+ * disk-dirs to hold the migrated information.
+ *
+ * @since 6.0.1
+ */
+public class MigrationServer {
+ final static boolean VERBOSE = Boolean.getBoolean("Migration.VERBOSE");
+
+ final static int VERSION = 551; // version for backward communications compatibility
+
+ protected static final int CODE_ERROR = 0;
+ protected static final int CODE_ENTRY = 1; /* serialized key, serialized value */
+ protected static final int CODE_COMPLETED = 2;
+
+ public static void main(String[] args) throws Exception {
+ int argIdx = 0;
+ String cacheXmlFileName = "cache.xml";
+ String bindAddressName = null;
+ int listenPort = 10533;
+
+ if (args.length > 0) {
+ cacheXmlFileName = args[argIdx++];
+ } else {
+ System.err.println("MigrationServer cache-xml-file [server-address] [server-port]");
+ }
+ if (args.length > argIdx) {
+ listenPort = Integer.parseInt(args[argIdx++]);
+ }
+ if (args.length > argIdx) {
+ bindAddressName = args[argIdx++];
+ }
+
+ MigrationServer instance = null;
+ try {
+ instance = new MigrationServer(cacheXmlFileName, bindAddressName, listenPort);
+ } catch (IllegalArgumentException e) {
+ System.err.println(e.getMessage());
+ System.exit(1);
+ }
+ instance.createDistributedSystem();
+ instance.createCache();
+ instance.serve();
+ }
+
+
+ private InetAddress bindAddress;
+ private int listenPort;
+ private ServerSocket serverSocket;
+ private DistributedSystem distributedSystem;
+ private File cacheXmlFile;
+ private Cache cache;
+
+
+ /**
+ * Create a MigrationServer to be used with a DistributedSystem and Cache
+ * that are created using GemFire APIs
+ * @param bindAddressName the NIC to bind to, or null to use all interfaces
+ * @param listenPort the port to listen on
+ */
+ public MigrationServer(String bindAddressName, int listenPort) {
+ this.listenPort = listenPort;
+ if (bindAddressName != null) {
+ if (!isLocalHost(bindAddressName)) {
+ throw new IllegalArgumentException("Error - bind address is not an address of this machine: '" + bindAddressName + "'");
+ }
+ try {
+ this.bindAddress = InetAddress.getByName(bindAddressName);
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Error - bind address cannot be resolved: '" + bindAddressName + "'");
+ }
+ }
+ try {
+ if (this.bindAddress != null) {
+ this.serverSocket = new ServerSocket();
+ SocketAddress addr = new InetSocketAddress(this.bindAddress, listenPort);
+ this.serverSocket.bind(addr);
+ } else {
+ this.serverSocket = new ServerSocket(listenPort);
+ }
+ if (VERBOSE) {
+ System.out.println("created server socket " + serverSocket);
+ }
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Port is already in use", e);
+ }
+ }
+
+ /**
+ * this is for use by main()
+ *
+ * @param cacheXmlFileName the name of the xml file describing the cache, or null
+ * @param bindAddressName the name of the NIC to bind to, or null
+ * @param listenPort the port to listen on (must not be zero)
+ */
+ private MigrationServer(String cacheXmlFileName, String bindAddressName, int listenPort) {
+ this(bindAddressName, listenPort);
+ this.cacheXmlFile = new File(cacheXmlFileName);
+ if (!this.cacheXmlFile.exists()) {
+ // in 6.x this should be localizable
+ System.err.println("Warning - file not found in local directory: '" + cacheXmlFileName + "'");
+ }
+ }
+
+ /**
+ * Create a distributed system. If this method is not invoked before running
+ * the MigrationServer, an existing distributed system must exist for the
+ * server to use.
+ *
+ * @throws Exception if there are any problems
+ */
+ private void createDistributedSystem() throws Exception {
+ Properties dsProps = new Properties();
+ // if no discovery information has been explicitly given, use a loner ds
+ if (System.getProperty("gemfire." + DistributionConfig.MCAST_PORT_NAME) == null
+ && System.getProperty("gemfire." + DistributionConfig.LOCATORS_NAME) == null) {
+ dsProps.put(DistributionConfig.MCAST_PORT_NAME, "0");
+ }
+ dsProps.put(DistributionConfig.LOG_FILE_NAME, "migrationServer.log");
+ if (this.cacheXmlFile != null) {
+ dsProps.put(DistributionConfig.CACHE_XML_FILE_NAME, this.cacheXmlFile.getName());
+ }
+ this.distributedSystem = DistributedSystem.connect(dsProps);
+ if (VERBOSE) {
+ System.out.println("created distributed system " + this.distributedSystem);
+ }
+ }
+
+
+ /**
+ * create the cache to be used by this migration server
+ * @throws Exception if there are any problems
+ */
+ private void createCache() throws Exception {
+ if (this.distributedSystem == null) {
+ this.distributedSystem = InternalDistributedSystem.getConnectedInstance();
+ }
+ this.cache = CacheFactory.create(this.distributedSystem);
+ if (VERBOSE) {
+ System.out.println("created cache " + this.cache);
+ }
+ }
+
+
+ /**
+ * This locates the distributed system and cache, if they have not been
+ * created by this server, and then listens for requests from MigrationClient
+ * processes.
+ * @throws IllegalStateException if an attempt is made to reuse a server that has been stopped
+ */
+ public void serve() throws Exception {
+ if (this.serverSocket == null || this.serverSocket.isClosed()) {
+ throw new IllegalStateException("This server has been closed and cannot be reused");
+ }
+ try {
+ if (this.distributedSystem == null) {
+ this.distributedSystem = InternalDistributedSystem.getConnectedInstance();
+ }
+ if (this.cache == null) {
+ this.cache = GemFireCacheImpl.getInstance();
+ }
+ if (this.bindAddress != null) {
+ System.out.println("Migration server on port " + this.listenPort +
+ " bound to " + this.bindAddress + " is ready for client requets");
+ } else {
+ System.out.println("Migration server on port " + this.listenPort +
+ " is ready for client requests");
+ }
+ for (;;) {
+ if (Thread.interrupted() || this.serverSocket.isClosed()) {
+ return;
+ }
+ Socket clientSocket;
+ try {
+ clientSocket = this.serverSocket.accept();
+ } catch (java.net.SocketException e) {
+ return;
+ }
+ (new RequestHandler(clientSocket)).serveClientRequest();
+ }
+ } finally {
+ System.out.println("Closing migration server");
+ try {
+ this.serverSocket.close();
+ } catch (Exception e) {
+ this.serverSocket = null;
+ }
+ }
+ }
+
+ /**
+ * this causes the migration server to stop serving after it finishes dispatching
+ * any in-process requests
+ * @throws IOException if there is a problem closing the server socket
+ */
+ public void stop() throws IOException {
+ if (this.serverSocket != null && !this.serverSocket.isClosed()) {
+ this.serverSocket.close();
+ }
+ }
+
+ /**
+ * get the cache being used by this migration server
+ * @return the cache, or null if a cache has not yet been associated with this server
+ */
+ public Cache getCache() {
+ return this.cache;
+ }
+
+ /**
+ * get the distributed system being used by this migration server
+ * @return the distributed system, or null if a system has not yet been associated with this server
+ */
+ public DistributedSystem getDistributedSystem() {
+ return this.distributedSystem;
+ }
+
+
+
+
+
+
+ // copied from 6.0 SocketCreator
+ public static boolean isLocalHost(Object host) {
+ if (host instanceof InetAddress) {
+ if (InetAddressUtil.LOCALHOST.equals(host)) {
+ return true;
+ }
+ else {
+ try {
+ Enumeration en=NetworkInterface.getNetworkInterfaces();
+ while(en.hasMoreElements()) {
+ NetworkInterface i=(NetworkInterface)en.nextElement();
+ for(Enumeration en2=i.getInetAddresses(); en2.hasMoreElements();) {
+ InetAddress addr=(InetAddress)en2.nextElement();
+ if (host.equals(addr)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+ catch (SocketException e) {
+ throw new IllegalArgumentException(LocalizedStrings.InetAddressUtil_UNABLE_TO_QUERY_NETWORK_INTERFACE.toLocalizedString(), e);
+ }
+ }
+ }
+ else {
+ return isLocalHost(toInetAddress(host.toString()));
+ }
+ }
+
+ // copied from 6.0 SocketCreator
+ public static InetAddress toInetAddress(String host) {
+ if (host == null || host.length() == 0) {
+ return null;
+ }
+ try {
+ if (host.indexOf("/") > -1) {
+ return InetAddress.getByName(host.substring(host.indexOf("/") + 1));
+ }
+ else {
+ return InetAddress.getByName(host);
+ }
+ } catch (java.net.UnknownHostException e) {
+ throw new IllegalArgumentException(e.getMessage());
+ }
+ }
+
+
+
+
+
+ // R E Q U E S T H A N D L E R
+
+
+ class RequestHandler implements Runnable {
+ Socket clientSocket;
+ DataInputStream dis;
+ DataOutputStream dos;
+
+ RequestHandler(Socket clientSocket) throws IOException {
+ this.clientSocket = clientSocket;
+ dos = new DataOutputStream(this.clientSocket.getOutputStream());
+ dis = new DataInputStream(this.clientSocket.getInputStream());
+ }
+
+
+ // for now this is a blocking operation - multithread later if necessary
+ void serveClientRequest() {
+ try {
+ run();
+ }
+ finally {
+ if (!this.clientSocket.isClosed()) {
+ try {
+ this.clientSocket.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ public void run() {
+ try {
+ // first exchange version information so we can communicate correctly
+ dos.writeShort(VERSION);
+ int version = dis.readShort();
+ handleRequest(version);
+ }
+ catch (IOException e) {
+ System.err.println("Trouble dispatching request: " + e.getMessage());
+ return;
+ }
+ finally {
+ try {
+ this.clientSocket.close();
+ } catch (IOException e) {
+ System.err.println("Trouble closing client socket: " + e.getMessage());
+ }
+ }
+ }
+
+ /**
+ * read and dispatch a single request on client socket
+ * @param clientVersion
+ */
+ private void handleRequest(int clientVersion) {
+ // for now we ignore the client version in the server. The client
+ // is typically of a later release than the server, and this information
+ // is given to the server in case a situation arises where it's needed
+ try {
+ ClientRequest req = ClientRequest.readRequest(this.clientSocket, dis, dos);
+ if (req != null) {
+ System.out.println("Processing " + req + " from " + this.clientSocket.getInetAddress().getHostAddress());
+ req.process(MigrationServer.this);
+ dos.flush();
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ }
+
+
+ // R E Q U E S T C L A S S E S
+
+
+
+ static abstract class ClientRequest {
+ Socket clientSocket;
+ DataInputStream dsi;
+ DataOutputStream dso;
+
+ final static int REGION_REQUEST = 1;
+
+ /**
+ * Use readRequest to create a new request object, not this constructor.
+ * Subclasses may refine this constructor to perform other initialization
+ * @param dsi socket's input stream
+ * @param dso socket's output stream
+ * @throws IOException if there are any problems reading initialization information
+ */
+ ClientRequest(Socket clientSocket, DataInputStream dsi, DataOutputStream dso) throws IOException {
+ this.clientSocket = clientSocket;
+ this.dsi = dsi;
+ this.dso = dso;
+ }
+
+ /**
+ * Read and return a request from a client
+ * @param clientSocket
+ * @param dsi socket input stream
+ * @param dso socket output stream
+ * @return the new request
+ * @throws IOException
+ */
+ static ClientRequest readRequest(Socket clientSocket, DataInputStream dsi, DataOutputStream dso) throws IOException {
+ int requestType = dsi.readShort();
+ switch (requestType) {
+ case REGION_REQUEST:
+ return new RegionRequest(clientSocket, dsi, dso);
+ }
+ String errorMessage = "Type of request is not implemented in this server";
+ dso.writeShort(CODE_ERROR);
+ dso.writeUTF(errorMessage);
+ System.err.println("Migration server received unknown type of request ("
+ + requestType + ") from " + clientSocket.getInetAddress().getHostAddress());
+ return null;
+ }
+
+ void writeErrorResponse(String message) throws IOException {
+ this.dso.writeShort(CODE_ERROR);
+ this.dso.writeUTF(message);
+ }
+
+ abstract void process(MigrationServer server) throws IOException ;
+
+ }
+
+ /**
+ * RegionRequest represents a request for the keys and values of a Region
+ * from a client.
+ */
+ static class RegionRequest extends ClientRequest {
+ String regionName;
+
+ RegionRequest(Socket clientSocket, DataInputStream dsi, DataOutputStream dso) throws IOException {
+ super(clientSocket, dsi, dso);
+ regionName = dsi.readUTF();
+ }
+
+ @Override
+ public String toString() {
+ return "request for contents of region '" + this.regionName + "'";
+ }
+
+ @Override
+ void process(MigrationServer server) throws IOException {
+ Cache cache = server.getCache();
+ Region region = null;
+ try {
+ region = cache.getRegion(regionName);
+ if (region == null) {
+ String errorMessage = "Error: region " + this.regionName + " not found in cache";
+ System.err.println(errorMessage);
+ writeErrorResponse(errorMessage);
+ }
+ } catch (IllegalArgumentException e) {
+ String errorMessage = "Error: malformed region name";
+ System.err.println(errorMessage);
+ writeErrorResponse(errorMessage);
+ }
+ try {
+ for (Iterator it = region.entrySet().iterator(); it.hasNext(); ) {
+ sendEntry((Region.Entry)it.next());
+ }
+ this.dso.writeShort(CODE_COMPLETED);
+ }
+ catch (Exception e) {
+ writeErrorResponse(e.getMessage());
+ }
+ }
+
+ private void sendEntry(Region.Entry entry) throws Exception {
+ Object key = entry.getKey();
+ Object value = entry.getValue();
+ if ( !(key instanceof Serializable) ) {
+ throw new IOException("Could not serialize entry for '" + key + "'");
+ }
+ if ( !(value instanceof Serializable) ) {
+ throw new IOException("Could not serialize entry for '" + key + "'");
+ }
+ if (VERBOSE) {
+ System.out.println("Sending " + key);
+ }
+ dso.writeShort(CODE_ENTRY);
+ (new ObjectOutputStream(clientSocket.getOutputStream())).writeObject(key);
+ (new ObjectOutputStream(clientSocket.getOutputStream())).writeObject(value);
+ }
+ }
+
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
new file mode 100644
index 000000000000..327170ea529b
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
@@ -0,0 +1,2241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.logging.log4j.Logger;
+
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ABSTRACT_REGION_ENTRY_FILL_IN_VALUE;
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE;
+
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.InvalidDeltaException;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.EntryEvent;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
+import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
+import com.gemstone.gemfire.cache.util.GatewayConflictHelper;
+import com.gemstone.gemfire.cache.util.GatewayConflictResolver;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.ByteArrayDataInput;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.InternalStatisticsDisabledException;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.lru.LRUClockNode;
+import com.gemstone.gemfire.internal.cache.lru.NewLRUClockHand;
+import com.gemstone.gemfire.internal.cache.persistence.DiskStoreID;
+import com.gemstone.gemfire.internal.cache.versions.ConcurrentCacheModificationException;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
+import com.gemstone.gemfire.internal.cache.versions.VersionSource;
+import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventImpl;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.ChunkWithHeapForm;
+import com.gemstone.gemfire.internal.offheap.GemFireChunk;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocator;
+import com.gemstone.gemfire.internal.offheap.OffHeapCachedDeserializable;
+import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.internal.offheap.annotations.Released;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+import com.gemstone.gemfire.internal.util.BlobHelper;
+import com.gemstone.gemfire.internal.util.Versionable;
+import com.gemstone.gemfire.internal.util.concurrent.CustomEntryConcurrentHashMap;
+import com.gemstone.gemfire.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
+import com.gemstone.gemfire.pdx.PdxInstance;
+import com.gemstone.gemfire.pdx.PdxSerializable;
+import com.gemstone.gemfire.pdx.PdxSerializationException;
+import com.gemstone.gemfire.pdx.PdxSerializer;
+import com.gemstone.gemfire.pdx.internal.ConvertableToBytes;
+import com.gemstone.gemfire.pdx.internal.PdxInstanceImpl;
+
+/**
+ * Abstract implementation class of RegionEntry interface.
+ * This is the topmost implementation class so common behavior
+ * lives here.
+ *
+ * @since 3.5.1
+ *
+ *
+ */
+public abstract class AbstractRegionEntry implements RegionEntry,
+ HashEntry {
+
+ private static final Logger logger = LogService.getLogger();
+
+ /**
+ * Whether to disable last access time update when a put occurs. The default
+ * is false (enable last access time update on put). To disable it, set the
+ * 'gemfire.disableAccessTimeUpdateOnPut' system property.
+ */
+ protected static final boolean DISABLE_ACCESS_TIME_UPDATE_ON_PUT = Boolean
+ .getBoolean("gemfire.disableAccessTimeUpdateOnPut");
+
+ /*
+ * Flags for a Region Entry.
+ * These flags are stored in the msb of the long used to also store the lastModicationTime.
+ */
+ private static final long VALUE_RESULT_OF_SEARCH = 0x01L<<56;
+ private static final long UPDATE_IN_PROGRESS = 0x02L<<56;
+ private static final long TOMBSTONE_SCHEDULED = 0x04L<<56;
+ private static final long LISTENER_INVOCATION_IN_PROGRESS = 0x08L<<56;
+ /** used for LRUEntry instances. */
+ protected static final long RECENTLY_USED = 0x10L<<56;
+ /** used for LRUEntry instances. */
+ protected static final long EVICTED = 0x20L<<56;
+ /**
+ * Set if the entry is being used by a transactions.
+ * Some features (eviction and expiration) will not modify an entry when a tx is using it
+ * to prevent the tx to fail do to conflict.
+ */
+ protected static final long IN_USE_BY_TX = 0x40L<<56;
+
+
+ protected static final long MARKED_FOR_EVICTION = 0x80L<<56;
+// public Exception removeTrace; // debugging hot loop in AbstractRegionMap.basicPut()
+
+ protected AbstractRegionEntry(RegionEntryContext context,
+ @Retained(ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE) Object value) {
+
+ setValue(context,this.prepareValueForCache(context, value, false),false);
+// setLastModified(System.currentTimeMillis()); [bruce] this must be set later so we can use ==0 to know this is a new entry in checkForConflicts
+ }
+
+ /////////////////////////////////////////////////////////////////////
+ ////////////////////////// instance methods /////////////////////////
+ /////////////////////////////////////////////////////////////////////
+
+ @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IMSE_DONT_CATCH_IMSE")
+ public boolean dispatchListenerEvents(final EntryEventImpl event) throws InterruptedException {
+ final LocalRegion rgn = event.getRegion();
+
+ if (event.callbacksInvoked()) {
+ return true;
+ }
+
+ // don't wait for certain events to reach the head of the queue before
+ // dispatching listeners. However, we must not notify the gateways for
+ // remote-origin ops out of order. Otherwise the other systems will have
+ // inconsistent content.
+
+ event.setCallbacksInvokedByCurrentThread();
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("{} dispatching event {}", this, event);
+ }
+ // All the following code that sets "thr" is to workaround
+ // spurious IllegalMonitorStateExceptions caused by JVM bugs.
+ try {
+ // call invokeCallbacks while synced on RegionEntry
+ event.invokeCallbacks(rgn, event.inhibitCacheListenerNotification(), false);
+ return true;
+
+ } finally {
+ if (isRemoved() && !isTombstone() && !event.isEvicted()) {
+ // Phase 2 of region entry removal is done here. The first phase is done
+ // by the RegionMap. It is unclear why this code is needed. ARM destroy
+ // does this also and we are now doing it as phase3 of the ARM destroy.
+ removePhase2();
+ rgn.getRegionMap().removeEntry(event.getKey(), this, true, event, rgn, rgn.getIndexUpdater());
+ }
+ }
+ }
+
+ public long getLastAccessed() throws InternalStatisticsDisabledException {
+ throw new InternalStatisticsDisabledException();
+ }
+
+ public long getHitCount() throws InternalStatisticsDisabledException {
+ throw new InternalStatisticsDisabledException();
+ }
+
+ public long getMissCount() throws InternalStatisticsDisabledException {
+ throw new InternalStatisticsDisabledException();
+ }
+
+ protected void setLastModified(long lastModified) {
+ _setLastModified(lastModified);
+ }
+
+ public void txDidDestroy(long currTime) {
+ setLastModified(currTime);
+ }
+
+ public final void updateStatsForPut(long lastModifiedTime) {
+ setLastModified(lastModifiedTime);
+ }
+
+ public void setRecentlyUsed() {
+ // do nothing by default; only needed for LRU
+ }
+
+ public void updateStatsForGet(boolean hit, long time) {
+ // nothing needed
+ }
+
+ public void resetCounts() throws InternalStatisticsDisabledException {
+ throw new InternalStatisticsDisabledException();
+ }
+
+ public void _removePhase1() {
+ _setValue(Token.REMOVED_PHASE1);
+ // debugging for 38467 (hot thread in ARM.basicUpdate)
+// this.removeTrace = new Exception("stack trace for thread " + Thread.currentThread());
+ }
+ public void removePhase1(LocalRegion r, boolean isClear) throws RegionClearedException {
+ _removePhase1();
+ }
+
+ public void removePhase2() {
+ _setValue(Token.REMOVED_PHASE2);
+// this.removeTrace = new Exception("stack trace for thread " + Thread.currentThread());
+ }
+
+ public void makeTombstone(LocalRegion r, VersionTag version) throws RegionClearedException {
+ assert r.getVersionVector() != null;
+ assert version != null;
+ if (r.getServerProxy() == null &&
+ r.getVersionVector().isTombstoneTooOld(version.getMemberID(), version.getRegionVersion())) {
+ // distributed gc with higher vector version preempts this operation
+ if (!isTombstone()) {
+ setValue(r, Token.TOMBSTONE);
+ r.incTombstoneCount(1);
+ }
+ r.getRegionMap().removeTombstone(this, version, false, true);
+ } else {
+ if (isTombstone()) {
+ // unschedule the old tombstone
+ r.unscheduleTombstone(this);
+ }
+ setRecentlyUsed();
+ boolean newEntry = (getValueAsToken() == Token.REMOVED_PHASE1);
+ setValue(r, Token.TOMBSTONE);
+ r.scheduleTombstone(this, version);
+ if (newEntry) {
+ // bug #46631 - entry count is decremented by scheduleTombstone but this is a new entry
+ r.getCachePerfStats().incEntryCount(1);
+ }
+ }
+ }
+
+
+ @Override
+ public void setValueWithTombstoneCheck(@Unretained Object v, EntryEvent e) throws RegionClearedException {
+ if (v == Token.TOMBSTONE) {
+ makeTombstone((LocalRegion)e.getRegion(), ((EntryEventImpl)e).getVersionTag());
+ } else {
+ setValue((LocalRegion)e.getRegion(), v, (EntryEventImpl)e);
+ }
+ }
+
+ /**
+ * Return true if the object is removed.
+ *
+ * TODO this method does NOT return true if the object
+ * is Token.DESTROYED. dispatchListenerEvents relies on that
+ * fact to avoid removing destroyed tokens from the map.
+ * We should refactor so that this method calls Token.isRemoved,
+ * and places that don't want a destroyed Token can explicitly check
+ * for a DESTROY token.
+ */
+ public final boolean isRemoved() {
+ Token o = getValueAsToken();
+ return (o == Token.REMOVED_PHASE1) || (o == Token.REMOVED_PHASE2) || (o == Token.TOMBSTONE);
+ }
+
+ public final boolean isDestroyedOrRemoved() {
+ return Token.isRemoved(getValueAsToken());
+ }
+
+ public final boolean isDestroyedOrRemovedButNotTombstone() {
+ Token o = getValueAsToken();
+ return o == Token.DESTROYED || o == Token.REMOVED_PHASE1 || o == Token.REMOVED_PHASE2;
+ }
+
+ public final boolean isTombstone() {
+ return getValueAsToken() == Token.TOMBSTONE;
+ }
+
+ public final boolean isRemovedPhase2() {
+ return getValueAsToken() == Token.REMOVED_PHASE2;
+ }
+
+ public boolean fillInValue(LocalRegion region,
+ @Retained(ABSTRACT_REGION_ENTRY_FILL_IN_VALUE) InitialImageOperation.Entry dst,
+ ByteArrayDataInput in,
+ DM mgr)
+ {
+ dst.setSerialized(false); // starting default value
+
+ @Retained(ABSTRACT_REGION_ENTRY_FILL_IN_VALUE) final Object v;
+ if (isTombstone()) {
+ v = Token.TOMBSTONE;
+ } else {
+ v = getValue(region); // OFFHEAP: need to incrc, copy bytes, decrc
+ if (v == null) {
+ return false;
+ }
+ }
+
+ final boolean isEagerDeserialize = dst.isEagerDeserialize();
+ if (isEagerDeserialize) {
+ dst.clearEagerDeserialize();
+ }
+ dst.setLastModified(mgr, getLastModified()); // fix for bug 31059
+ if (v == Token.INVALID) {
+ dst.setInvalid();
+ }
+ else if (v == Token.LOCAL_INVALID) {
+ dst.setLocalInvalid();
+ }
+ else if (v == Token.TOMBSTONE) {
+ dst.setTombstone();
+ }
+ else if (v instanceof CachedDeserializable) {
+ // don't serialize here if it is not already serialized
+// if(v instanceof ByteSource && CachedDeserializableFactory.preferObject()) {
+// // For SQLFire we prefer eager deserialized
+// dst.setEagerDeserialize();
+// }
+
+ if (v instanceof StoredObject && !((StoredObject) v).isSerialized()) {
+ dst.value = ((StoredObject) v).getDeserializedForReading();
+ } else {
+ /*if (v instanceof ByteSource && CachedDeserializableFactory.preferObject()) {
+ dst.value = v;
+ } else */ {
+ Object tmp = ((CachedDeserializable) v).getValue();
+ if (tmp instanceof byte[]) {
+ byte[] bb = (byte[]) tmp;
+ dst.value = bb;
+ } else {
+ try {
+ HeapDataOutputStream hdos = new HeapDataOutputStream(
+ Version.CURRENT);
+ BlobHelper.serializeTo(tmp, hdos);
+ hdos.trim();
+ dst.value = hdos;
+ } catch (IOException e) {
+ RuntimeException e2 = new IllegalArgumentException(
+ LocalizedStrings.AbstractRegionEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING
+ .toLocalizedString());
+ e2.initCause(e);
+ throw e2;
+ }
+ }
+ dst.setSerialized(true);
+ }
+ }
+ }
+ else if (v instanceof byte[]) {
+ dst.value = v;
+ }
+ else {
+ Object preparedValue = v;
+ if (preparedValue != null) {
+ preparedValue = prepareValueForGII(preparedValue);
+ if (preparedValue == null) {
+ return false;
+ }
+ }
+ if (CachedDeserializableFactory.preferObject()) {
+ dst.value = preparedValue;
+ dst.setEagerDeserialize();
+ }
+ else {
+ try {
+ HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
+ BlobHelper.serializeTo(preparedValue, hdos);
+ hdos.trim();
+ dst.value = hdos;
+ dst.setSerialized(true);
+ } catch (IOException e) {
+ RuntimeException e2 = new IllegalArgumentException(LocalizedStrings.AbstractRegionEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING.toLocalizedString());
+ e2.initCause(e);
+ throw e2;
+ }
+ }
+ }
+ return true;
+ }
+
+ /**
+ * To fix bug 49901 if v is a GatewaySenderEventImpl then make
+ * a heap copy of it if it is offheap.
+ * @return the value to provide to the gii request; null if no value should be provided.
+ */
+ public static Object prepareValueForGII(Object v) {
+ assert v != null;
+ if (v instanceof GatewaySenderEventImpl) {
+ return ((GatewaySenderEventImpl) v).makeHeapCopyIfOffHeap();
+ } else {
+ return v;
+ }
+ }
+
+ public boolean isOverflowedToDisk(LocalRegion r, DistributedRegion.DiskPosition dp) {
+ return false;
+ }
+
+ @Override
+ public Object getValue(RegionEntryContext context) {
+ ReferenceCountHelper.createReferenceCountOwner();
+ @Retained Object result = _getValueRetain(context, true);
+ //Asif: If the thread is an Index Creation Thread & the value obtained is
+ //Token.REMOVED , we can skip synchronization block. This is required to prevent
+ // the dead lock caused if an Index Update Thread has gone into a wait holding the
+ // lock of the Entry object. There should not be an issue if the Index creation thread
+ // gets the temporary value of token.REMOVED as the correct value will get indexed
+ // by the Index Update Thread , once the index creation thread has exited.
+ // Part of Bugfix # 33336
+// if ((result == Token.REMOVED_PHASE1 || result == Token.REMOVED_PHASE2) && !r.isIndexCreationThread()) {
+// synchronized (this) {
+// result = _getValue();
+// }
+// }
+
+ if (Token.isRemoved(result)) {
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ return null;
+ } else {
+ result = OffHeapHelper.copyAndReleaseIfNeeded(result); // sqlf does not dec ref count in this call
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ setRecentlyUsed();
+ return result;
+ }
+ }
+
+ @Override
+ @Retained
+ public Object getValueRetain(RegionEntryContext context) {
+ @Retained Object result = _getValueRetain(context, true);
+ if (Token.isRemoved(result)) {
+ return null;
+ } else {
+ setRecentlyUsed();
+ return result;
+ }
+ }
+
+ @Override
+ @Released
+ public void setValue(RegionEntryContext context, @Unretained Object value) throws RegionClearedException {
+ // @todo darrel: This will mark new entries as being recently used
+ // It might be better to only mark them when they are modified.
+ // Or should we only mark them on reads?
+ setValue(context,value,true);
+ }
+
+ @Override
+ public void setValue(RegionEntryContext context, Object value, EntryEventImpl event) throws RegionClearedException {
+ setValue(context,value);
+ }
+
+ @Released
+ protected void setValue(RegionEntryContext context, @Unretained Object value, boolean recentlyUsed) {
+ _setValue(value);
+ if (value != null && context != null && (this instanceof OffHeapRegionEntry)
+ && context instanceof LocalRegion && ((LocalRegion)context).isThisRegionBeingClosedOrDestroyed()) {
+ ((OffHeapRegionEntry)this).release();
+ ((LocalRegion)context).checkReadiness();
+ }
+ if (recentlyUsed) {
+ setRecentlyUsed();
+ }
+ }
+
+ /**
+ * This method determines if the value is in a compressed representation and decompresses it if it is.
+ *
+ * @param context the values context.
+ * @param value a region entry value.
+ *
+ * @return the decompressed form of the value parameter.
+ */
+ static Object decompress(RegionEntryContext context,Object value) {
+ if(isCompressible(context, value)) {
+ long time = context.getCachePerfStats().startDecompression();
+ value = EntryEventImpl.deserialize(context.getCompressor().decompress((byte[]) value));
+ context.getCachePerfStats().endDecompression(time);
+ }
+
+ return value;
+ }
+
+ static protected Object compress(RegionEntryContext context,Object value) {
+ return compress(context, value, null);
+ }
+
+ /**
+ * This method determines if the value is compressible and compresses it if it is.
+ *
+ * @param context the values context.
+ * @param value a region entry value.
+ *
+ * @return the compressed form of the value parameter.
+ */
+ static protected Object compress(RegionEntryContext context,Object value, EntryEventImpl event) {
+ if(isCompressible(context, value)) {
+ long time = context.getCachePerfStats().startCompression();
+ byte[] serializedValue;
+ if (event != null && event.getCachedSerializedNewValue() != null) {
+ serializedValue = event.getCachedSerializedNewValue();
+ if (value instanceof CachedDeserializable) {
+ CachedDeserializable cd = (CachedDeserializable) value;
+ if (!(cd.getValue() instanceof byte[])) {
+ // The cd now has the object form so use the cached serialized form in a new cd.
+ // This serialization is much cheaper than reserializing the object form.
+ serializedValue = EntryEventImpl.serialize(CachedDeserializableFactory.create(serializedValue));
+ } else {
+ serializedValue = EntryEventImpl.serialize(cd);
+ }
+ }
+ } else {
+ serializedValue = EntryEventImpl.serialize(value);
+ if (event != null && !(value instanceof byte[])) {
+ // See if we can cache the serialized new value in the event.
+ // If value is a byte[] then we don't cache it since it is not serialized.
+ if (value instanceof CachedDeserializable) {
+ // For a CacheDeserializable we want to only cache the wrapped value;
+ // not the serialized CacheDeserializable.
+ CachedDeserializable cd = (CachedDeserializable) value;
+ Object cdVal = cd.getValue();
+ if (cdVal instanceof byte[]) {
+ event.setCachedSerializedNewValue((byte[])cdVal);
+ }
+ } else {
+ event.setCachedSerializedNewValue(serializedValue);
+ }
+ }
+ }
+ value = context.getCompressor().compress(serializedValue);
+ context.getCachePerfStats().endCompression(time, serializedValue.length, ((byte []) value).length);
+ }
+
+ return value;
+ }
+
+ private static byte[] compressBytes(RegionEntryContext context, byte[] uncompressedBytes) {
+ byte[] result = uncompressedBytes;
+ if (isCompressible(context, uncompressedBytes)) {
+ long time = context.getCachePerfStats().startCompression();
+ result = context.getCompressor().compress(uncompressedBytes);
+ context.getCachePerfStats().endCompression(time, uncompressedBytes.length, result.length);
+ }
+ return result;
+ }
+
+
+ @Retained
+ public final Object getValueInVM(RegionEntryContext context) {
+ ReferenceCountHelper.createReferenceCountOwner();
+ @Retained Object v = _getValueRetain(context, true);
+
+ if (v == null) { // should only be possible if disk entry
+ v = Token.NOT_AVAILABLE;
+ }
+ @Retained Object result = OffHeapHelper.copyAndReleaseIfNeeded(v); // TODO OFFHEAP keep it offheap?
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ return result;
+ }
+
+ @Retained
+ public Object getValueInVMOrDiskWithoutFaultIn(LocalRegion owner) {
+ return getValueInVM(owner);
+ }
+
+ @Override
+ @Retained
+ public Object getValueOffHeapOrDiskWithoutFaultIn(LocalRegion owner) {
+ @Retained Object result = _getValueRetain(owner, true);
+// if (result instanceof ByteSource) {
+// // If the ByteSource contains a Delta or ListOfDelta then we want to deserialize it
+// Object deserVal = ((CachedDeserializable)result).getDeserializedForReading();
+// if (deserVal != result) {
+// OffHeapHelper.release(result);
+// result = deserVal;
+// }
+// }
+ return result;
+ }
+
+ public Object getValueOnDisk(LocalRegion r)
+ throws EntryNotFoundException
+ {
+ throw new IllegalStateException(LocalizedStrings.AbstractRegionEntry_CANNOT_GET_VALUE_ON_DISK_FOR_A_REGION_THAT_DOES_NOT_ACCESS_THE_DISK.toLocalizedString());
+ }
+
+ public Object getSerializedValueOnDisk(final LocalRegion r)
+ throws EntryNotFoundException
+ {
+ throw new IllegalStateException(LocalizedStrings.AbstractRegionEntry_CANNOT_GET_VALUE_ON_DISK_FOR_A_REGION_THAT_DOES_NOT_ACCESS_THE_DISK.toLocalizedString());
+ }
+
+ public Object getValueOnDiskOrBuffer(LocalRegion r)
+ throws EntryNotFoundException
+ {
+ throw new IllegalStateException(LocalizedStrings.AbstractRegionEntry_CANNOT_GET_VALUE_ON_DISK_FOR_A_REGION_THAT_DOES_NOT_ACCESS_THE_DISK.toLocalizedString());
+ // @todo darrel if value is Token.REMOVED || Token.DESTROYED throw EntryNotFoundException
+ }
+
+ public final boolean initialImagePut(final LocalRegion region,
+ final long lastModifiedTime,
+ Object newValue,
+ boolean wasRecovered,
+ boolean versionTagAccepted) throws RegionClearedException
+ {
+ // note that the caller has already write synced this RegionEntry
+ return initialImageInit(region, lastModifiedTime, newValue, this.isTombstone(), wasRecovered, versionTagAccepted);
+ }
+
+ public boolean initialImageInit(final LocalRegion region,
+ final long lastModifiedTime,
+ final Object newValue,
+ final boolean create,
+ final boolean wasRecovered,
+ final boolean versionTagAccepted) throws RegionClearedException
+ {
+ // note that the caller has already write synced this RegionEntry
+ boolean result = false;
+ // if it has been destroyed then don't do anything
+ Token vTok = getValueAsToken();
+ if (versionTagAccepted || create || (vTok != Token.DESTROYED || vTok != Token.TOMBSTONE)) { // OFFHEAP noop
+ Object newValueToWrite = newValue;
+ boolean putValue = versionTagAccepted || create
+ || (newValueToWrite != Token.LOCAL_INVALID
+ && (wasRecovered || (vTok == Token.LOCAL_INVALID))); // OFFHEAP noop
+
+ if (region.isUsedForPartitionedRegionAdmin() && newValueToWrite instanceof CachedDeserializable) {
+ // Special case for partitioned region meta data
+ // We do not need the RegionEntry on this case.
+ // Because the pr meta data region will not have an LRU.
+ newValueToWrite = ((CachedDeserializable) newValueToWrite).getDeserializedValue(region, null);
+ if (!create && newValueToWrite instanceof Versionable) {
+ @Retained @Released final Object oldValue = getValueInVM(region); // Heap value should always be deserialized at this point // OFFHEAP will not be deserialized
+ try {
+ // BUGFIX for 35029. If oldValue is null the newValue should be put.
+ if(oldValue == null) {
+ putValue = true;
+ }
+ else if (oldValue instanceof Versionable) {
+ Versionable nv = (Versionable) newValueToWrite;
+ Versionable ov = (Versionable) oldValue;
+ putValue = nv.isNewerThan(ov);
+ }
+ } finally {
+ OffHeapHelper.release(oldValue);
+ }
+ }
+ }
+
+ if (putValue) {
+ // change to INVALID if region itself has been invalidated,
+ // and current value is recovered
+ if (create || versionTagAccepted) {
+ // At this point, since we now always recover from disk first,
+ // we only care about "isCreate" since "isRecovered" is impossible
+ // if we had a regionInvalidate or regionClear
+ ImageState imageState = region.getImageState();
+ // this method is called during loadSnapshot as well as getInitialImage
+ if (imageState.getRegionInvalidated()) {
+ if (newValueToWrite != Token.TOMBSTONE) {
+ newValueToWrite = Token.INVALID;
+ }
+ }
+ else if (imageState.getClearRegionFlag()) {
+ boolean entryOK = false;
+ RegionVersionVector rvv = imageState.getClearRegionVersionVector();
+ if (rvv != null) { // a filtered clear
+ VersionSource id = getVersionStamp().getMemberID();
+ if (id == null) {
+ id = region.getVersionMember();
+ }
+ if (!rvv.contains(id, getVersionStamp().getRegionVersion())) {
+ entryOK = true;
+ }
+ }
+ if (!entryOK) {
+ //Asif: If the region has been issued cleared during
+ // the GII , then those entries loaded before this one would have
+ // been cleared from the Map due to clear operation & for the
+ // currententry whose key may have escaped the clearance , will be
+ // cleansed by the destroy token.
+ newValueToWrite = Token.DESTROYED;
+ imageState.addDestroyedEntry(this.getKey());
+ throw new RegionClearedException(LocalizedStrings.AbstractRegionEntry_DURING_THE_GII_PUT_OF_ENTRY_THE_REGION_GOT_CLEARED_SO_ABORTING_THE_OPERATION.toLocalizedString());
+ }
+ }
+ }
+ setValue(region, this.prepareValueForCache(region, newValueToWrite, false));
+ result = true;
+
+ if (newValueToWrite != Token.TOMBSTONE){
+ if (create) {
+ region.getCachePerfStats().incCreates();
+ }
+ region.updateStatsForPut(this, lastModifiedTime, false);
+ }
+
+ if (logger.isTraceEnabled()) {
+ if (newValueToWrite instanceof CachedDeserializable) {
+ logger.trace("ProcessChunk: region={}; put a CachedDeserializable ({},{})",
+ region.getFullPath(), getKey(),((CachedDeserializable)newValueToWrite).getStringForm());
+ }
+ else {
+ logger.trace("ProcessChunk: region={}; put({},{})", region.getFullPath(), getKey(), StringUtils.forceToString(newValueToWrite));
+ }
+ }
+ }
+ }
+ return result;
+ }
+
+ /**
+ * @throws EntryNotFoundException if expectedOldValue is
+ * not null and is not equal to current value
+ */
+ @Released
+ public final boolean destroy(LocalRegion region,
+ EntryEventImpl event,
+ boolean inTokenMode,
+ boolean cacheWrite,
+ @Unretained Object expectedOldValue,
+ boolean forceDestroy,
+ boolean removeRecoveredEntry)
+ throws CacheWriterException,
+ EntryNotFoundException,
+ TimeoutException,
+ RegionClearedException {
+ boolean proceed = false;
+ {
+ // A design decision was made to not retrieve the old value from the disk
+ // if the entry has been evicted to only have the CacheListener afterDestroy
+ // method ignore it. We don't want to pay the performance penalty. The
+ // getValueInVM method does not retrieve the value from disk if it has been
+ // evicted. Instead, it uses the NotAvailable token.
+ //
+ // If the region is a WAN queue region, the old value is actually used by the
+ // afterDestroy callback on a secondary. It is not needed on a primary.
+ // Since the destroy that sets WAN_QUEUE_TOKEN always originates on the primary
+ // we only pay attention to WAN_QUEUE_TOKEN if the event is originRemote.
+ //
+ // :ezoerner:20080814 We also read old value from disk or buffer
+ // in the case where there is a non-null expectedOldValue
+ // see PartitionedRegion#remove(Object key, Object value)
+ ReferenceCountHelper.skipRefCountTracking();
+ @Retained @Released Object curValue = _getValueRetain(region, true);
+ ReferenceCountHelper.unskipRefCountTracking();
+ try {
+ if (curValue == null) curValue = Token.NOT_AVAILABLE;
+
+ if (curValue == Token.NOT_AVAILABLE) {
+ // In some cases we need to get the current value off of disk.
+
+ // if the event is transmitted during GII and has an old value, it was
+ // the state of the transmitting cache's entry & should be used here
+ if (event.getCallbackArgument() != null
+ && event.getCallbackArgument().equals(RegionQueue.WAN_QUEUE_TOKEN)
+ && event.isOriginRemote()) { // check originRemote for bug 40508
+ //curValue = getValue(region); can cause deadlock if GII is occurring
+ curValue = getValueOnDiskOrBuffer(region);
+ }
+ else {
+ FilterProfile fp = region.getFilterProfile();
+ // rdubey: Old value also required for SqlfIndexManager.
+ if (fp != null && ((fp.getCqCount() > 0) || expectedOldValue != null
+ || event.getRegion().getIndexUpdater() != null)) {
+ //curValue = getValue(region); can cause deadlock will fault in the value
+ // and will confuse LRU. rdubey.
+ curValue = getValueOnDiskOrBuffer(region);
+ }
+ }
+ }
+
+ if (expectedOldValue != null) {
+ if (!checkExpectedOldValue(expectedOldValue, curValue, region)) {
+ throw new EntryNotFoundException(
+ LocalizedStrings.AbstractRegionEntry_THE_CURRENT_VALUE_WAS_NOT_EQUAL_TO_EXPECTED_VALUE.toLocalizedString());
+ }
+ }
+
+ if (inTokenMode && event.hasOldValue()) {
+ proceed = true;
+ }
+ else {
+ proceed = event.setOldValue(curValue, curValue instanceof GatewaySenderEventImpl) || removeRecoveredEntry
+ || forceDestroy || region.getConcurrencyChecksEnabled() // fix for bug #47868 - create a tombstone
+ || (event.getOperation() == Operation.REMOVE // fix for bug #42242
+ && (curValue == null || curValue == Token.LOCAL_INVALID
+ || curValue == Token.INVALID));
+ }
+ } finally {
+ OffHeapHelper.releaseWithNoTracking(curValue);
+ }
+ } // end curValue block
+
+ if (proceed) {
+ //Generate the version tag if needed. This method should only be
+ //called if we are in fact going to destroy the entry, so it must be
+ //after the entry not found exception above.
+ if(!removeRecoveredEntry) {
+ region.generateAndSetVersionTag(event, this);
+ }
+ if (cacheWrite) {
+ region.cacheWriteBeforeDestroy(event, expectedOldValue);
+ if (event.getRegion().getServerProxy() != null) { // server will return a version tag
+ // update version information (may throw ConcurrentCacheModificationException)
+ VersionStamp stamp = getVersionStamp();
+ if (stamp != null) {
+ stamp.processVersionTag(event);
+ }
+ }
+ }
+ region.recordEvent(event);
+ // don't do index maintenance on a destroy if the value in the
+ // RegionEntry (the old value) is invalid
+ if (!region.isProxy() && !isInvalid()) {
+ IndexManager indexManager = region.getIndexManager();
+ if (indexManager != null) {
+ try {
+ if(isValueNull()) {
+ @Released Object value = getValueOffHeapOrDiskWithoutFaultIn(region);
+ try {
+ _setValue(prepareValueForCache(region, value, false));
+ if (value != null && region != null && (this instanceof OffHeapRegionEntry) && region.isThisRegionBeingClosedOrDestroyed()) {
+ ((OffHeapRegionEntry)this).release();
+ region.checkReadiness();
+ }
+ } finally {
+ OffHeapHelper.release(value);
+ }
+ }
+ indexManager.updateIndexes(this,
+ IndexManager.REMOVE_ENTRY,
+ IndexProtocol.OTHER_OP);
+ }
+ catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }
+
+ boolean removeEntry = false;
+ VersionTag v = event.getVersionTag();
+ if (region.concurrencyChecksEnabled && !removeRecoveredEntry
+ && !event.isFromRILocalDestroy()) { // bug #46780, don't retain tombstones for entries destroyed for register-interest
+ // Destroy will write a tombstone instead
+ if (v == null || !v.hasValidVersion()) {
+ // localDestroy and eviction and ops received with no version tag
+ // should create a tombstone using the existing version stamp, as should
+ // (bug #45245) responses from servers that do not have valid version information
+ VersionStamp stamp = this.getVersionStamp();
+ if (stamp != null) { // proxy has no stamps
+ v = stamp.asVersionTag();
+ event.setVersionTag(v);
+ }
+ }
+ removeEntry = (v == null) || !v.hasValidVersion();
+ } else {
+ removeEntry = true;
+ }
+
+ // See #47887, we do not insert a tombstone for evicted HDFS
+ // entries since the value is still present in HDFS
+ // Check if we have to evict or just do destroy.
+ boolean forceRemoveEntry =
+ (event.isEviction() || event.isExpiration())
+ && event.getRegion().isUsedForPartitionedRegionBucket()
+ && event.getRegion().getPartitionedRegion().isHDFSRegion();
+
+ if (removeEntry || forceRemoveEntry) {
+ boolean isThisTombstone = isTombstone();
+ if(inTokenMode && !event.getOperation().isEviction()) {
+ setValue(region, Token.DESTROYED);
+ } else {
+ removePhase1(region, false);
+ }
+ if (isThisTombstone) {
+ region.unscheduleTombstone(this);
+ }
+ } else {
+ makeTombstone(region, v);
+ }
+
+ return true;
+ }
+ else {
+ return false;
+ }
+ }
+
+
+
+ static boolean checkExpectedOldValue(@Unretained Object expectedOldValue, @Unretained Object actualValue, LocalRegion lr) {
+ if (Token.isInvalid(expectedOldValue)) {
+ return (actualValue == null) || Token.isInvalid(actualValue);
+ } else {
+ boolean isCompressedOffHeap = lr.getAttributes().getOffHeap() && lr.getAttributes().getCompressor() != null;
+ return checkEquals(expectedOldValue, actualValue, isCompressedOffHeap);
+ }
+ }
+
+ private static boolean basicEquals(Object v1, Object v2) {
+ if (v2 != null) {
+ if (v2.getClass().isArray()) {
+ // fix for 52093
+ if (v2 instanceof byte[]) {
+ if (v1 instanceof byte[]) {
+ return Arrays.equals((byte[])v2, (byte[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof Object[]) {
+ if (v1 instanceof Object[]) {
+ return Arrays.deepEquals((Object[])v2, (Object[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof int[]) {
+ if (v1 instanceof int[]) {
+ return Arrays.equals((int[])v2, (int[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof long[]) {
+ if (v1 instanceof long[]) {
+ return Arrays.equals((long[])v2, (long[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof boolean[]) {
+ if (v1 instanceof boolean[]) {
+ return Arrays.equals((boolean[])v2, (boolean[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof short[]) {
+ if (v1 instanceof short[]) {
+ return Arrays.equals((short[])v2, (short[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof char[]) {
+ if (v1 instanceof char[]) {
+ return Arrays.equals((char[])v2, (char[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof float[]) {
+ if (v1 instanceof float[]) {
+ return Arrays.equals((float[])v2, (float[])v1);
+ } else {
+ return false;
+ }
+ } else if (v2 instanceof double[]) {
+ if (v1 instanceof double[]) {
+ return Arrays.equals((double[])v2, (double[])v1);
+ } else {
+ return false;
+ }
+ }
+ // fall through and call equals method
+ }
+ return v2.equals(v1);
+ } else {
+ return v1 == null;
+ }
+ }
+
+ static boolean checkEquals(@Unretained Object v1, @Unretained Object v2, boolean isCompressedOffHeap) {
+ // need to give PdxInstance#equals priority
+ if (v1 instanceof PdxInstance) {
+ return checkPdxEquals((PdxInstance)v1, v2);
+ } else if (v2 instanceof PdxInstance) {
+ return checkPdxEquals((PdxInstance)v2, v1);
+ } else if (v1 instanceof OffHeapCachedDeserializable) {
+ return checkOffHeapEquals((OffHeapCachedDeserializable)v1, v2);
+ } else if (v2 instanceof OffHeapCachedDeserializable) {
+ return checkOffHeapEquals((OffHeapCachedDeserializable)v2, v1);
+ } else if (v1 instanceof CachedDeserializable) {
+ return checkCDEquals((CachedDeserializable)v1, v2, isCompressedOffHeap);
+ } else if (v2 instanceof CachedDeserializable) {
+ return checkCDEquals((CachedDeserializable)v2, v1, isCompressedOffHeap);
+ } else {
+ return basicEquals(v1, v2);
+ }
+ }
+ private static boolean checkOffHeapEquals(@Unretained OffHeapCachedDeserializable cd, @Unretained Object obj) {
+ if (cd.isSerializedPdxInstance()) {
+ PdxInstance pi = InternalDataSerializer.readPdxInstance(cd.getSerializedValue(), GemFireCacheImpl.getForPdx("Could not check value equality"));
+ return checkPdxEquals(pi, obj);
+ }
+ if (obj instanceof OffHeapCachedDeserializable) {
+ return cd.checkDataEquals((OffHeapCachedDeserializable)obj);
+ } else {
+ byte[] serializedObj;
+ if (obj instanceof CachedDeserializable) {
+ if (!cd.isSerialized()) {
+ if (obj instanceof StoredObject && !((StoredObject) obj).isSerialized()) {
+ // both are byte[]
+ // obj must be DataAsAddress since it was not OffHeapCachedDeserializable
+ // so its byte[] will be small.
+ byte[] objBytes = (byte[]) ((StoredObject) obj).getDeserializedForReading();
+ return cd.checkDataEquals(objBytes);
+ } else {
+ return false;
+ }
+ }
+ serializedObj = ((CachedDeserializable) obj).getSerializedValue();
+ } else if (obj instanceof byte[]) {
+ if (cd.isSerialized()) {
+ return false;
+ }
+ serializedObj = (byte[]) obj;
+ } else {
+ if (!cd.isSerialized()) {
+ return false;
+ }
+ if (obj == null || obj == Token.NOT_AVAILABLE
+ || Token.isInvalidOrRemoved(obj)) {
+ return false;
+ }
+ serializedObj = EntryEventImpl.serialize(obj);
+ }
+ return cd.checkDataEquals(serializedObj);
+ }
+ }
+
+ private static boolean checkCDEquals(CachedDeserializable cd, Object obj, boolean isCompressedOffHeap) {
+ if (cd instanceof StoredObject && !((StoredObject) cd).isSerialized()) {
+ // cd is an actual byte[].
+ byte[] ba2;
+ if (obj instanceof StoredObject) {
+ if (!((StoredObject) obj).isSerialized()) {
+ return false;
+ }
+ ba2 = (byte[]) ((StoredObject) obj).getDeserializedForReading();
+ } else if (obj instanceof byte[]) {
+ ba2 = (byte[]) obj;
+ } else {
+ return false;
+ }
+ byte[] ba1 = (byte[]) cd.getDeserializedForReading();
+ return Arrays.equals(ba1, ba2);
+ }
+ Object cdVal = cd.getValue();
+ if (cdVal instanceof byte[]) {
+ byte[] cdValBytes = (byte[])cdVal;
+ PdxInstance pi = InternalDataSerializer.readPdxInstance(cdValBytes, GemFireCacheImpl.getForPdx("Could not check value equality"));
+ if (pi != null) {
+ return checkPdxEquals(pi, obj);
+ }
+ if (isCompressedOffHeap) { // fix for bug 52248
+ byte[] serializedObj;
+ if (obj instanceof CachedDeserializable) {
+ serializedObj = ((CachedDeserializable) obj).getSerializedValue();
+ } else {
+ serializedObj = EntryEventImpl.serialize(obj);
+ }
+ return Arrays.equals(cdValBytes, serializedObj);
+ } else {
+ /**
+ * To be more compatible with previous releases do not compare the serialized forms here.
+ * Instead deserialize and call the equals method.
+ */
+ Object deserializedObj;
+ if (obj instanceof CachedDeserializable) {
+ deserializedObj =((CachedDeserializable) obj).getDeserializedForReading();
+ } else {
+ if (obj == null || obj == Token.NOT_AVAILABLE
+ || Token.isInvalidOrRemoved(obj)) {
+ return false;
+ }
+ // TODO OPTIMIZE: Before serializing all of obj we could get the top
+ // level class name of cdVal and compare it to the top level class name of obj.
+ deserializedObj = obj;
+ }
+ return basicEquals(deserializedObj, cd.getDeserializedForReading());
+ }
+// boolean result = Arrays.equals((byte[])cdVal, serializedObj);
+// if (!result) {
+// try {
+// Object o1 = BlobHelper.deserializeBlob((byte[])cdVal);
+// Object o2 = BlobHelper.deserializeBlob(serializedObj);
+// SimpleMemoryAllocatorImpl.debugLog("checkCDEquals o1=<" + o1 + "> o2=<" + o2 + ">", false);
+// if (o1.equals(o2)) {
+// SimpleMemoryAllocatorImpl.debugLog("they are equal! a1=<" + Arrays.toString((byte[])cdVal) + "> a2=<" + Arrays.toString(serializedObj) + ">", false);
+// }
+// } catch (IOException e) {
+// // TODO Auto-generated catch block
+// e.printStackTrace();
+// } catch (ClassNotFoundException e) {
+// // TODO Auto-generated catch block
+// e.printStackTrace();
+// }
+// }
+// return result;
+ } else {
+ // prefer object form
+ if (obj instanceof CachedDeserializable) {
+ // TODO OPTIMIZE: Before deserializing all of obj we could get the top
+ // class name of cdVal and the top level class name of obj and compare.
+ obj = ((CachedDeserializable) obj).getDeserializedForReading();
+ }
+ return basicEquals(cdVal, obj);
+ }
+ }
+ /**
+ * This method fixes bug 43643
+ */
+ private static boolean checkPdxEquals(PdxInstance pdx, Object obj) {
+ if (!(obj instanceof PdxInstance)) {
+ // obj may be a CachedDeserializable in which case we want to convert it to a PdxInstance even if we are not readSerialized.
+ if (obj instanceof CachedDeserializable) {
+ if (obj instanceof StoredObject && !((StoredObject) obj).isSerialized()) {
+ // obj is actually a byte[] which will never be equal to a PdxInstance
+ return false;
+ }
+ Object cdVal = ((CachedDeserializable) obj).getValue();
+ if (cdVal instanceof byte[]) {
+ byte[] cdValBytes = (byte[]) cdVal;
+ PdxInstance pi = InternalDataSerializer.readPdxInstance(cdValBytes, GemFireCacheImpl.getForPdx("Could not check value equality"));
+ if (pi != null) {
+ return pi.equals(pdx);
+ } else {
+ // since obj is serialized as something other than pdx it must not equal our pdx
+ return false;
+ }
+ } else {
+ // remove the cd wrapper so that obj is the actual value we want to compare.
+ obj = cdVal;
+ }
+ }
+ if (obj.getClass().getName().equals(pdx.getClassName())) {
+ GemFireCacheImpl gfc = GemFireCacheImpl.getForPdx("Could not access Pdx registry");
+ if (gfc != null) {
+ PdxSerializer pdxSerializer;
+ if (obj instanceof PdxSerializable) {
+ pdxSerializer = null;
+ } else {
+ pdxSerializer = gfc.getPdxSerializer();
+ }
+ if (pdxSerializer != null || obj instanceof PdxSerializable) {
+ // try to convert obj to a PdxInstance
+ HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
+ try {
+ if (InternalDataSerializer.autoSerialized(obj, hdos) ||
+ InternalDataSerializer.writePdx(hdos, gfc, obj, pdxSerializer)) {
+ PdxInstance pi = InternalDataSerializer.readPdxInstance(hdos.toByteArray(), gfc);
+ if (pi != null) {
+ obj = pi;
+ }
+ }
+ } catch (IOException ignore) {
+ // we are not able to convert it so just fall through
+ } catch (PdxSerializationException ignore) {
+ // we are not able to convert it so just fall through
+ }
+ }
+ }
+ }
+ }
+ return basicEquals(obj, pdx);
+ }
+
+
+ /////////////////////////////////////////////////////////////
+ /////////////////////////// fields //////////////////////////
+ /////////////////////////////////////////////////////////////
+ // Do not add any instance fields to this class.
+ // Instead add them to LeafRegionEntry.cpp
+
+ public static class HashRegionEntryCreator implements
+ CustomEntryConcurrentHashMap.HashEntryCreator {
+
+ public HashEntry newEntry(final Object key, final int hash,
+ final HashEntry next, final Object value) {
+ final AbstractRegionEntry entry = (AbstractRegionEntry)value;
+ // if hash is already set then assert that the two should be same
+ final int entryHash = entry.getEntryHash();
+ if (hash == 0 || entryHash != 0) {
+ if (entryHash != hash) {
+ Assert.fail("unexpected mismatch of hash, expected=" + hash
+ + ", actual=" + entryHash + " for " + entry);
+ }
+ }
+ entry.setEntryHash(hash);
+ entry.setNextEntry(next);
+ return entry;
+ }
+
+ public int keyHashCode(final Object key, final boolean compareValues) {
+ return CustomEntryConcurrentHashMap.keyHash(key, compareValues);
+ }
+ };
+
+ public abstract Object getKey();
+
+ protected static boolean okToStoreOffHeap(Object v, AbstractRegionEntry e) {
+ if (v == null) return false;
+ if (Token.isInvalidOrRemoved(v)) return false;
+ if (v == Token.NOT_AVAILABLE) return false;
+ if (v instanceof DiskEntry.RecoveredEntry) return false; // The disk layer has special logic that ends up storing the nested value in the RecoveredEntry off heap
+ if (!(e instanceof OffHeapRegionEntry)) return false;
+ // TODO should we check for deltas here or is that a user error?
+ return true;
+ }
+
+ /**
+ * Default implementation. Override in subclasses with primitive keys
+ * to prevent creating an Object form of the key for each equality check.
+ */
+ @Override
+ public boolean isKeyEqual(Object k) {
+ return k.equals(getKey());
+ }
+
+ private static final long LAST_MODIFIED_MASK = 0x00FFFFFFFFFFFFFFL;
+
+ protected final void _setLastModified(long lastModifiedTime) {
+ if (lastModifiedTime < 0 || lastModifiedTime > LAST_MODIFIED_MASK) {
+ throw new IllegalStateException("Expected lastModifiedTime " + lastModifiedTime + " to be >= 0 and <= " + LAST_MODIFIED_MASK);
+ }
+ long storedValue;
+ long newValue;
+ do {
+ storedValue = getlastModifiedField();
+ newValue = storedValue & ~LAST_MODIFIED_MASK;
+ newValue |= lastModifiedTime;
+ } while (!compareAndSetLastModifiedField(storedValue, newValue));
+ }
+ protected abstract long getlastModifiedField();
+ protected abstract boolean compareAndSetLastModifiedField(long expectedValue, long newValue);
+ public final long getLastModified() {
+ return getlastModifiedField() & LAST_MODIFIED_MASK;
+ }
+ protected final boolean areAnyBitsSet(long bitMask) {
+ return ( getlastModifiedField() & bitMask ) != 0L;
+ }
+ /**
+ * Any bits in "bitMask" that are 1 will be set.
+ */
+ protected final void setBits(long bitMask) {
+ boolean done = false;
+ do {
+ long bits = getlastModifiedField();
+ long newBits = bits | bitMask;
+ if (bits == newBits) return;
+ done = compareAndSetLastModifiedField(bits, newBits);
+ } while(!done);
+ }
+ /**
+ * Any bits in "bitMask" that are 0 will be cleared.
+ */
+ protected final void clearBits(long bitMask) {
+ boolean done = false;
+ do {
+ long bits = getlastModifiedField();
+ long newBits = bits & bitMask;
+ if (bits == newBits) return;
+ done = compareAndSetLastModifiedField(bits, newBits);
+ } while(!done);
+ }
+
+ @Override
+ @Retained(ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE)
+ public Object prepareValueForCache(RegionEntryContext r,
+ @Retained(ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE) Object val,
+ boolean isEntryUpdate) {
+ return prepareValueForCache(r, val, null, isEntryUpdate);
+ }
+
+ @Override
+ @Retained(ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE)
+ public Object prepareValueForCache(RegionEntryContext r,
+ @Retained(ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE) Object val,
+ EntryEventImpl event, boolean isEntryUpdate) {
+ if (r != null && r.getOffHeap() && okToStoreOffHeap(val, this)) {
+ if (val instanceof StoredObject) {
+ // Check to see if val has the same compression settings as this region.
+ // The recursive calls in this section are safe because
+ // we only do it after copy the off-heap value to the heap.
+ // This is needed to fix bug 52057.
+ StoredObject soVal = (StoredObject) val;
+ assert !soVal.isCompressed();
+ if (r.getCompressor() != null) {
+ // val is uncompressed and we need a compressed value.
+ // So copy the off-heap value to the heap in a form that can be compressed.
+ byte[] valAsBytes = soVal.getValueAsHeapByteArray();
+ Object heapValue;
+ if (soVal.isSerialized()) {
+ heapValue = CachedDeserializableFactory.create(valAsBytes);
+ } else {
+ heapValue = valAsBytes;
+ }
+ return prepareValueForCache(r, heapValue, event, isEntryUpdate);
+ }
+ if (val instanceof Chunk) {
+ // if the reused guy has a refcount then need to inc it
+ if (!((Chunk)val).retain()) {
+ throw new IllegalStateException("Could not use an off heap value because it was freed");
+ }
+ }
+ // else it is DataAsAddress. This code just returns it as prepared.
+ // TODO OFFHEAP: Review the callers to see if they will handle DataAsAddress correctly.
+ } else {
+ byte[] data;
+ boolean isSerialized = !(val instanceof byte[]);
+ if (isSerialized) {
+ if (event != null && event.getCachedSerializedNewValue() != null) {
+ data = event.getCachedSerializedNewValue();
+ } else if (val instanceof CachedDeserializable) {
+ data = ((CachedDeserializable)val).getSerializedValue();
+ // TODO OFFHEAP: cache data in event?
+ } else if (val instanceof PdxInstance) {
+ try {
+ data = ((ConvertableToBytes)val).toBytes();
+ // TODO OFFHEAP: cache data in event?
+ } catch (IOException e) {
+ throw new PdxSerializationException("Could not convert " + val + " to bytes", e);
+ }
+ } else {
+ data = EntryEventImpl.serialize(val);
+ // TODO OFFHEAP: cache data in event?
+ }
+ } else {
+ data = (byte[]) val;
+ }
+ byte[] compressedData = compressBytes(r, data);
+ boolean isCompressed = compressedData != data;
+ ReferenceCountHelper.setReferenceCountOwner(this);
+ MemoryAllocator ma = SimpleMemoryAllocatorImpl.getAllocator(); // fix for bug 47875
+ val = ma.allocateAndInitialize(compressedData, isSerialized, isCompressed, GemFireChunk.TYPE); // TODO:KIRK:48068 race happens right after this line
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ if (val instanceof GemFireChunk) {
+ val = new com.gemstone.gemfire.internal.offheap.ChunkWithHeapForm((GemFireChunk)val, data);
+ }
+// if (val instanceof Chunk && r instanceof LocalRegion) {
+// Chunk c = (Chunk) val;
+// LocalRegion lr = (LocalRegion) r;
+// SimpleMemoryAllocatorImpl.debugLog("allocated @" + Long.toHexString(c.getMemoryAddress()) + " reg=" + lr.getFullPath(), false);
+// }
+ }
+ return val;
+ }
+ @Unretained Object nv = val;
+ if (nv instanceof StoredObject) {
+ // This off heap value is being put into a on heap region.
+ byte[] data = ((StoredObject) nv).getSerializedValue();
+ nv = CachedDeserializableFactory.create(data);
+ }
+ // don't bother checking for SQLFire
+ if (!GemFireCacheImpl.sqlfSystem() && nv instanceof PdxInstanceImpl) {
+ // We do not want to put PDXs in the cache as values.
+ // So get the serialized bytes and use a CachedDeserializable.
+ try {
+ byte[] data = ((ConvertableToBytes)nv).toBytes();
+ byte[] compressedData = compressBytes(r, data);
+ if (data == compressedData) {
+ nv = CachedDeserializableFactory.create(data);
+ } else {
+ nv = compressedData;
+ }
+ } catch (IOException e) {
+ throw new PdxSerializationException("Could not convert " + nv + " to bytes", e);
+ }
+ } else {
+ nv = compress(r, nv, event);
+ }
+ return nv;
+ }
+
+ @Override
+ @Unretained
+ public final Object _getValue() {
+ return getValueField();
+ }
+
+ public final boolean isUpdateInProgress() {
+ return areAnyBitsSet(UPDATE_IN_PROGRESS);
+ }
+
+ public final void setUpdateInProgress(final boolean underUpdate) {
+ if (underUpdate) {
+ setBits(UPDATE_IN_PROGRESS);
+ } else {
+ clearBits(~UPDATE_IN_PROGRESS);
+ }
+ }
+
+
+ public final boolean isCacheListenerInvocationInProgress() {
+ return areAnyBitsSet(LISTENER_INVOCATION_IN_PROGRESS);
+ }
+
+ public final void setCacheListenerInvocationInProgress(final boolean listenerInvoked) {
+ if (listenerInvoked) {
+ setBits(LISTENER_INVOCATION_IN_PROGRESS);
+ } else {
+ clearBits(~LISTENER_INVOCATION_IN_PROGRESS);
+ }
+ }
+
+ @Override
+ public final boolean isInUseByTransaction() {
+ return areAnyBitsSet(IN_USE_BY_TX);
+ }
+
+ @Override
+ public final void setInUseByTransaction(final boolean v) {
+ if (v) {
+ setBits(IN_USE_BY_TX);
+ } else {
+ clearBits(~IN_USE_BY_TX);
+ }
+ }
+
+ @Override
+ public final synchronized void incRefCount() {
+ TXManagerImpl.incRefCount(this);
+ setInUseByTransaction(true);
+ }
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public final boolean isMarkedForEviction() {
+ return areAnyBitsSet(MARKED_FOR_EVICTION);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public final void setMarkedForEviction() {
+ setBits(MARKED_FOR_EVICTION);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public final void clearMarkedForEviction() {
+ clearBits(~MARKED_FOR_EVICTION);
+ }
+
+ @Override
+ public final synchronized void decRefCount(NewLRUClockHand lruList, LocalRegion lr) {
+ if (TXManagerImpl.decRefCount(this)) {
+ if (isInUseByTransaction()) {
+ setInUseByTransaction(false);
+ if (lruList != null) {
+ // No more transactions, place in lru list
+ lruList.appendEntry((LRUClockNode)this);
+ }
+ if (lr != null && lr.isEntryExpiryPossible()) {
+ lr.addExpiryTaskIfAbsent(this);
+ }
+ }
+ }
+ }
+
+ @Override
+ public final synchronized void resetRefCount(NewLRUClockHand lruList) {
+ if (isInUseByTransaction()) {
+ setInUseByTransaction(false);
+ if (lruList != null) {
+ lruList.appendEntry((LRUClockNode)this);
+ }
+ }
+ }
+ /**
+ * soubhik: this method is overridden in sqlf flavor of entries.
+ * Instead of overriding this method; override areSetValue.
+ */
+ protected final void _setValue(Object val) {
+ setValueField(val);
+ }
+
+ @Override
+ public Token getValueAsToken() {
+ Object v = getValueField();
+ if (v == null || v instanceof Token) {
+ return (Token)v;
+ } else {
+ return Token.NOT_A_TOKEN;
+ }
+ }
+
+ /**
+ * Reads the value of this region entry.
+ * Provides low level access to the value field.
+ * @return possible OFF_HEAP_OBJECT (caller uses region entry reference)
+ */
+ @Unretained
+ protected abstract Object getValueField();
+ /**
+ * Set the value of this region entry.
+ * Provides low level access to the value field.
+ * @param v the new value to set
+ */
+ protected abstract void setValueField(@Unretained Object v);
+
+ @Retained
+ public Object getTransformedValue() {
+ return _getValueRetain(null, false);
+ }
+
+ public final boolean getValueWasResultOfSearch() {
+ return areAnyBitsSet(VALUE_RESULT_OF_SEARCH);
+ }
+
+ public final void setValueResultOfSearch(boolean v) {
+ if (v) {
+ setBits(VALUE_RESULT_OF_SEARCH);
+ } else {
+ clearBits(~VALUE_RESULT_OF_SEARCH);
+ }
+ }
+
+ public boolean hasValidVersion() {
+ VersionStamp stamp = (VersionStamp)this;
+ boolean has = stamp.getRegionVersion() != 0 || stamp.getEntryVersion() != 0;
+ return has;
+ }
+
+ public boolean hasStats() {
+ // override this in implementations that have stats
+ return false;
+ }
+
+ /**
+ * @see HashEntry#getMapValue()
+ */
+ public final Object getMapValue() {
+ return this;
+ }
+
+ /**
+ * @see HashEntry#setMapValue(Object)
+ */
+ public final void setMapValue(final Object newValue) {
+ if (this != newValue) {
+ Assert.fail("AbstractRegionEntry#setMapValue: unexpected setMapValue "
+ + "with newValue=" + newValue + ", this=" + this);
+ }
+ }
+
+ protected abstract void setEntryHash(int v);
+
+ @Override
+ public final String toString() {
+ final StringBuilder sb = new StringBuilder(this.getClass().getSimpleName())
+ .append('@').append(Integer.toHexString(System.identityHashCode(this)))
+ .append(" (");
+ return appendFieldsToString(sb).append(')').toString();
+ }
+
+ protected StringBuilder appendFieldsToString(final StringBuilder sb) {
+ sb.append("key=").append(getKey()).append("; rawValue=")
+ .append(_getValue()); // OFFHEAP _getValue ok: the current toString on OffHeapCachedDeserializable is safe to use without incing refcount.
+ VersionStamp stamp = getVersionStamp();
+ if (stamp != null) {
+ sb.append("; version=").append(stamp.asVersionTag()+";member="+stamp.getMemberID());
+ }
+ return sb;
+ }
+
+ /*
+ * (non-Javadoc)
+ * This generates version tags for outgoing messages for all subclasses
+ * supporting concurrency versioning. It also sets the entry's version
+ * stamp to the tag's values.
+ *
+ * @see com.gemstone.gemfire.internal.cache.RegionEntry#generateVersionTag(com.gemstone.gemfire.distributed.DistributedMember, boolean)
+ */
+ public VersionTag generateVersionTag(VersionSource mbr, boolean withDelta, LocalRegion region, EntryEventImpl event) {
+ VersionStamp stamp = this.getVersionStamp();
+ if (stamp != null && region.getServerProxy() == null) { // clients do not generate versions
+ int v = stamp.getEntryVersion()+1;
+ if (v > 0xFFFFFF) {
+ v -= 0x1000000; // roll-over
+ }
+ VersionSource previous = stamp.getMemberID();
+
+
+ //For non persistent regions, we allow the member to be null and
+ //when we send a message and the remote side can determine the member
+ //from the sender. For persistent regions, we need to send
+ //the persistent id to the remote side.
+ //
+ //TODO - RVV - optimize the way we send the persistent id to save
+ //space.
+ if(mbr == null) {
+ VersionSource regionMember = region.getVersionMember();
+ if(regionMember instanceof DiskStoreID) {
+ mbr = regionMember;
+ }
+ }
+
+ VersionTag tag = VersionTag.create(mbr);
+ tag.setEntryVersion(v);
+ if (region.getVersionVector() != null) {
+ // Use region version if already provided, else generate
+ long nextRegionVersion = event.getNextRegionVersion();
+ if (nextRegionVersion != -1) {
+ // Set on the tag and record it locally
+ tag.setRegionVersion(nextRegionVersion);
+ RegionVersionVector rvv = region.getVersionVector();
+ rvv.recordVersion(rvv.getOwnerId(),nextRegionVersion);
+ if (logger.isDebugEnabled()) {
+ logger.debug("recorded region version {}; region={}", nextRegionVersion, region.getFullPath());
+ }
+ } else {
+ tag.setRegionVersion(region.getVersionVector().getNextVersion());
+ }
+ }
+ if (withDelta) {
+ tag.setPreviousMemberID(previous);
+ }
+ VersionTag remoteTag = event.getVersionTag();
+ if (remoteTag != null && remoteTag.isGatewayTag()) {
+ // if this event was received from a gateway we use the remote system's
+ // timestamp and dsid.
+ tag.setVersionTimeStamp(remoteTag.getVersionTimeStamp());
+ tag.setDistributedSystemId(remoteTag.getDistributedSystemId());
+ tag.setAllowedByResolver(remoteTag.isAllowedByResolver());
+ } else {
+ long time = region.cacheTimeMillis();
+ int dsid = region.getDistributionManager().getDistributedSystemId();
+ // a locally generated change should always have a later timestamp than
+ // one received from a wan gateway, so fake a timestamp if necessary
+ if (time <= stamp.getVersionTimeStamp() && dsid != tag.getDistributedSystemId()) {
+ time = stamp.getVersionTimeStamp() + 1;
+ }
+ tag.setVersionTimeStamp(time);
+ tag.setDistributedSystemId(dsid);
+ }
+ stamp.setVersions(tag);
+ stamp.setMemberID(mbr);
+ event.setVersionTag(tag);
+ if (logger.isDebugEnabled()) {
+ logger.debug("generated tag {}; key={}; oldvalue={} newvalue={} client={} region={}; rvv={}", tag,
+ event.getKey(), event.getOldValueStringForm(), event.getNewValueStringForm(),
+ (event.getContext() == null? "none" : event.getContext().getDistributedMember().getName()),
+ region.getFullPath(), region.getVersionVector());
+ }
+ return tag;
+ }
+ return null;
+ }
+
+ /** set/unset the flag noting that a tombstone has been scheduled for this entry */
+ public void setTombstoneScheduled(boolean scheduled) {
+ if (scheduled) {
+ setBits(TOMBSTONE_SCHEDULED);
+ } else {
+ clearBits(~TOMBSTONE_SCHEDULED);
+ }
+ }
+
+ /**
+ * return the flag noting whether a tombstone has been scheduled for this entry. This should
+ * be called under synchronization on the region entry if you want an accurate result.
+ */
+ public boolean isTombstoneScheduled() {
+ return areAnyBitsSet(TOMBSTONE_SCHEDULED);
+ }
+
+ /*
+ * (non-Javadoc)
+ * This performs a concurrency check.
+ *
+ * This check compares the version number first, followed by the member ID.
+ *
+ * Wraparound of the version number is detected and handled by extending the
+ * range of versions by one bit.
+ *
+ * The normal membership ID comparison method is used.
+ *
+ * Note that a tag from a remote (WAN) system may be in the event. If this
+ * is the case this method will either invoke a user plugin that allows/disallows
+ * the event (and may modify the value) or it determines whether to allow
+ * or disallow the event based on timestamps and distributedSystemIDs.
+ *
+ * @throws ConcurrentCacheModificationException if the event conflicts with
+ * an event that has already been applied to the entry.
+ *
+ * @see com.gemstone.gemfire.internal.cache.RegionEntry#concurrencyCheck(com.gemstone.gemfire.cache.EntryEvent)
+ */
+ public void processVersionTag(EntryEvent cacheEvent) {
+ processVersionTag(cacheEvent, true);
+ }
+
+
+ protected void processVersionTag(EntryEvent cacheEvent, boolean conflictCheck) {
+ EntryEventImpl event = (EntryEventImpl)cacheEvent;
+ VersionTag tag = event.getVersionTag();
+ if (tag == null) {
+ return;
+ }
+
+ try {
+ if (tag.isGatewayTag()) {
+ // this may throw ConcurrentCacheModificationException or modify the event
+ if (processGatewayTag(cacheEvent)) {
+ return;
+ }
+ assert false : "processGatewayTag failure - returned false";
+ }
+
+ if (!tag.isFromOtherMember()) {
+ if (!event.getOperation().isNetSearch()) {
+ // except for netsearch, all locally-generated tags can be ignored
+ return;
+ }
+ }
+
+ final InternalDistributedMember originator = (InternalDistributedMember)event.getDistributedMember();
+ final VersionSource dmId = event.getRegion().getVersionMember();
+ LocalRegion r = event.getLocalRegion();
+ boolean eventHasDelta = event.getDeltaBytes() != null && event.getRawNewValue() == null;
+
+ VersionStamp stamp = getVersionStamp();
+ // bug #46223, an event received from a peer or a server may be from a different
+ // distributed system than the last modification made to this entry so we must
+ // perform a gateway conflict check
+ if (stamp != null && !tag.isAllowedByResolver()) {
+ int stampDsId = stamp.getDistributedSystemId();
+ int tagDsId = tag.getDistributedSystemId();
+
+ if (stampDsId != 0 && stampDsId != tagDsId && stampDsId != -1) {
+ StringBuilder verbose = null;
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE)) {
+ verbose = new StringBuilder();
+ verbose.append("processing tag for key " + getKey() + ", stamp=" + stamp.asVersionTag() + ", tag=").append(tag);
+ }
+ long stampTime = stamp.getVersionTimeStamp();
+ long tagTime = tag.getVersionTimeStamp();
+ if (stampTime > 0 && (tagTime > stampTime
+ || (tagTime == stampTime && tag.getDistributedSystemId() >= stamp.getDistributedSystemId()))) {
+ if (verbose != null) {
+ verbose.append(" - allowing event");
+ logger.trace(LogMarker.TOMBSTONE, verbose);
+ }
+ // Update the stamp with event's version information.
+ applyVersionTag(r, stamp, tag, originator);
+ return;
+ }
+
+ if (stampTime > 0) {
+ if (verbose != null) {
+ verbose.append(" - disallowing event");
+ logger.trace(LogMarker.TOMBSTONE, verbose);
+ }
+ r.getCachePerfStats().incConflatedEventsCount();
+ persistConflictingTag(r, tag);
+ throw new ConcurrentCacheModificationException("conflicting event detected");
+ }
+ }
+ }
+
+ if (r.getVersionVector() != null &&
+ r.getServerProxy() == null &&
+ (r.getDataPolicy().withPersistence() ||
+ !r.getScope().isLocal())) { // bug #45258 - perf degradation for local regions and RVV
+ VersionSource who = tag.getMemberID();
+ if (who == null) {
+ who = originator;
+ }
+ r.getVersionVector().recordVersion(who, tag);
+ }
+
+ assert !tag.isFromOtherMember() || tag.getMemberID() != null : "remote tag is missing memberID";
+
+
+ // [bruce] for a long time I had conflict checks turned off in clients when
+ // receiving a response from a server and applying it to the cache. This lowered
+ // the CPU cost of versioning but eventually had to be pulled for bug #45453
+// if (r.getServerProxy() != null && conflictCheck) {
+// // events coming from servers while a local sync is held on the entry
+// // do not require a conflict check. Conflict checks were already
+// // performed on the server and here we just consume whatever was sent back.
+// // Event.isFromServer() returns true for client-update messages and
+// // for putAll/getAll, which do not hold syncs during the server operation.
+// conflictCheck = event.isFromServer();
+// }
+// else
+
+ // [bruce] for a very long time we had conflict checks turned off for PR buckets.
+ // Bug 45669 showed a primary dying in the middle of distribution. This caused
+ // one backup bucket to have a v2. The other bucket was promoted to primary and
+ // generated a conflicting v2. We need to do the check so that if this second
+ // v2 loses to the original one in the delta-GII operation that the original v2
+ // will be the winner in both buckets.
+// if (r.isUsedForPartitionedRegionBucket()) {
+// conflictCheck = false; // primary/secondary model
+// }
+
+ // The new value in event is not from GII, even it could be tombstone
+ basicProcessVersionTag(r, tag, false, eventHasDelta, dmId, originator, conflictCheck);
+ } catch (ConcurrentCacheModificationException ex) {
+ event.isConcurrencyConflict(true);
+ throw ex;
+ }
+ }
+
+ protected final void basicProcessVersionTag(LocalRegion region, VersionTag tag, boolean isTombstoneFromGII,
+ boolean deltaCheck, VersionSource dmId, InternalDistributedMember sender, boolean checkForConflict) {
+
+ StringBuilder verbose = null;
+
+ if (tag != null) {
+ VersionStamp stamp = getVersionStamp();
+
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE)) {
+ VersionTag stampTag = stamp.asVersionTag();
+ if (stampTag.hasValidVersion() && checkForConflict) { // only be verbose here if there's a possibility we might reject the operation
+ verbose = new StringBuilder();
+ verbose.append("processing tag for key " + getKey() + ", stamp=" + stamp.asVersionTag() + ", tag=").append(tag)
+ .append(", checkForConflict=").append(checkForConflict); //.append(", current value=").append(_getValue());
+ }
+ }
+
+ if (stamp == null) {
+ throw new IllegalStateException("message contained a version tag but this region has no version storage");
+ }
+
+ boolean apply = true;
+
+ try {
+ if (checkForConflict) {
+ apply = checkForConflict(region, stamp, tag, isTombstoneFromGII, deltaCheck, dmId, sender, verbose);
+ }
+ } catch (ConcurrentCacheModificationException e) {
+ // Even if we don't apply the operation we should always retain the
+ // highest timestamp in order for WAN conflict checks to work correctly
+ // because the operation may have been sent to other systems and been
+ // applied there
+ if (!tag.isGatewayTag()
+ && stamp.getDistributedSystemId() == tag.getDistributedSystemId()
+ && tag.getVersionTimeStamp() > stamp.getVersionTimeStamp()) {
+ stamp.setVersionTimeStamp(tag.getVersionTimeStamp());
+ tag.setTimeStampApplied(true);
+ if (verbose != null) {
+ verbose.append("\nThough in conflict the tag timestamp was more recent and was recorded.");
+ }
+ }
+ throw e;
+ } finally {
+ if (verbose != null) {
+ logger.trace(LogMarker.TOMBSTONE, verbose);
+ }
+ }
+
+ if (apply) {
+ applyVersionTag(region, stamp, tag, sender);
+ }
+ }
+ }
+
+
+ private void applyVersionTag(LocalRegion region, VersionStamp stamp, VersionTag tag, InternalDistributedMember sender) {
+ // stamp.setPreviousMemberID(stamp.getMemberID());
+ VersionSource mbr = tag.getMemberID();
+ if (mbr == null) {
+ mbr = sender;
+ }
+ mbr = region.getVersionVector().getCanonicalId(mbr);
+ tag.setMemberID(mbr);
+ stamp.setVersions(tag);
+ if (tag.hasPreviousMemberID()) {
+ if (tag.getPreviousMemberID() == null) {
+ tag.setPreviousMemberID(stamp.getMemberID());
+ } else {
+ tag.setPreviousMemberID(region.getVersionVector().getCanonicalId(
+ tag.getPreviousMemberID()));
+ }
+ }
+ }
+
+ /** perform conflict checking for a stamp/tag */
+ protected boolean checkForConflict(LocalRegion region,
+ VersionStamp stamp, VersionTag tag,
+ boolean isTombstoneFromGII,
+ boolean deltaCheck, VersionSource dmId,
+ InternalDistributedMember sender, StringBuilder verbose) {
+
+ int stampVersion = stamp.getEntryVersion();
+ int tagVersion = tag.getEntryVersion();
+
+ boolean throwex = false;
+ boolean apply = false;
+
+ if (stamp.getVersionTimeStamp() != 0) { // new entries have no timestamp
+ // check for wrap-around on the version number
+ long difference = tagVersion - stampVersion;
+ if (0x10000 < difference || difference < -0x10000) {
+ if (verbose != null) {
+ verbose.append("\nversion rollover detected: tag="+tagVersion + " stamp=" + stampVersion);
+ }
+ if (difference < 0) {
+ tagVersion += 0x1000000L;
+ } else {
+ stampVersion += 0x1000000L;
+ }
+ }
+ }
+ if (verbose != null) {
+ verbose.append("\nstamp=v").append(stampVersion)
+ .append(" tag=v").append(tagVersion);
+ }
+
+ if (deltaCheck) {
+ checkForDeltaConflict(region, stampVersion, tagVersion, stamp, tag, dmId, sender, verbose);
+ }
+
+ if (stampVersion == 0 || stampVersion < tagVersion) {
+ if (verbose != null) { verbose.append(" - applying change"); }
+ apply = true;
+ } else if (stampVersion > tagVersion) {
+ if (overwritingOldTombstone(region, stamp, tag, verbose) && tag.getVersionTimeStamp() > stamp.getVersionTimeStamp()) {
+ apply = true;
+ } else {
+ // check for an incoming expired tombstone from an initial image chunk.
+ if (tagVersion > 0
+ && isExpiredTombstone(region, tag.getVersionTimeStamp(), isTombstoneFromGII)
+ && tag.getVersionTimeStamp() > stamp.getVersionTimeStamp()) {
+ // A special case to apply: when remote entry is expired tombstone, then let local vs remote with newer timestamp to win
+ if (verbose != null) { verbose.append(" - applying change in Delta GII"); }
+ apply = true;
+ } else {
+ if (verbose != null) { verbose.append(" - disallowing"); }
+ throwex= true;
+ }
+ }
+ } else {
+ if (overwritingOldTombstone(region, stamp, tag, verbose)) {
+ apply = true;
+ } else {
+ // compare member IDs
+ VersionSource stampID = stamp.getMemberID();
+ if (stampID == null) {
+ stampID = dmId;
+ }
+ VersionSource tagID = tag.getMemberID();
+ if (tagID == null) {
+ tagID = sender;
+ }
+ if (verbose != null) { verbose.append("\ncomparing IDs"); }
+ int compare = stampID.compareTo(tagID);
+ if (compare < 0) {
+ if (verbose != null) { verbose.append(" - applying change"); }
+ apply = true;
+ } else if (compare > 0) {
+ if (verbose != null) { verbose.append(" - disallowing"); }
+ throwex = true;
+ } else if (tag.isPosDup()) {
+ if (verbose != null) { verbose.append(" - disallowing duplicate marked with posdup"); }
+ throwex = true;
+ } else /* if (isTombstoneFromGII && isTombstone()) {
+ if (verbose != null) { verbose.append(" - disallowing duplicate tombstone from GII"); }
+ return false; // bug #49601 don't schedule tombstones from GII if there's already one here
+ } else */ {
+ if (verbose != null) { verbose.append(" - allowing duplicate"); }
+ }
+ }
+ }
+
+ if (!apply && throwex) {
+ region.getCachePerfStats().incConflatedEventsCount();
+ persistConflictingTag(region, tag);
+ throw new ConcurrentCacheModificationException();
+ }
+
+ return apply;
+ }
+
+ private boolean isExpiredTombstone(LocalRegion region, long timestamp, boolean isTombstone) {
+ return isTombstone && (timestamp + TombstoneService.REPLICATED_TOMBSTONE_TIMEOUT) <= region.cacheTimeMillis();
+ }
+
+ private boolean overwritingOldTombstone(LocalRegion region, VersionStamp stamp, VersionTag tag, StringBuilder verbose) {
+ // Tombstone GC does not use locking to stop operations when old tombstones
+ // are being removed. Because of this we might get an operation that was applied
+ // in another VM that has just reaped a tombstone and is now using a reset
+ // entry version number. Because of this we check the timestamp on the current
+ // local entry and see if it is old enough to have expired. If this is the case
+ // we accept the change and allow the tag to be recorded
+ long stampTime = stamp.getVersionTimeStamp();
+ if (isExpiredTombstone(region, stampTime, this.isTombstone())) {
+ // no local change since the tombstone would have timed out - accept the change
+ if (verbose != null) { verbose.append(" - accepting because local timestamp is old"); }
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ protected void persistConflictingTag(LocalRegion region, VersionTag tag) {
+ // only persist region needs to persist conflict tag
+ }
+
+ /**
+ * for an event containing a delta we must check to see if the tag's
+ * previous member id is the stamp's member id and ensure that the
+ * version is only incremented by 1. Otherwise the delta is being
+ * applied to a value that does not match the source of the delta.
+ *
+ * @throws InvalidDeltaException
+ */
+ private void checkForDeltaConflict(LocalRegion region,
+ long stampVersion, long tagVersion,
+ VersionStamp stamp, VersionTag tag,
+ VersionSource dmId, InternalDistributedMember sender,
+ StringBuilder verbose) {
+
+ if (tagVersion != stampVersion+1) {
+ if (verbose != null) {
+ verbose.append("\ndelta requires full value due to version mismatch");
+ }
+ region.getCachePerfStats().incDeltaFailedUpdates();
+ throw new InvalidDeltaException("delta cannot be applied due to version mismatch");
+
+ } else {
+ // make sure the tag was based on the value in this entry by checking the
+ // tag's previous-changer ID against this stamp's current ID
+ VersionSource stampID = stamp.getMemberID();
+ if (stampID == null) {
+ stampID = dmId;
+ }
+ VersionSource tagID = tag.getPreviousMemberID();
+ if (tagID == null) {
+ tagID = sender;
+ }
+ if (!tagID.equals(stampID)) {
+ if (verbose != null) {
+ verbose.append("\ndelta requires full value. tag.previous=")
+ .append(tagID).append(" but stamp.current=").append(stampID);
+ }
+ region.getCachePerfStats().incDeltaFailedUpdates();
+ throw new InvalidDeltaException("delta cannot be applied due to version ID mismatch");
+ }
+ }
+ }
+
+ private boolean processGatewayTag(EntryEvent cacheEvent) {
+ // Gateway tags are installed in the server-side LocalRegion cache
+ // modification methods. They do not have version numbers or distributed
+ // member IDs. Instead they only have timestamps and distributed system IDs.
+
+ // If there is a resolver plug-in, invoke it. Otherwise we use the timestamps and
+ // distributed system IDs to determine whether to allow the event to proceed.
+
+ final boolean isDebugEnabled = logger.isDebugEnabled();
+
+ if (this.isRemoved() && !this.isTombstone()) {
+ return true; // no conflict on a new entry
+ }
+ EntryEventImpl event = (EntryEventImpl)cacheEvent;
+ VersionTag tag = event.getVersionTag();
+ long stampTime = getVersionStamp().getVersionTimeStamp();
+ long tagTime = tag.getVersionTimeStamp();
+ int stampDsid = getVersionStamp().getDistributedSystemId();
+ int tagDsid = tag.getDistributedSystemId();
+ if (isDebugEnabled) {
+ logger.debug("processing gateway version information for {}. Stamp dsid={} time={} Tag dsid={} time={}",
+ event.getKey(), stampDsid, stampTime, tagDsid, tagTime);
+ }
+ if (tagTime == VersionTag.ILLEGAL_VERSION_TIMESTAMP) {
+ return true; // no timestamp received from other system - just apply it
+ }
+ if (tagDsid == stampDsid || stampDsid == -1) {
+ return true;
+ }
+ GatewayConflictResolver resolver = event.getRegion().getCache().getGatewayConflictResolver();
+ if (resolver != null) {
+ if (isDebugEnabled) {
+ logger.debug("invoking gateway conflict resolver");
+ }
+ final boolean[] disallow = new boolean[1];
+ final Object[] newValue = new Object[] { this };
+ GatewayConflictHelper helper = new GatewayConflictHelper() {
+ @Override
+ public void disallowEvent() {
+ disallow[0] = true;
+ }
+
+ @Override
+ public void changeEventValue(Object v) {
+ newValue[0] = v;
+ }
+ };
+ TimestampedEntryEventImpl timestampedEvent =
+ (TimestampedEntryEventImpl)event.getTimestampedEvent(tagDsid, stampDsid, tagTime, stampTime);
+
+ // gateway conflict resolvers will usually want to see the old value
+ if (!timestampedEvent.hasOldValue() && isRemoved()) {
+ timestampedEvent.setOldValue(getValue(timestampedEvent.getRegion())); // OFFHEAP: since isRemoved I think getValue will never be stored off heap in this case
+ }
+
+ Throwable thr = null;
+ try {
+ resolver.onEvent(timestampedEvent, helper);
+ }
+ catch (CancelException cancelled) {
+ throw cancelled;
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.error(LocalizedMessage.create(LocalizedStrings.LocalRegion_EXCEPTION_OCCURRED_IN_CONFLICTRESOLVER), t);
+ thr = t;
+ } finally {
+ timestampedEvent.release();
+ }
+
+ if (isDebugEnabled) {
+ logger.debug("done invoking resolver {}", thr);
+ }
+ if (thr == null) {
+ if (disallow[0]) {
+ if (isDebugEnabled) {
+ logger.debug("conflict resolver rejected the event for {}", event.getKey());
+ }
+ throw new ConcurrentCacheModificationException("WAN conflict resolver rejected the operation");
+ }
+
+ tag.setAllowedByResolver(true);
+
+ if (newValue[0] != this) {
+ if (isDebugEnabled) {
+ logger.debug("conflict resolver changed the value of the event for {}", event.getKey());
+ }
+ // the resolver changed the event value!
+ event.setNewValue(newValue[0]);
+ }
+ // if nothing was done then we allow the event
+ if (isDebugEnabled) {
+ logger.debug("change was allowed by conflict resolver: {}", tag);
+ }
+ return true;
+ }
+ }
+ if (isDebugEnabled) {
+ logger.debug("performing normal WAN conflict check");
+ }
+ if (tagTime > stampTime
+ || (tagTime == stampTime && tagDsid >= stampDsid)) {
+ if (isDebugEnabled) {
+ logger.debug("allowing event");
+ }
+ return true;
+ }
+ if (isDebugEnabled) {
+ logger.debug("disallowing event for " + event.getKey());
+ }
+ throw new ConcurrentCacheModificationException("conflicting WAN event detected");
+ }
+
+ static boolean isCompressible(RegionEntryContext context,Object value) {
+ return ((value != null) && (context != null) && (context.getCompressor() != null) && !Token.isInvalidOrRemoved(value));
+ }
+
+ /* subclasses supporting versions must override this */
+ public VersionStamp getVersionStamp() {
+ return null;
+ }
+
+ public boolean isValueNull() {
+ return (null == getValueAsToken());
+ }
+
+ public boolean isInvalid() {
+ return Token.isInvalid(getValueAsToken());
+ }
+
+ public boolean isDestroyed() {
+ return Token.isDestroyed(getValueAsToken());
+ }
+
+ public void setValueToNull() {
+ _setValue(null);
+ }
+
+ public boolean isInvalidOrRemoved() {
+ return Token.isInvalidOrRemoved(getValueAsToken());
+ }
+
+ /**
+ * Maximum size of a string that can be encoded as char.
+ */
+ public static final int MAX_INLINE_STRING_KEY_CHAR_ENCODING = 7;
+ /**
+ * Maximum size of a string that can be encoded as byte.
+ */
+ public static final int MAX_INLINE_STRING_KEY_BYTE_ENCODING = 15;
+
+ /**
+ * This is only retained in off-heap subclasses. However, it's marked as
+ * Retained here so that callers are aware that the value may be retained.
+ */
+ @Override
+ @Retained
+ public Object _getValueRetain(RegionEntryContext context, boolean decompress) {
+ if (decompress) {
+ return decompress(context, _getValue());
+ } else {
+ return _getValue();
+ }
+ }
+
+ @Override
+ public void returnToPool() {
+ // noop by default
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
new file mode 100644
index 000000000000..7661f82fdaf6
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
@@ -0,0 +1,4163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.gemstone.gemfire.internal.cache.region.entry.RegionEntryFactoryBuilder;
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.GemFireIOException;
+import com.gemstone.gemfire.InvalidDeltaException;
+import com.gemstone.gemfire.cache.CacheRuntimeException;
+import com.gemstone.gemfire.cache.CacheWriter;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.CustomEvictionAttributes;
+import com.gemstone.gemfire.cache.DiskAccessException;
+import com.gemstone.gemfire.cache.EntryExistsException;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
+import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
+import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.ClassPathLoader;
+import com.gemstone.gemfire.internal.cache.DiskInitFile.DiskRegionFlag;
+import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
+import com.gemstone.gemfire.internal.cache.delta.Delta;
+import com.gemstone.gemfire.internal.cache.ha.HAContainerWrapper;
+import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
+import com.gemstone.gemfire.internal.cache.lru.LRUEntry;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tier.sockets.HAEventWrapper;
+import com.gemstone.gemfire.internal.cache.versions.ConcurrentCacheModificationException;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
+import com.gemstone.gemfire.internal.cache.versions.VersionHolder;
+import com.gemstone.gemfire.internal.cache.versions.VersionSource;
+import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventImpl;
+import com.gemstone.gemfire.internal.concurrent.MapCallbackAdapter;
+import com.gemstone.gemfire.internal.concurrent.MapResult;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
+import com.gemstone.gemfire.internal.offheap.annotations.Released;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
+import com.gemstone.gemfire.internal.util.concurrent.CustomEntryConcurrentHashMap;
+import com.gemstone.gemfire.pdx.PdxInstance;
+import com.gemstone.gemfire.pdx.PdxSerializationException;
+import com.gemstone.gemfire.pdx.internal.ConvertableToBytes;
+
+/**
+ * Abstract implementation of {@link RegionMap}that has all the common
+ * behavior.
+ *
+ * @since 3.5.1
+ *
+ *
+ */
+
+//Asif: In case of sqlFabric System, we are creating a different set of RegionEntry
+// which are derived from the concrete GFE RegionEntry classes.
+// In future if any new concrete RegionEntry class is defined, the new SqlFabric
+// RegionEntry Classes need to be created. There is a junit test in sqlfabric
+// which checks for RegionEntry classes of GFE and validates the same with its
+// own classes.
+
+public abstract class AbstractRegionMap implements RegionMap {
+
+ private static final Logger logger = LogService.getLogger();
+
+ /** The underlying map for this region. */
+ protected CustomEntryConcurrentHashMap map;
+ /** An internal Listener for index maintenance for SQLFabric. */
+ private final IndexUpdater indexUpdater;
+
+ /**
+ * This test hook is used to force the conditions for defect 48182.
+ * This hook is used by Bug48182JUnitTest.
+ */
+ static Runnable testHookRunnableFor48182 = null;
+
+ private RegionEntryFactory entryFactory;
+ private Attributes attr;
+ private transient Object owner; // the region that owns this map
+
+ protected AbstractRegionMap(InternalRegionArguments internalRegionArgs) {
+ if (internalRegionArgs != null) {
+ this.indexUpdater = internalRegionArgs.getIndexUpdater();
+ }
+ else {
+ this.indexUpdater = null;
+ }
+ }
+
+ public final IndexUpdater getIndexUpdater() {
+ return this.indexUpdater;
+ }
+
+ protected void initialize(Object owner,
+ Attributes attr,
+ InternalRegionArguments internalRegionArgs,
+ boolean isLRU) {
+ _setAttributes(attr);
+ setOwner(owner);
+ _setMap(createConcurrentMap(attr.initialCapacity, attr.loadFactor,
+ attr.concurrencyLevel, false,
+ new AbstractRegionEntry.HashRegionEntryCreator()));
+
+ final GemFireCacheImpl cache;
+ boolean isDisk;
+ boolean withVersioning = false;
+ boolean offHeap = false;
+ if (owner instanceof LocalRegion) {
+ LocalRegion region = (LocalRegion)owner;
+ isDisk = region.getDiskRegion() != null;
+ cache = region.getGemFireCache();
+ withVersioning = region.getConcurrencyChecksEnabled();
+ offHeap = region.getOffHeap();
+ }
+ else if (owner instanceof PlaceHolderDiskRegion) {
+ offHeap = ((PlaceHolderDiskRegion) owner).getOffHeap();
+ isDisk = true;
+ withVersioning = ((PlaceHolderDiskRegion)owner).getFlags().contains(
+ DiskRegionFlag.IS_WITH_VERSIONING);
+ cache = GemFireCacheImpl.getInstance();
+ }
+ else {
+ throw new IllegalStateException(
+ "expected LocalRegion or PlaceHolderDiskRegion");
+ }
+
+ if (cache != null && cache.isSqlfSystem()) {
+ String provider = GemFireCacheImpl.SQLF_ENTRY_FACTORY_PROVIDER;
+ try {
+ Class> factoryProvider = ClassPathLoader.getLatest().forName(provider);
+ Method method = factoryProvider.getDeclaredMethod(
+ "getRegionEntryFactory", new Class[] { Boolean.TYPE, Boolean.TYPE,
+ Boolean.TYPE, Object.class, InternalRegionArguments.class });
+ RegionEntryFactory ref = (RegionEntryFactory)method.invoke(null,
+ new Object[] { Boolean.valueOf(attr.statisticsEnabled),
+ Boolean.valueOf(isLRU), Boolean.valueOf(isDisk), owner,
+ internalRegionArgs });
+
+ // TODO need to have the SQLF entry factory support version stamp storage
+ setEntryFactory(ref);
+
+ }
+ catch (Exception e) {
+ throw new CacheRuntimeException(
+ "Exception in obtaining RegionEntry Factory" + " provider class ",
+ e) {
+ };
+ }
+ }
+ else {
+ setEntryFactory(new RegionEntryFactoryBuilder().getRegionEntryFactoryOrNull(attr.statisticsEnabled,isLRU,isDisk,withVersioning,offHeap));
+ }
+ }
+
+ protected CustomEntryConcurrentHashMap createConcurrentMap(
+ int initialCapacity, float loadFactor, int concurrencyLevel,
+ boolean isIdentityMap,
+ CustomEntryConcurrentHashMap.HashEntryCreator entryCreator) {
+ if (entryCreator != null) {
+ return new CustomEntryConcurrentHashMap(initialCapacity, loadFactor,
+ concurrencyLevel, isIdentityMap, entryCreator);
+ }
+ else {
+ return new CustomEntryConcurrentHashMap(initialCapacity,
+ loadFactor, concurrencyLevel, isIdentityMap);
+ }
+ }
+
+ public void changeOwner(LocalRegion r) {
+ if (r == _getOwnerObject()) {
+ return;
+ }
+ setOwner(r);
+ }
+
+ @Override
+ public final void setEntryFactory(RegionEntryFactory f) {
+ this.entryFactory = f;
+ }
+
+ public final RegionEntryFactory getEntryFactory() {
+ return this.entryFactory;
+ }
+
+ protected final void _setAttributes(Attributes a) {
+ this.attr = a;
+ }
+
+ public final Attributes getAttributes() {
+ return this.attr;
+ }
+
+ protected final LocalRegion _getOwner() {
+ return (LocalRegion)this.owner;
+ }
+
+ protected boolean _isOwnerALocalRegion() {
+ return this.owner instanceof LocalRegion;
+ }
+
+ protected final Object _getOwnerObject() {
+ return this.owner;
+ }
+
+ public final void setOwner(Object r) {
+ this.owner = r;
+ }
+
+ protected final CustomEntryConcurrentHashMap _getMap() {
+ return this.map;
+ }
+
+ protected final void _setMap(CustomEntryConcurrentHashMap m) {
+ this.map = m;
+ }
+
+ public int size()
+ {
+ return _getMap().size();
+ }
+
+ // this is currently used by stats and eviction
+ @Override
+ public int sizeInVM() {
+ return _getMap().size();
+ }
+
+ public boolean isEmpty()
+ {
+ return _getMap().isEmpty();
+ }
+
+ public Set keySet()
+ {
+ return _getMap().keySet();
+ }
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ public Collection regionEntries() {
+ return (Collection)_getMap().values();
+ }
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ @Override
+ public Collection regionEntriesInVM() {
+ return (Collection)_getMap().values();
+ }
+
+ public final boolean containsKey(Object key) {
+ RegionEntry re = getEntry(key);
+ if (re == null) {
+ return false;
+ }
+ if (re.isRemoved()) {
+ return false;
+ }
+ return true;
+ }
+
+ public RegionEntry getEntry(Object key) {
+ RegionEntry re = (RegionEntry)_getMap().get(key);
+ if (re != null && re.isMarkedForEviction()) {
+ // entry has been faulted in from HDFS
+ return null;
+ }
+ return re;
+ }
+
+ protected RegionEntry getEntry(EntryEventImpl event) {
+ return getEntry(event.getKey());
+ }
+
+
+ @Override
+ public final RegionEntry getEntryInVM(Object key) {
+ return (RegionEntry)_getMap().get(key);
+ }
+
+
+ public final RegionEntry putEntryIfAbsent(Object key, RegionEntry re) {
+ RegionEntry value = (RegionEntry)_getMap().putIfAbsent(key, re);
+ if (value == null && (re instanceof OffHeapRegionEntry)
+ && _isOwnerALocalRegion() && _getOwner().isThisRegionBeingClosedOrDestroyed()) {
+ // prevent orphan during concurrent destroy (#48068)
+ if (_getMap().remove(key, re)) {
+ ((OffHeapRegionEntry)re).release();
+ }
+ _getOwner().checkReadiness(); // throw RegionDestroyedException
+ }
+ return value;
+ }
+
+ @Override
+ public final RegionEntry getOperationalEntryInVM(Object key) {
+ RegionEntry re = (RegionEntry)_getMap().get(key);
+ if (re != null && re.isMarkedForEviction()) {
+ // entry has been faulted in from HDFS
+ return null;
+ }
+ return re;
+ }
+
+
+ public final void removeEntry(Object key, RegionEntry re, boolean updateStat) {
+ if (re.isTombstone() && _getMap().get(key) == re && !re.isMarkedForEviction()){
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.AbstractRegionMap_ATTEMPT_TO_REMOVE_TOMBSTONE), new Exception("stack trace"));
+ return; // can't remove tombstones except from the tombstone sweeper
+ }
+ if (_getMap().remove(key, re)) {
+ re.removePhase2();
+ if (updateStat) {
+ incEntryCount(-1);
+ }
+ }
+ }
+
+ public final void removeEntry(Object key, RegionEntry re, boolean updateStat,
+ EntryEventImpl event, final LocalRegion owner,
+ final IndexUpdater indexUpdater) {
+ boolean success = false;
+ if (re.isTombstone()&& _getMap().get(key) == re && !re.isMarkedForEviction()) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.AbstractRegionMap_ATTEMPT_TO_REMOVE_TOMBSTONE), new Exception("stack trace"));
+ return; // can't remove tombstones except from the tombstone sweeper
+ }
+ try {
+ if (indexUpdater != null) {
+ indexUpdater.onEvent(owner, event, re);
+ }
+
+ //This is messy, but custom eviction calls removeEntry
+ //rather than re.destroy I think to avoid firing callbacks, etc.
+ //However, the value still needs to be set to removePhase1
+ //in order to remove the entry from disk.
+ if(event.isCustomEviction() && !re.isRemoved()) {
+ try {
+ re.removePhase1(owner, false);
+ } catch (RegionClearedException e) {
+ //that's ok, we were just trying to do evict incoming eviction
+ }
+ }
+
+ if (_getMap().remove(key, re)) {
+ re.removePhase2();
+ success = true;
+ if (updateStat) {
+ incEntryCount(-1);
+ }
+ }
+ } finally {
+ if (indexUpdater != null) {
+ indexUpdater.postEvent(owner, event, re, success);
+ }
+ }
+ }
+
+ protected final void incEntryCount(int delta) {
+ LocalRegion lr = _getOwner();
+ if (lr != null) {
+ CachePerfStats stats = lr.getCachePerfStats();
+ if (stats != null) {
+ stats.incEntryCount(delta);
+ }
+ }
+ }
+
+ final void incClearCount(LocalRegion lr) {
+ if (lr != null && !(lr instanceof HARegion)) {
+ CachePerfStats stats = lr.getCachePerfStats();
+ if (stats != null) {
+ stats.incClearCount();
+ }
+ }
+ }
+
+ private void _mapClear() {
+ _getMap().clear();
+ }
+
+ public void close() {
+ /*
+ for (SuspectEntryList l: this.suspectEntries.values()) {
+ for (EntryEventImpl e: l) {
+ e.release();
+ }
+ }
+ */
+ clear(null);
+ }
+
+ /**
+ * Clear the region and, if an RVV is given, return a collection of the
+ * version sources in all remaining tags
+ */
+ public Set clear(RegionVersionVector rvv)
+ {
+ Set result = new HashSet();
+
+ if(!_isOwnerALocalRegion()) {
+ //Fix for #41333. Just clear the the map
+ //if we failed during initialization.
+ _mapClear();
+ return null;
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("Clearing entries for {} rvv={}", _getOwner(), " rvv=" + rvv);
+ }
+ LocalRegion lr = _getOwner();
+ RegionVersionVector localRvv = lr.getVersionVector();
+ incClearCount(lr);
+ // lock for size calcs if the region might have tombstones
+ Object lockObj = lr.getConcurrencyChecksEnabled()? lr.getSizeGuard() : new Object();
+ synchronized (lockObj) {
+ if (rvv == null) {
+ int delta = 0;
+ try {
+ delta = sizeInVM(); // TODO soplog need to determine if stats should
+ // reflect only size in memory or the complete thing
+ } catch (GemFireIOException e) {
+ // ignore rather than throwing an exception during cache close
+ }
+ int tombstones = lr.getTombstoneCount();
+ _mapClear();
+ _getOwner().updateSizeOnClearRegion(delta - tombstones);
+ _getOwner().incTombstoneCount(-tombstones);
+ if (delta != 0) {
+ incEntryCount(-delta);
+ }
+ } else {
+ int delta = 0;
+ int tombstones = 0;
+ VersionSource myId = _getOwner().getVersionMember();
+ if (localRvv != rvv) {
+ localRvv.recordGCVersions(rvv);
+ }
+ final boolean isTraceEnabled = logger.isTraceEnabled();
+ for (RegionEntry re : regionEntries()) {
+ synchronized(re) {
+ Token value = re.getValueAsToken();
+ // if it's already being removed or the entry is being created we leave it alone
+ if (value == Token.REMOVED_PHASE1 || value == Token.REMOVED_PHASE2) {
+ continue;
+ }
+
+ VersionSource id = re.getVersionStamp().getMemberID();
+ if (id == null) {
+ id = myId;
+ }
+ if (rvv.contains(id, re.getVersionStamp().getRegionVersion())) {
+ if (isTraceEnabled) {
+ logger.trace("region clear op is removing {} {}", re.getKey(), re.getVersionStamp());
+ }
+
+ boolean tombstone = re.isTombstone();
+ // note: it.remove() did not reliably remove the entry so we use remove(K,V) here
+ if (_getMap().remove(re.getKey(), re)) {
+ if (OffHeapRegionEntryHelper.doesClearNeedToCheckForOffHeap()) {
+ GatewaySenderEventImpl.release(re._getValue()); // OFFHEAP _getValue ok
+ }
+ //If this is an overflow only region, we need to free the entry on
+ //disk at this point.
+ try {
+ re.removePhase1(lr, true);
+ } catch (RegionClearedException e) {
+ //do nothing, it's already cleared.
+ }
+ re.removePhase2();
+ lruEntryDestroy(re);
+ if (tombstone) {
+ _getOwner().incTombstoneCount(-1);
+ tombstones += 1;
+ } else {
+ delta += 1;
+ }
+ }
+ } else { // rvv does not contain this entry so it is retained
+ result.add(id);
+ }
+ }
+ }
+ _getOwner().updateSizeOnClearRegion(delta);
+ incEntryCount(-delta);
+ incEntryCount(-tombstones);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Size after clearing = {}", _getMap().size());
+ }
+ if (isTraceEnabled && _getMap().size() < 20) {
+ _getOwner().dumpBackingMap();
+ }
+ }
+ }
+ return result;
+ }
+
+ public void lruUpdateCallback()
+ {
+ // By default do nothing; LRU maps needs to override this method
+ }
+ public void lruUpdateCallback(boolean b)
+ {
+ // By default do nothing; LRU maps needs to override this method
+ }
+ public void lruUpdateCallback(int i)
+ {
+ // By default do nothing; LRU maps needs to override this method
+ }
+
+ public boolean disableLruUpdateCallback()
+ {
+ // By default do nothing; LRU maps needs to override this method
+ return false;
+ }
+
+ public void enableLruUpdateCallback()
+ {
+ // By default do nothing; LRU maps needs to override this method
+ }
+
+ public void resetThreadLocals()
+ {
+ // By default do nothing; LRU maps needs to override this method
+ }
+
+ /**
+ * Tell an LRU that a new entry has been created
+ */
+ protected void lruEntryCreate(RegionEntry e)
+ {
+ // do nothing by default
+ }
+
+ /**
+ * Tell an LRU that an existing entry has been destroyed
+ */
+ protected void lruEntryDestroy(RegionEntry e)
+ {
+ // do nothing by default
+ }
+
+ /**
+ * Tell an LRU that an existing entry has been modified
+ */
+ protected void lruEntryUpdate(RegionEntry e)
+ {
+ // do nothing by default
+ }
+
+ @Override
+ public void decTxRefCount(RegionEntry e)
+ {
+ LocalRegion lr = null;
+ if (_isOwnerALocalRegion()) {
+ lr = _getOwner();
+ }
+ e.decRefCount(null, lr);
+ }
+
+ public boolean lruLimitExceeded() {
+ return false;
+ }
+
+ public void lruCloseStats() {
+ // do nothing by default
+ }
+
+ public void lruEntryFaultIn(LRUEntry entry) {
+ // do nothing by default
+ }
+
+ /**
+ * Process an incoming version tag for concurrent operation detection.
+ * This must be done before modifying the region entry.
+ * @param re the entry that is to be modified
+ * @param event the modification to the entry
+ * @throws InvalidDeltaException if the event contains a delta that cannot be applied
+ * @throws ConcurrentCacheModificationException if the event is in conflict
+ * with a previously applied change
+ */
+ private void processVersionTag(RegionEntry re, EntryEventImpl event) {
+ if (re.getVersionStamp() != null) {
+ re.getVersionStamp().processVersionTag(event);
+
+ // during initialization we record version tag info to detect ops the
+ // image provider hasn't seen
+ VersionTag> tag = event.getVersionTag();
+ if (tag != null && !event.getRegion().isInitialized()) {
+ ImageState is = event.getRegion().getImageState();
+ if (is != null && !event.getRegion().isUsedForPartitionedRegionBucket()) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("recording version tag in image state: {}", tag);
+ }
+ is.addVersionTag(event.getKey(), tag);
+ }
+ }
+ }
+ }
+
+ private void processVersionTagForGII(RegionEntry re, LocalRegion owner, VersionTag entryVersion, boolean isTombstone, InternalDistributedMember sender, boolean checkConflicts) {
+
+ re.getVersionStamp().processVersionTag(_getOwner(), entryVersion, isTombstone, false, owner.getMyId(), sender, checkConflicts);
+ }
+
+ public void copyRecoveredEntries(RegionMap rm) {
+ //We need to sort the tombstones before scheduling them,
+ //so that they will be in the correct order.
+ OrderedTombstoneMap tombstones = new OrderedTombstoneMap();
+ if (rm != null) {
+ CustomEntryConcurrentHashMap other = ((AbstractRegionMap)rm)._getMap();
+ Iterator> it = other
+ .entrySetWithReusableEntries().iterator();
+ while (it.hasNext()) {
+ Map.Entry me = it.next();
+ it.remove(); // This removes the RegionEntry from "rm" but it does not decrement its refcount to an offheap value.
+ RegionEntry oldRe = (RegionEntry)me.getValue();
+ Object key = me.getKey();
+
+ @Retained @Released Object value = oldRe._getValueRetain((RegionEntryContext) ((AbstractRegionMap) rm)._getOwnerObject(), true);
+
+ try {
+ if (value == Token.NOT_AVAILABLE) {
+ // fix for bug 43993
+ value = null;
+ }
+ if (value == Token.TOMBSTONE && !_getOwner().getConcurrencyChecksEnabled()) {
+ continue;
+ }
+ RegionEntry newRe = getEntryFactory().createEntry((RegionEntryContext) _getOwnerObject(), key, value);
+ copyRecoveredEntry(oldRe, newRe);
+ // newRe is now in this._getMap().
+ if (newRe.isTombstone()) {
+ VersionTag tag = newRe.getVersionStamp().asVersionTag();
+ tombstones.put(tag, newRe);
+ }
+ _getOwner().updateSizeOnCreate(key, _getOwner().calculateRegionEntryValueSize(newRe));
+ incEntryCount(1);
+ lruEntryUpdate(newRe);
+ } finally {
+ if (OffHeapHelper.release(value)) {
+ ((OffHeapRegionEntry)oldRe).release();
+ }
+ }
+ lruUpdateCallback();
+ }
+ } else {
+ incEntryCount(size());
+ for (Iterator iter = regionEntries().iterator(); iter.hasNext(); ) {
+ RegionEntry re = iter.next();
+ if (re.isTombstone()) {
+ if (re.getVersionStamp() == null) { // bug #50992 - recovery from versioned to non-versioned
+ incEntryCount(-1);
+ iter.remove();
+ continue;
+ } else {
+ tombstones.put(re.getVersionStamp().asVersionTag(), re);
+ }
+ }
+ _getOwner().updateSizeOnCreate(re.getKey(), _getOwner().calculateRegionEntryValueSize(re));
+ }
+ // Since lru was not being done during recovery call it now.
+ lruUpdateCallback();
+ }
+
+ //Schedule all of the tombstones, now that we have sorted them
+ Map.Entry entry;
+ while((entry = tombstones.take()) != null) {
+ // refresh the tombstone so it doesn't time out too soon
+ _getOwner().scheduleTombstone(entry.getValue(), entry.getKey());
+ }
+
+ }
+
+ protected void copyRecoveredEntry(RegionEntry oldRe, RegionEntry newRe) {
+ if(newRe.getVersionStamp() != null) {
+ newRe.getVersionStamp().setMemberID(oldRe.getVersionStamp().getMemberID());
+ newRe.getVersionStamp().setVersions(oldRe.getVersionStamp().asVersionTag());
+ }
+
+ if (newRe instanceof AbstractOplogDiskRegionEntry) {
+ ((AbstractOplogDiskRegionEntry)newRe).setDiskId(oldRe);
+ _getOwner().getDiskRegion().replaceIncompatibleEntry((DiskEntry) oldRe, (DiskEntry) newRe);
+ }
+ _getMap().put(newRe.getKey(), newRe);
+ }
+
+ @Retained // Region entry may contain an off-heap value
+ public final RegionEntry initRecoveredEntry(Object key, DiskEntry.RecoveredEntry value) {
+ boolean needsCallback = false;
+ @Retained RegionEntry newRe = getEntryFactory().createEntry((RegionEntryContext) _getOwnerObject(), key, value);
+ synchronized (newRe) {
+ if (value.getVersionTag()!=null && newRe.getVersionStamp()!=null) {
+ newRe.getVersionStamp().setVersions(value.getVersionTag());
+ }
+ RegionEntry oldRe = putEntryIfAbsent(key, newRe);
+ while (oldRe != null) {
+ synchronized (oldRe) {
+ if (oldRe.isRemoved() && !oldRe.isTombstone()) {
+ oldRe = putEntryIfAbsent(key, newRe);
+ if (oldRe != null) {
+ if (_isOwnerALocalRegion()) {
+ _getOwner().getCachePerfStats().incRetries();
+ }
+ }
+ }
+ /*
+ * Entry already exists which should be impossible.
+ * Free the current entry (if off-heap) and
+ * throw an exception.
+ */
+ else {
+ if (newRe instanceof OffHeapRegionEntry) {
+ ((OffHeapRegionEntry) newRe).release();
+ }
+
+ throw new IllegalStateException("Could not recover entry for key " + key + ". The entry already exists!");
+ }
+ } // synchronized
+ }
+ if (_isOwnerALocalRegion()) {
+ _getOwner().updateSizeOnCreate(key, _getOwner().calculateRegionEntryValueSize(newRe));
+ if (newRe.isTombstone()) {
+ // refresh the tombstone so it doesn't time out too soon
+ _getOwner().scheduleTombstone(newRe, newRe.getVersionStamp().asVersionTag());
+ }
+
+ incEntryCount(1); // we are creating an entry that was recovered from disk including tombstone
+ }
+ lruEntryUpdate(newRe);
+ needsCallback = true;
+ }
+ if (needsCallback) {
+ lruUpdateCallback();
+ }
+
+ EntryLogger.logRecovery(_getOwnerObject(), key, value);
+
+ return newRe;
+ }
+
+ public final RegionEntry updateRecoveredEntry(Object key, DiskEntry.RecoveredEntry value) {
+ boolean needsCallback = false;
+ RegionEntry re = getEntry(key);
+ if (re == null) {
+ return null;
+ }
+ synchronized (re) {
+ if (re.isRemoved() && !re.isTombstone()) {
+ return null;
+ }
+ if (value.getVersionTag()!=null && re.getVersionStamp()!=null) {
+ re.getVersionStamp().setVersions(value.getVersionTag());
+ }
+ try {
+ if (_isOwnerALocalRegion()) {
+ if (re.isTombstone()) {
+ // when a tombstone is to be overwritten, unschedule it first
+ _getOwner().unscheduleTombstone(re);
+ }
+ final int oldSize = _getOwner().calculateRegionEntryValueSize(re);
+ re.setValue(_getOwner(), value); // OFFHEAP no need to call AbstractRegionMap.prepareValueForCache because setValue is overridden for disk and that code takes apart value (RecoveredEntry) and prepares its nested value for the cache
+ if (re.isTombstone()) {
+ _getOwner().scheduleTombstone(re, re.getVersionStamp().asVersionTag());
+ }
+ _getOwner().updateSizeOnPut(key, oldSize, _getOwner().calculateRegionEntryValueSize(re));
+ } else {
+ DiskEntry.Helper.updateRecoveredEntry((PlaceHolderDiskRegion)_getOwnerObject(),
+ (DiskEntry)re, value, (RegionEntryContext) _getOwnerObject());
+ }
+ } catch (RegionClearedException rce) {
+ throw new IllegalStateException("RegionClearedException should never happen in this context", rce);
+ }
+ lruEntryUpdate(re);
+ needsCallback = true;
+ }
+ if (needsCallback) {
+ lruUpdateCallback();
+ }
+
+ EntryLogger.logRecovery(_getOwnerObject(), key, value);
+
+ return re;
+ }
+
+ public final boolean initialImagePut(final Object key,
+ final long lastModified,
+ Object newValue,
+ final boolean wasRecovered,
+ boolean deferLRUCallback,
+ VersionTag entryVersion, InternalDistributedMember sender, boolean isSynchronizing)
+ {
+ boolean result = false;
+ boolean done = false;
+ boolean cleared = false;
+ final LocalRegion owner = _getOwner();
+
+ if (newValue == Token.TOMBSTONE && !owner.getConcurrencyChecksEnabled()) {
+ return false;
+ }
+
+ if (owner instanceof HARegion && newValue instanceof CachedDeserializable) {
+ Object actualVal = ((CachedDeserializable)newValue)
+ .getDeserializedValue(null, null);
+ if (actualVal instanceof HAEventWrapper) {
+ HAEventWrapper haEventWrapper = (HAEventWrapper)actualVal;
+ // Key was removed at sender side so not putting it into the HARegion
+ if (haEventWrapper.getClientUpdateMessage() == null) {
+ return false;
+ }
+ // Getting the instance from singleton CCN..This assumes only one bridge
+ // server in the VM
+ HAContainerWrapper haContainer = (HAContainerWrapper)CacheClientNotifier
+ .getInstance().getHaContainer();
+ Map.Entry entry = null;
+ HAEventWrapper original = null;
+ synchronized (haContainer) {
+ entry = (Map.Entry)haContainer.getEntry(haEventWrapper);
+ if (entry != null) {
+ original = (HAEventWrapper)entry.getKey();
+ original.incAndGetReferenceCount();
+ }
+ else {
+ haEventWrapper.incAndGetReferenceCount();
+ haEventWrapper.setHAContainer(haContainer);
+ haContainer.put(haEventWrapper, haEventWrapper
+ .getClientUpdateMessage());
+ haEventWrapper.setClientUpdateMessage(null);
+ haEventWrapper.setIsRefFromHAContainer(true);
+ }
+ }
+ if (entry != null) {
+ HARegionQueue.addClientCQsAndInterestList(entry, haEventWrapper,
+ haContainer, owner.getName());
+ haEventWrapper.setClientUpdateMessage(null);
+ newValue = CachedDeserializableFactory.create(original,
+ ((CachedDeserializable)newValue).getSizeInBytes());
+ }
+ }
+ }
+
+ try {
+ RegionEntry newRe = getEntryFactory().createEntry(owner, key,
+ Token.REMOVED_PHASE1);
+ EntryEventImpl event = null;
+
+ @Retained @Released Object oldValue = null;
+
+ try {
+ RegionEntry oldRe = null;
+ synchronized (newRe) {
+ try {
+ oldRe = putEntryIfAbsent(key, newRe);
+ while (!done && oldRe != null) {
+ synchronized (oldRe) {
+ if (oldRe.isRemovedPhase2()) {
+ oldRe = putEntryIfAbsent(key, newRe);
+ if (oldRe != null) {
+ owner.getCachePerfStats().incRetries();
+ }
+ }
+ else {
+ boolean acceptedVersionTag = false;
+ if (entryVersion != null && owner.concurrencyChecksEnabled) {
+ Assert.assertTrue(entryVersion.getMemberID() != null, "GII entry versions must have identifiers");
+ try {
+ boolean isTombstone = (newValue == Token.TOMBSTONE);
+ // don't reschedule the tombstone if it hasn't changed
+ boolean isSameTombstone = oldRe.isTombstone() && isTombstone
+ && oldRe.getVersionStamp().asVersionTag()
+ .equals(entryVersion);
+ if (isSameTombstone) {
+ return true;
+ }
+ processVersionTagForGII(oldRe, owner, entryVersion, isTombstone, sender, !wasRecovered || isSynchronizing);
+ acceptedVersionTag = true;
+ } catch (ConcurrentCacheModificationException e) {
+ return false;
+ }
+ }
+ final boolean oldIsTombstone = oldRe.isTombstone();
+ final int oldSize = owner.calculateRegionEntryValueSize(oldRe);
+ // Neeraj: The below if block is to handle the special
+ // scenario witnessed in SqlFabric for now. (Though its
+ // a general scenario). The scenario is that during GII
+ // it is possible that updates start coming before the
+ // base value reaches through GII. In that scenario the deltas
+ // for that particular key is kept on being added to a list
+ // of deltas. When the base value arrives through this path
+ // of GII the oldValue will be that list of deltas. When the
+ // base values arrives the deltas are applied one by one on that list.
+ // The same scenario is applicable for GemFire also but the below
+ // code will be executed only in case of sqlfabric now. Probably
+ // the code can be made more generic for both SQL Fabric and GemFire.
+ if (indexUpdater != null) {
+ oldValue = oldRe.getValueInVM(owner); // OFFHEAP: ListOfDeltas
+ if (oldValue instanceof ListOfDeltas) {
+ // apply the deltas on this new value. update index
+ // Make a new event object
+ // make it an insert operation
+ LocalRegion rgn = owner;
+ if (owner instanceof BucketRegion) {
+ rgn = ((BucketRegion)owner).getPartitionedRegion();
+ }
+ event = EntryEventImpl.create(rgn, Operation.CREATE, key, null,
+ Boolean.TRUE /* indicate that GII is in progress */,
+ false, null);
+ try {
+ event.setOldValue(newValue);
+ if (logger.isDebugEnabled()) {
+ logger.debug("initialImagePut: received base value for list of deltas; event: {}", event);
+ }
+ ((ListOfDeltas)oldValue).apply(event);
+ Object preparedNewValue =oldRe.prepareValueForCache(owner,
+ event.getNewValueAsOffHeapDeserializedOrRaw(), true);
+ if(preparedNewValue instanceof Chunk) {
+ event.setNewValue(preparedNewValue);
+ }
+ oldRe.setValue(owner, preparedNewValue, event);
+ //event.setNewValue(event.getOldValue());
+ event.setOldValue(null);
+ try {
+ indexUpdater.onEvent(owner, event, oldRe);
+ lruEntryUpdate(oldRe);
+ owner.updateSizeOnPut(key, oldSize, owner.calculateRegionEntryValueSize(oldRe));
+ EntryLogger.logInitialImagePut(_getOwnerObject(), key, newValue);
+ result = true;
+ done = true;
+ break;
+ } finally {
+ // this must be done within the oldRe sync block
+ indexUpdater.postEvent(owner, event, oldRe, done);
+ }
+ } finally {
+ if (event != null) {
+ event.release();
+ event = null;
+ }
+ }
+ }
+ }
+ try {
+ if (indexUpdater != null) {
+ event = EntryEventImpl.create(owner, Operation.CREATE, key,
+ newValue,
+ Boolean.TRUE /* indicate that GII is in progress */,
+ false, null);
+ indexUpdater.onEvent(owner, event, oldRe);
+ }
+ result = oldRe.initialImagePut(owner, lastModified, newValue, wasRecovered, acceptedVersionTag);
+ if (result) {
+ if (oldIsTombstone) {
+ owner.unscheduleTombstone(oldRe);
+ if (newValue != Token.TOMBSTONE){
+ lruEntryCreate(oldRe);
+ } else {
+ lruEntryUpdate(oldRe);
+ }
+ }
+ if (newValue == Token.TOMBSTONE) {
+ if (owner.getServerProxy() == null &&
+ owner.getVersionVector().isTombstoneTooOld(entryVersion.getMemberID(), entryVersion.getRegionVersion())) {
+ // the received tombstone has already been reaped, so don't retain it
+ removeTombstone(oldRe, entryVersion, false, false);
+ return false;
+ } else {
+ owner.scheduleTombstone(oldRe, entryVersion);
+ lruEntryDestroy(oldRe);
+ }
+ } else {
+ int newSize = owner.calculateRegionEntryValueSize(oldRe);
+ if(!oldIsTombstone) {
+ owner.updateSizeOnPut(key, oldSize, newSize);
+ } else {
+ owner.updateSizeOnCreate(key, newSize);
+ }
+ EntryLogger.logInitialImagePut(_getOwnerObject(), key, newValue);
+ }
+ }
+ if (owner.getIndexManager() != null) {
+ owner.getIndexManager().updateIndexes(oldRe, oldRe.isRemoved() ? IndexManager.ADD_ENTRY : IndexManager.UPDATE_ENTRY,
+ oldRe.isRemoved() ? IndexProtocol.OTHER_OP : IndexProtocol.AFTER_UPDATE_OP);
+ }
+ done = true;
+ } finally {
+ if (indexUpdater != null) {
+ indexUpdater.postEvent(owner, event, oldRe, result);
+ }
+ if (event != null) {
+ event.release();
+ event = null;
+ }
+ }
+ }
+ }
+ }
+ if (!done) {
+ boolean versionTagAccepted = false;
+ if (entryVersion != null && owner.concurrencyChecksEnabled) {
+ Assert.assertTrue(entryVersion.getMemberID() != null, "GII entry versions must have identifiers");
+ try {
+ boolean isTombstone = (newValue == Token.TOMBSTONE);
+ processVersionTagForGII(newRe, owner, entryVersion, isTombstone, sender, !wasRecovered || isSynchronizing);
+ versionTagAccepted = true;
+ } catch (ConcurrentCacheModificationException e) {
+ return false;
+ }
+ }
+ result = newRe.initialImageInit(owner, lastModified, newValue,
+ true, wasRecovered, versionTagAccepted);
+ try {
+ if (result) {
+ if (indexUpdater != null) {
+ event = EntryEventImpl.create(owner, Operation.CREATE, key,
+ newValue,
+ Boolean.TRUE /* indicate that GII is in progress */,
+ false, null);
+ indexUpdater.onEvent(owner, event, newRe);
+ }
+ if (newValue == Token.TOMBSTONE) {
+ owner.scheduleTombstone(newRe, entryVersion);
+ } else {
+ owner.updateSizeOnCreate(key, owner.calculateRegionEntryValueSize(newRe));
+ EntryLogger.logInitialImagePut(_getOwnerObject(), key, newValue);
+ lruEntryCreate(newRe);
+ }
+ incEntryCount(1);
+ }
+ //Update local indexes
+ if (owner.getIndexManager() != null) {
+ owner.getIndexManager().updateIndexes(newRe, newRe.isRemoved() ? IndexManager.REMOVE_ENTRY : IndexManager.UPDATE_ENTRY,
+ newRe.isRemoved() ? IndexProtocol.OTHER_OP : IndexProtocol.AFTER_UPDATE_OP);
+ }
+ done = true;
+ } finally {
+ if (result && indexUpdater != null) {
+ indexUpdater.postEvent(owner, event, newRe, done);
+ }
+ if (event != null) {
+ event.release();
+ event = null;
+ }
+ }
+ }
+ }
+ finally {
+ if (done && result) {
+ initialImagePutEntry(newRe);
+ }
+ if (!done) {
+ removeEntry(key, newRe, false);
+ if (owner.getIndexManager() != null) {
+ owner.getIndexManager().updateIndexes(newRe, IndexManager.REMOVE_ENTRY, IndexProtocol.OTHER_OP);
+ }
+ }
+ }
+ } // synchronized
+ } finally {
+ if (event != null) event.release();
+ OffHeapHelper.release(oldValue);
+ }
+ } catch(RegionClearedException rce) {
+ //Asif: do not issue any sort of callbacks
+ done = false;
+ cleared= true;
+ }catch(QueryException qe) {
+ done = false;
+ cleared= true;
+ }
+ finally {
+ if (done && !deferLRUCallback) {
+ lruUpdateCallback();
+ }
+ else if (!cleared) {
+ resetThreadLocals();
+ }
+ }
+ return result;
+ }
+
+ protected void initialImagePutEntry(RegionEntry newRe) {
+ }
+
+ boolean confirmEvictionDestroy(RegionEntry re)
+ {
+ /* We arn't in an LRU context, and should never get here */
+ Assert.assertTrue(false,
+ "Not an LRU region, can not confirm LRU eviction operation");
+ return true;
+ }
+
+ public final boolean destroy(EntryEventImpl event,
+ boolean inTokenMode,
+ boolean duringRI,
+ boolean cacheWrite,
+ boolean isEviction,
+ Object expectedOldValue,
+ boolean removeRecoveredEntry)
+ throws CacheWriterException, EntryNotFoundException, TimeoutException {
+
+ final LocalRegion owner = _getOwner();
+
+ if (owner == null) {
+ Assert.assertTrue(false, "The owner for RegionMap " + this // "fix" for bug 32440
+ + " is null for event " + event);
+ }
+
+ boolean retry = true;
+
+ while (retry) {
+ retry = false;
+
+ boolean opCompleted = false;
+ boolean doPart3 = false;
+
+ // We need to acquire the region entry while holding the lock to avoid #45620.
+ // However, we also want to release the lock before distribution to prevent
+ // potential deadlocks. The outer try/finally ensures that the lock will be
+ // released without fail. I'm avoiding indenting just to preserve the ability
+ // to track diffs since the code is fairly complex.
+ boolean doUnlock = true;
+ lockForCacheModification(owner, event);
+ try {
+
+
+ RegionEntry re = getOrCreateRegionEntry(owner, event, Token.REMOVED_PHASE1, null, true, true);
+ RegionEntry tombstone = null;
+ boolean haveTombstone = false;
+ /*
+ * Execute the test hook runnable inline (not threaded) if it is not null.
+ */
+ if(null != testHookRunnableFor48182) {
+ testHookRunnableFor48182.run();
+ }
+
+ try {
+ if (logger.isTraceEnabled(LogMarker.LRU_TOMBSTONE_COUNT) && !(owner instanceof HARegion)) {
+ logger.trace(LogMarker.LRU_TOMBSTONE_COUNT,
+ "ARM.destroy() inTokenMode={}; duringRI={}; riLocalDestroy={}; withRepl={}; fromServer={}; concurrencyEnabled={}; isOriginRemote={}; isEviction={}; operation={}; re={}",
+ inTokenMode, duringRI, event.isFromRILocalDestroy(), owner.dataPolicy.withReplication(), event.isFromServer(),
+ owner.concurrencyChecksEnabled, event.isOriginRemote(), isEviction, event.getOperation(), re);
+ }
+ if (event.isFromRILocalDestroy()) {
+ // for RI local-destroy we don't want to keep tombstones.
+ // In order to simplify things we just set this recovery
+ // flag to true to force the entry to be removed
+ removeRecoveredEntry = true;
+ }
+ // the logic in this method is already very involved, and adding tombstone
+ // permutations to (re != null) greatly complicates it. So, we check
+ // for a tombstone here and, if found, pretend for a bit that the entry is null
+ if (re != null && re.isTombstone() && !removeRecoveredEntry) {
+ tombstone = re;
+ haveTombstone = true;
+ re = null;
+ }
+ IndexManager oqlIndexManager = owner.getIndexManager() ;
+ if (re == null) {
+ // we need to create an entry if in token mode or if we've received
+ // a destroy from a peer or WAN gateway and we need to retain version
+ // information for concurrency checks
+ boolean retainForConcurrency = (!haveTombstone
+ && (owner.dataPolicy.withReplication() || event.isFromServer())
+ && owner.concurrencyChecksEnabled
+ && (event.isOriginRemote() /* destroy received from other must create tombstone */
+ || event.isFromWANAndVersioned() /* wan event must create a tombstone */
+ || event.isBridgeEvent())); /* event from client must create a tombstone so client has a version # */
+ if (inTokenMode
+ || retainForConcurrency) {
+ // removeRecoveredEntry should be false in this case
+ RegionEntry newRe = getEntryFactory().createEntry(owner,
+ event.getKey(),
+ Token.REMOVED_PHASE1);
+ // Fix for Bug #44431. We do NOT want to update the region and wait
+ // later for index INIT as region.clear() can cause inconsistency if
+ // happened in parallel as it also does index INIT.
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ try {
+ synchronized (newRe) {
+ RegionEntry oldRe = putEntryIfAbsent(event.getKey(), newRe);
+ while (!opCompleted && oldRe != null) {
+ synchronized (oldRe) {
+ if (oldRe.isRemovedPhase2()) {
+ oldRe = putEntryIfAbsent(event.getKey(), newRe);
+ if (oldRe != null) {
+ owner.getCachePerfStats().incRetries();
+ }
+ } else {
+ event.setRegionEntry(oldRe);
+
+ // Last transaction related eviction check. This should
+ // prevent
+ // transaction conflict (caused by eviction) when the entry
+ // is being added to transaction state.
+ if (isEviction) {
+ if (!confirmEvictionDestroy(oldRe) || (owner.getEvictionCriteria() != null && !owner.getEvictionCriteria().doEvict(event))) {
+ opCompleted = false;
+ return opCompleted;
+ }
+ }
+ try {
+ //if concurrency checks are enabled, destroy will
+ //set the version tag
+ boolean destroyed = destroyEntry(oldRe, event, inTokenMode, cacheWrite, expectedOldValue, false, removeRecoveredEntry);
+ if (destroyed) {
+ if (retainForConcurrency) {
+ owner.basicDestroyBeforeRemoval(oldRe, event);
+ }
+ owner.basicDestroyPart2(oldRe, event, inTokenMode,
+ false /* conflict with clear */, duringRI, true);
+ lruEntryDestroy(oldRe);
+ doPart3 = true;
+ }
+ }
+ catch (RegionClearedException rce) {
+ // Ignore. The exception will ensure that we do not update
+ // the LRU List
+ owner.basicDestroyPart2(oldRe, event, inTokenMode,
+ true/* conflict with clear */, duringRI, true);
+ doPart3 = true;
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ }
+ re = oldRe;
+ opCompleted = true;
+ }
+ } // synchronized oldRe
+ } // while
+ if (!opCompleted) {
+ // The following try has a finally that cleans up the newRe.
+ // This is only needed if newRe was added to the map which only
+ // happens if we didn't get completed with oldRe in the above while loop.
+ try {
+ re = newRe;
+ event.setRegionEntry(newRe);
+
+ try {
+ //if concurrency checks are enabled, destroy will
+ //set the version tag
+ if (isEviction) {
+ opCompleted = false;
+ return opCompleted;
+ }
+ opCompleted = destroyEntry(newRe, event, inTokenMode, cacheWrite, expectedOldValue, true, removeRecoveredEntry);
+ if (opCompleted) {
+ // This is a new entry that was created because we are in
+ // token mode or are accepting a destroy operation by adding
+ // a tombstone. There is no oldValue, so we don't need to
+ // call updateSizeOnRemove
+ // owner.recordEvent(event);
+ event.setIsRedestroyedEntry(true); // native clients need to know if the entry didn't exist
+ if (retainForConcurrency) {
+ owner.basicDestroyBeforeRemoval(oldRe, event);
+ }
+ owner.basicDestroyPart2(newRe, event, inTokenMode,
+ false /* conflict with clear */, duringRI, true);
+ doPart3 = true;
+ }
+ }
+ catch (RegionClearedException rce) {
+ // Ignore. The exception will ensure that we do not update
+ // the LRU List
+ opCompleted = true;
+ EntryLogger.logDestroy(event);
+ owner.basicDestroyPart2(newRe, event, inTokenMode, true /* conflict with clear*/, duringRI, true);
+ doPart3 = true;
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ }
+ // Note no need for LRU work since the entry is destroyed
+ // and will be removed when gii completes
+ } finally {
+ if (!opCompleted && !haveTombstone /* to fix bug 51583 do this for all operations */ ) {
+ removeEntry(event.getKey(), newRe, false);
+ }
+ if (!opCompleted && isEviction) {
+ removeEntry(event.getKey(), newRe, false);
+ }
+ }
+ } // !opCompleted
+ } // synchronized newRe
+ } finally {
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ }
+ } // inTokenMode or tombstone creation
+ else {
+ if (!isEviction || owner.concurrencyChecksEnabled) {
+ // The following ensures that there is not a concurrent operation
+ // on the entry and leaves behind a tombstone if concurrencyChecksEnabled.
+ // It fixes bug #32467 by propagating the destroy to the server even though
+ // the entry isn't in the client
+ RegionEntry newRe = haveTombstone? tombstone : getEntryFactory().createEntry(owner, event.getKey(),
+ Token.REMOVED_PHASE1);
+ synchronized(newRe) {
+ if (haveTombstone && !tombstone.isTombstone()) {
+ // we have to check this again under synchronization since it may have changed
+ retry = true;
+ //retryEntry = tombstone; // leave this in place for debugging
+ continue;
+ }
+ re = (RegionEntry)_getMap().putIfAbsent(event.getKey(), newRe);
+ if (re != null && re != tombstone) {
+ // concurrent change - try again
+ retry = true;
+ //retryEntry = tombstone; // leave this in place for debugging
+ continue;
+ }
+ else if (!isEviction) {
+ boolean throwex = false;
+ EntryNotFoundException ex = null;
+ try {
+ if (!cacheWrite) {
+ throwex = true;
+ } else {
+ try {
+ if (!removeRecoveredEntry) {
+ throwex = !owner.bridgeWriteBeforeDestroy(event, expectedOldValue);
+ }
+ } catch (EntryNotFoundException e) {
+ throwex = true;
+ ex = e;
+ }
+ }
+ if (throwex) {
+ if (!event.isOriginRemote() && !event.getOperation().isLocal() &&
+ (event.isFromBridgeAndVersioned() || // if this is a replayed client event that already has a version
+ event.isFromWANAndVersioned())) { // or if this is a WAN event that has been applied in another system
+ // we must distribute these since they will update the version information in peers
+ if (logger.isDebugEnabled()) {
+ logger.debug("ARM.destroy is allowing wan/client destroy of {} to continue", event.getKey());
+ }
+ throwex = false;
+ event.setIsRedestroyedEntry(true);
+ // Distribution of this op happens on re and re might me null here before
+ // distributing this destroy op.
+ if (re == null) {
+ re = newRe;
+ }
+ doPart3 = true;
+ }
+ }
+ if (throwex) {
+ if (ex == null) {
+ // Fix for 48182, check cache state and/or region state before sending entry not found.
+ // this is from the server and any exceptions will propogate to the client
+ owner.checkEntryNotFound(event.getKey());
+ } else {
+ throw ex;
+ }
+ }
+ } finally {
+ // either remove the entry or leave a tombstone
+ try {
+ if (!event.isOriginRemote() && event.getVersionTag() != null && owner.concurrencyChecksEnabled) {
+ // this shouldn't fail since we just created the entry.
+ // it will either generate a tag or apply a server's version tag
+ processVersionTag(newRe, event);
+ if (doPart3) {
+ owner.generateAndSetVersionTag(event, newRe);
+ }
+ try {
+ owner.recordEvent(event);
+ newRe.makeTombstone(owner, event.getVersionTag());
+ } catch (RegionClearedException e) {
+ // that's okay - when writing a tombstone into a disk, the
+ // region has been cleared (including this tombstone)
+ }
+ opCompleted = true;
+ // lruEntryCreate(newRe);
+ } else if (!haveTombstone) {
+ try {
+ assert newRe != tombstone;
+ newRe.setValue(owner, Token.REMOVED_PHASE2);
+ removeEntry(event.getKey(), newRe, false);
+ } catch (RegionClearedException e) {
+ // that's okay - we just need to remove the new entry
+ }
+ } else if (event.getVersionTag() != null ) { // haveTombstone - update the tombstone version info
+ processVersionTag(tombstone, event);
+ if (doPart3) {
+ owner.generateAndSetVersionTag(event, newRe);
+ }
+ // This is not conflict, we need to persist the tombstone again with new version tag
+ try {
+ tombstone.setValue(owner, Token.TOMBSTONE);
+ } catch (RegionClearedException e) {
+ // that's okay - when writing a tombstone into a disk, the
+ // region has been cleared (including this tombstone)
+ }
+ owner.recordEvent(event);
+ owner.rescheduleTombstone(tombstone, event.getVersionTag());
+ owner.basicDestroyPart2(tombstone, event, inTokenMode, true /* conflict with clear*/, duringRI, true);
+ opCompleted = true;
+ }
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ }
+ }
+ }
+ } // synchronized(newRe)
+ }
+ }
+ } // no current entry
+ else { // current entry exists
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ try {
+ synchronized (re) {
+ // if the entry is a tombstone and the event is from a peer or a client
+ // then we allow the operation to be performed so that we can update the
+ // version stamp. Otherwise we would retain an old version stamp and may allow
+ // an operation that is older than the destroy() to be applied to the cache
+ // Bug 45170: If removeRecoveredEntry, we treat tombstone as regular entry to be deleted
+ boolean createTombstoneForConflictChecks = (owner.concurrencyChecksEnabled
+ && (event.isOriginRemote() || event.getContext() != null || removeRecoveredEntry));
+ if (!re.isRemoved() || createTombstoneForConflictChecks) {
+ if (re.isRemovedPhase2()) {
+ retry = true;
+ continue;
+ }
+ if (!event.isOriginRemote() && event.getOperation().isExpiration()) {
+ // If this expiration started locally then only do it if the RE is not being used by a tx.
+ if (re.isInUseByTransaction()) {
+ opCompleted = false;
+ return opCompleted;
+ }
+ }
+ event.setRegionEntry(re);
+
+ // See comment above about eviction checks
+ if (isEviction) {
+ assert expectedOldValue == null;
+ if (!confirmEvictionDestroy(re) || (owner.getEvictionCriteria() != null && !owner.getEvictionCriteria().doEvict(event))) {
+ opCompleted = false;
+ return opCompleted;
+ }
+ }
+
+ boolean removed = false;
+ try {
+ opCompleted = destroyEntry(re, event, inTokenMode, cacheWrite, expectedOldValue, false, removeRecoveredEntry);
+ if (opCompleted) {
+ // It is very, very important for Partitioned Regions to keep
+ // the entry in the map until after distribution occurs so that other
+ // threads performing a create on this entry wait until the destroy
+ // distribution is finished.
+ // keeping backup copies consistent. Fix for bug 35906.
+ // -- mthomas 07/02/2007 <-- how about that date, kinda cool eh?
+ owner.basicDestroyBeforeRemoval(re, event);
+
+ // do this before basicDestroyPart2 to fix bug 31786
+ if (!inTokenMode) {
+ if ( re.getVersionStamp() == null) {
+ re.removePhase2();
+ removeEntry(event.getKey(), re, true, event, owner,
+ indexUpdater);
+ removed = true;
+ }
+ }
+ if (inTokenMode && !duringRI) {
+ event.inhibitCacheListenerNotification(true);
+ }
+ doPart3 = true;
+ owner.basicDestroyPart2(re, event, inTokenMode, false /* conflict with clear*/, duringRI, true);
+ // if (!re.isTombstone() || isEviction) {
+ lruEntryDestroy(re);
+ // } else {
+ // lruEntryUpdate(re);
+ // lruUpdateCallback = true;
+ // }
+ } else {
+ if (!inTokenMode) {
+ EntryLogger.logDestroy(event);
+ owner.recordEvent(event);
+ if (re.getVersionStamp() == null) {
+ re.removePhase2();
+ removeEntry(event.getKey(), re, true, event, owner,
+ indexUpdater);
+ lruEntryDestroy(re);
+ } else {
+ if (re.isTombstone()) {
+ // the entry is already a tombstone, but we're destroying it
+ // again, so we need to reschedule the tombstone's expiration
+ if (event.isOriginRemote()) {
+ owner.rescheduleTombstone(re, re.getVersionStamp().asVersionTag());
+ }
+ }
+ }
+ lruEntryDestroy(re);
+ opCompleted = true;
+ }
+ }
+ }
+ catch (RegionClearedException rce) {
+ // Ignore. The exception will ensure that we do not update
+ // the LRU List
+ opCompleted = true;
+ owner.recordEvent(event);
+ if (inTokenMode && !duringRI) {
+ event.inhibitCacheListenerNotification(true);
+ }
+ owner.basicDestroyPart2(re, event, inTokenMode, true /*conflict with clear*/, duringRI, true);
+ doPart3 = true;
+ }
+ finally {
+ if (re.isRemoved() && !re.isTombstone()) {
+ if (!removed) {
+ removeEntry(event.getKey(), re, true, event, owner,
+ indexUpdater);
+ }
+ }
+ }
+ } // !isRemoved
+ else { // already removed
+ if (owner.isHDFSReadWriteRegion() && re.isRemovedPhase2()) {
+ // For HDFS region there may be a race with eviction
+ // so retry the operation. fixes bug 49150
+ retry = true;
+ continue;
+ }
+ if (re.isTombstone() && event.getVersionTag() != null) {
+ // if we're dealing with a tombstone and this is a remote event
+ // (e.g., from cache client update thread) we need to update
+ // the tombstone's version information
+ // TODO use destroyEntry() here
+ processVersionTag(re, event);
+ try {
+ re.makeTombstone(owner, event.getVersionTag());
+ } catch (RegionClearedException e) {
+ // that's okay - when writing a tombstone into a disk, the
+ // region has been cleared (including this tombstone)
+ }
+ }
+ if (expectedOldValue != null) {
+ // if re is removed then there is no old value, so return false
+ return false;
+ }
+
+ if (!inTokenMode && !isEviction) {
+ owner.checkEntryNotFound(event.getKey());
+ }
+ }
+ } // synchronized re
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ } finally {
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ }
+ // No need to call lruUpdateCallback since the only lru action
+ // we may have taken was lruEntryDestroy. This fixes bug 31759.
+
+ } // current entry exists
+ if(opCompleted) {
+ EntryLogger.logDestroy(event);
+ }
+ return opCompleted;
+ }
+ finally {
+ releaseCacheModificationLock(owner, event);
+ doUnlock = false;
+
+ try {
+ // If concurrency conflict is there and event contains gateway version tag then
+ // do NOT distribute.
+ if (event.isConcurrencyConflict() &&
+ (event.getVersionTag() != null && event.getVersionTag().isGatewayTag())) {
+ doPart3 = false;
+ }
+ // distribution and listener notification
+ if (doPart3) {
+ owner.basicDestroyPart3(re, event, inTokenMode, duringRI, true, expectedOldValue);
+ }
+ } finally {
+ if (opCompleted) {
+ if (re != null) {
+ owner.cancelExpiryTask(re);
+ } else if (tombstone != null) {
+ owner.cancelExpiryTask(tombstone);
+ }
+ }
+ }
+ }
+
+ } finally { // failsafe on the read lock...see comment above
+ if (doUnlock) {
+ releaseCacheModificationLock(owner, event);
+ }
+ }
+ } // retry loop
+ return false;
+ }
+
+ public final void txApplyDestroy(Object key, TransactionId txId,
+ TXRmtEvent txEvent, boolean inTokenMode, boolean inRI, Operation op,
+ EventID eventId, Object aCallbackArgument,List pendingCallbacks,FilterRoutingInfo filterRoutingInfo,ClientProxyMembershipID bridgeContext,
+ boolean isOriginRemote, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+ final boolean isDebugEnabled = logger.isDebugEnabled();
+
+ final LocalRegion owner = _getOwner();
+ owner.checkBeforeEntrySync(txEvent);
+
+ final boolean isRegionReady = !inTokenMode;
+ final boolean hasRemoteOrigin = !((TXId)txId).getMemberId().equals(owner.getMyId());
+ boolean cbEventInPending = false;
+ lockForTXCacheModification(owner, versionTag);
+ IndexManager oqlIndexManager = owner.getIndexManager() ;
+ try {
+ RegionEntry re = getEntry(key);
+ if (re != null) {
+ // Fix for Bug #44431. We do NOT want to update the region and wait
+ // later for index INIT as region.clear() can cause inconsistency if
+ // happened in parallel as it also does index INIT.
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ try {
+ synchronized (re) {
+ if (!re.isRemoved() || re.isTombstone()) {
+ EntryEventImpl sqlfEvent = null;
+ @Retained @Released Object oldValue = re.getValueInVM(owner);
+ try {
+ final int oldSize = owner.calculateRegionEntryValueSize(re);
+ // Create an entry event only if the calling context is
+ // a receipt of a TXCommitMessage AND there are callbacks installed
+ // for this region
+ boolean invokeCallbacks = shouldCreateCBEvent(owner, false/*isInvalidate*/, isRegionReady || inRI);
+ EntryEventImpl cbEvent = createCBEvent(owner, op,
+ key, null, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+
+ if (/* owner.isUsedForPartitionedRegionBucket() && */
+ indexUpdater != null) {
+ sqlfEvent = cbEvent;
+ } else {
+ if (owner.isUsedForPartitionedRegionBucket()) {
+ txHandleWANEvent(owner, cbEvent, txEntryState);
+ }
+ cbEvent.setRegionEntry(re);
+ }
+ cbEvent.setOldValue(oldValue);
+ if (isDebugEnabled) {
+ logger.debug("txApplyDestroy cbEvent={}", cbEvent);
+ }
+
+ txRemoveOldIndexEntry(Operation.DESTROY, re);
+ if (txEvent != null) {
+ txEvent.addDestroy(owner, re, re.getKey(),aCallbackArgument);
+ }
+ boolean clearOccured = false;
+ try {
+ processAndGenerateTXVersionTag(owner, cbEvent, re, txEntryState);
+ if (inTokenMode) {
+ if (oldValue == Token.TOMBSTONE) {
+ owner.unscheduleTombstone(re);
+ }
+ re.setValue(owner, Token.DESTROYED);
+ }
+ else {
+ if (!re.isTombstone()) {
+ if (sqlfEvent != null) {
+ re.removePhase1(owner, false); // fix for bug 43063
+ re.removePhase2();
+ removeEntry(key, re, true, sqlfEvent, owner, indexUpdater);
+ } else {
+ if (shouldPerformConcurrencyChecks(owner, cbEvent) && cbEvent.getVersionTag() != null) {
+ re.makeTombstone(owner, cbEvent.getVersionTag());
+ } else {
+ re.removePhase1(owner, false); // fix for bug 43063
+ re.removePhase2();
+ removeEntry(key, re, false);
+ }
+ }
+ } else {
+ owner.rescheduleTombstone(re, re.getVersionStamp().asVersionTag());
+ }
+ }
+ EntryLogger.logTXDestroy(_getOwnerObject(), key);
+ owner.updateSizeOnRemove(key, oldSize);
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ owner.txApplyDestroyPart2(re, re.getKey(), inTokenMode,
+ clearOccured /* Clear Conflciting with the operation */);
+ if (invokeCallbacks) {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_DESTROY,
+ cbEvent, true/*callDispatchListenerEvent*/);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ if (!clearOccured) {
+ lruEntryDestroy(re);
+ }
+ if (owner.concurrencyChecksEnabled && txEntryState != null && cbEvent!= null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ } finally {
+ OffHeapHelper.release(oldValue);
+ }
+ }
+ }
+ } finally {
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ }
+ } else if (inTokenMode || owner.concurrencyChecksEnabled) {
+ // treating tokenMode and re == null as same, since we now want to
+ // generate versions and Tombstones for destroys
+ boolean dispatchListenerEvent = inTokenMode;
+ boolean opCompleted = false;
+ RegionEntry newRe = getEntryFactory().createEntry(owner, key,
+ Token.DESTROYED);
+ if ( oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ EntryEventImpl cbEvent = null;
+ try {
+ synchronized (newRe) {
+ RegionEntry oldRe = putEntryIfAbsent(key, newRe);
+ while (!opCompleted && oldRe != null) {
+ synchronized (oldRe) {
+ if (oldRe.isRemovedPhase2()) {
+ oldRe = putEntryIfAbsent(key, newRe);
+ if (oldRe != null) {
+ owner.getCachePerfStats().incRetries();
+ }
+ }
+ else {
+ try {
+ boolean invokeCallbacks = shouldCreateCBEvent(owner, false, isRegionReady || inRI);
+ cbEvent = createCBEvent(owner, op,
+ key, null, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ cbEvent.setRegionEntry(oldRe);
+ cbEvent.setOldValue(Token.NOT_AVAILABLE);
+ if (isDebugEnabled) {
+ logger.debug("txApplyDestroy token mode cbEvent={}", cbEvent);
+ }
+ if (owner.isUsedForPartitionedRegionBucket()) {
+ txHandleWANEvent(owner, cbEvent, txEntryState);
+ }
+ processAndGenerateTXVersionTag(owner, cbEvent, oldRe, txEntryState);
+ if (invokeCallbacks) {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_DESTROY,
+ cbEvent, dispatchListenerEvent);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ int oldSize = 0;
+ boolean wasTombstone = oldRe.isTombstone();
+ {
+ if (!wasTombstone) {
+ oldSize = owner.calculateRegionEntryValueSize(oldRe);
+ }
+ }
+ oldRe.setValue(owner, Token.DESTROYED);
+ EntryLogger.logTXDestroy(_getOwnerObject(), key);
+ if (wasTombstone) {
+ owner.unscheduleTombstone(oldRe);
+ }
+ owner.updateSizeOnRemove(oldRe.getKey(), oldSize);
+ owner.txApplyDestroyPart2(oldRe, oldRe.getKey(), inTokenMode,
+ false /* Clear Conflicting with the operation */);
+ lruEntryDestroy(oldRe);
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ }
+ catch (RegionClearedException rce) {
+ owner.txApplyDestroyPart2(oldRe, oldRe.getKey(), inTokenMode,
+ true /* Clear Conflicting with the operation */);
+ }
+ if (shouldPerformConcurrencyChecks(owner, cbEvent) && cbEvent.getVersionTag() != null) {
+ oldRe.makeTombstone(owner, cbEvent.getVersionTag());
+ } else if (!inTokenMode) {
+ // only remove for NORMAL regions if they do not generate versions see 51781
+ oldRe.removePhase1(owner, false); // fix for bug 43063
+ oldRe.removePhase2();
+ removeEntry(key, oldRe, false);
+ }
+ opCompleted = true;
+ }
+ }
+ }
+ if (!opCompleted) {
+ // already has value set to Token.DESTROYED
+ opCompleted = true;
+ boolean invokeCallbacks = shouldCreateCBEvent(owner, false, isRegionReady || inRI);
+ cbEvent = createCBEvent(owner, op,
+ key, null, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ cbEvent.setRegionEntry(newRe);
+ cbEvent.setOldValue(Token.NOT_AVAILABLE);
+ if (isDebugEnabled) {
+ logger.debug("txApplyDestroy token mode cbEvent={}", cbEvent);
+ }
+ if (owner.isUsedForPartitionedRegionBucket()) {
+ txHandleWANEvent(owner, cbEvent, txEntryState);
+ }
+ processAndGenerateTXVersionTag(owner, cbEvent, newRe, txEntryState);
+ if (invokeCallbacks) {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_DESTROY,
+ cbEvent, dispatchListenerEvent);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ EntryLogger.logTXDestroy(_getOwnerObject(), key);
+ owner.updateSizeOnCreate(newRe.getKey(), 0);
+ if (shouldPerformConcurrencyChecks(owner, cbEvent) && cbEvent.getVersionTag() != null) {
+ newRe.makeTombstone(owner, cbEvent.getVersionTag());
+ } else if (!inTokenMode) {
+ // only remove for NORMAL regions if they do not generate versions see 51781
+ newRe.removePhase1(owner, false); // fix for bug 43063
+ newRe.removePhase2();
+ removeEntry(key, newRe, false);
+ }
+ owner
+ .txApplyDestroyPart2(newRe, newRe.getKey(), inTokenMode,
+ false /*clearConflict*/);
+ // Note no need for LRU work since the entry is destroyed
+ // and will be removed when gii completes
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ }
+ if (owner.concurrencyChecksEnabled && txEntryState != null && cbEvent != null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ }
+ } catch (RegionClearedException e) {
+ // TODO
+ } finally {
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ }
+ } else if (re == null) {
+ // Fix bug#43594
+ // In cases where bucket region is re-created, it may so happen that
+ // the destroy is already applied on the Initial image provider, thus
+ // causing region entry to be absent.
+ // Notify clients with client events.
+ EntryEventImpl cbEvent = createCBEvent(owner, op,
+ key, null, txId, txEvent, eventId, aCallbackArgument,
+ filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ if (owner.isUsedForPartitionedRegionBucket()) {
+ txHandleWANEvent(owner, cbEvent, txEntryState);
+ }
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if (pendingCallbacks == null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_DESTROY,cbEvent,false);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ }
+ } catch( DiskAccessException dae) {
+ owner.handleDiskAccessException(dae);
+ throw dae;
+ }
+ finally {
+ releaseTXCacheModificationLock(owner, versionTag);
+ }
+ }
+
+ /**
+ * If true then invalidates that throw EntryNotFoundException
+ * or that are already invalid will first call afterInvalidate on CacheListeners.
+ * The old value on the event passed to afterInvalidate will be null.
+ */
+ public static boolean FORCE_INVALIDATE_EVENT = Boolean.getBoolean("gemfire.FORCE_INVALIDATE_EVENT");
+
+ /**
+ * If the FORCE_INVALIDATE_EVENT flag is true
+ * then invoke callbacks on the given event.
+ */
+ void forceInvalidateEvent(EntryEventImpl event) {
+ if (FORCE_INVALIDATE_EVENT) {
+ event.invokeCallbacks(_getOwner(), false, false);
+ }
+ }
+
+ public final boolean invalidate(EntryEventImpl event,
+ boolean invokeCallbacks, boolean forceNewEntry, boolean forceCallbacks)
+ throws EntryNotFoundException
+ {
+ final boolean isDebugEnabled = logger.isDebugEnabled();
+
+ final LocalRegion owner = _getOwner();
+ if (owner == null) {
+ // "fix" for bug 32440
+ Assert.assertTrue(false, "The owner for RegionMap " + this
+ + " is null for event " + event);
+
+ }
+ boolean didInvalidate = false;
+ RegionEntry invalidatedRe = null;
+ boolean clearOccured = false;
+
+ DiskRegion dr = owner.getDiskRegion();
+ // Fix for Bug #44431. We do NOT want to update the region and wait
+ // later for index INIT as region.clear() can cause inconsistency if
+ // happened in parallel as it also does index INIT.
+ IndexManager oqlIndexManager = owner.getIndexManager() ;
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ lockForCacheModification(owner, event);
+ try {
+ if (forceNewEntry || forceCallbacks) {
+ boolean opCompleted = false;
+ RegionEntry newRe = getEntryFactory().createEntry(owner, event.getKey(),
+ Token.REMOVED_PHASE1);
+ synchronized (newRe) {
+ try {
+ RegionEntry oldRe = putEntryIfAbsent(event.getKey(), newRe);
+
+ while (!opCompleted && oldRe != null) {
+ synchronized (oldRe) {
+ // if the RE is in phase 2 of removal, it will really be removed
+ // from the map. Otherwise, we can use it here and the thread
+ // that is destroying the RE will see the invalidation and not
+ // proceed to phase 2 of removal.
+ if (oldRe.isRemovedPhase2()) {
+ oldRe = putEntryIfAbsent(event.getKey(), newRe);
+ if (oldRe != null) {
+ owner.getCachePerfStats().incRetries();
+ }
+ } else {
+ opCompleted = true;
+ event.setRegionEntry(oldRe);
+ if (oldRe.isDestroyed()) {
+ if (isDebugEnabled) {
+ logger.debug("mapInvalidate: Found DESTROYED token, not invalidated; key={}", event.getKey());
+ }
+ } else if (oldRe.isInvalid()) {
+
+ // was already invalid, do not invoke listeners or increment stat
+ if (isDebugEnabled) {
+ logger.debug("mapInvalidate: Entry already invalid: '{}'", event.getKey());
+ }
+ processVersionTag(oldRe, event);
+ try {
+ oldRe.setValue(owner, oldRe.getValueInVM(owner)); // OFFHEAP noop setting an already invalid to invalid; No need to call prepareValueForCache since it is an invalid token.
+ } catch (RegionClearedException e) {
+ // that's okay - when writing an invalid into a disk, the
+ // region has been cleared (including this token)
+ }
+ forceInvalidateEvent(event);
+ } else {
+ owner.cacheWriteBeforeInvalidate(event, invokeCallbacks, forceNewEntry);
+ if (owner.concurrencyChecksEnabled && event.noVersionReceivedFromServer()) {
+ // server did not perform the invalidation, so don't leave an invalid
+ // entry here
+ return false;
+ }
+ final int oldSize = owner.calculateRegionEntryValueSize(oldRe);
+ //added for cq which needs old value. rdubey
+ FilterProfile fp = owner.getFilterProfile();
+ if (!oldRe.isRemoved() &&
+ (fp != null && fp.getCqCount() > 0)) {
+
+ @Retained @Released Object oldValue = oldRe.getValueInVM(owner); // OFFHEAP EntryEventImpl oldValue
+
+ // this will not fault in the value.
+ try {
+ if (oldValue == Token.NOT_AVAILABLE){
+ event.setOldValue(oldRe.getValueOnDiskOrBuffer(owner));
+ } else {
+ event.setOldValue(oldValue);
+ }
+ } finally {
+ OffHeapHelper.release(oldValue);
+ }
+ }
+ boolean isCreate = false;
+ try {
+ if (oldRe.isRemoved()) {
+ processVersionTag(oldRe, event);
+ event.putNewEntry(owner, oldRe);
+ EntryLogger.logInvalidate(event);
+ owner.recordEvent(event);
+ if (!oldRe.isTombstone()) {
+ owner.updateSizeOnPut(event.getKey(), oldSize, event.getNewValueBucketSize());
+ } else {
+ owner.updateSizeOnCreate(event.getKey(), event.getNewValueBucketSize());
+ isCreate = true;
+ }
+ } else {
+ processVersionTag(oldRe, event);
+ event.putExistingEntry(owner, oldRe);
+ EntryLogger.logInvalidate(event);
+ owner.recordEvent(event);
+ owner.updateSizeOnPut(event.getKey(), oldSize, event.getNewValueBucketSize());
+ }
+ }
+ catch (RegionClearedException e) {
+ // generate versionTag for the event
+ EntryLogger.logInvalidate(event);
+ owner.recordEvent(event);
+ clearOccured = true;
+ }
+ owner.basicInvalidatePart2(oldRe, event,
+ clearOccured /* conflict with clear */, invokeCallbacks);
+ if (!clearOccured) {
+ if (isCreate) {
+ lruEntryCreate(oldRe);
+ } else {
+ lruEntryUpdate(oldRe);
+ }
+ }
+ didInvalidate = true;
+ invalidatedRe = oldRe;
+ }
+ }
+ } // synchronized oldRe
+ } // while oldRe exists
+
+ if (!opCompleted) {
+ if (forceNewEntry && event.isFromServer()) {
+ // don't invoke listeners - we didn't force new entries for
+ // CCU invalidations before 7.0, and listeners don't care
+ event.inhibitCacheListenerNotification(true);
+ }
+ event.setRegionEntry(newRe);
+ owner.cacheWriteBeforeInvalidate(event, invokeCallbacks, forceNewEntry);
+ if (!forceNewEntry && event.noVersionReceivedFromServer()) {
+ // server did not perform the invalidation, so don't leave an invalid
+ // entry here
+ return false;
+ }
+ try {
+ if (!owner.isInitialized() && owner.getDataPolicy().withReplication()) {
+ final int oldSize = owner.calculateRegionEntryValueSize(newRe);
+ invalidateEntry(event, newRe, oldSize);
+ }
+ else {
+ invalidateNewEntry(event, owner, newRe);
+ }
+ }
+ catch (RegionClearedException e) {
+ // TODO: deltaGII: do we even need RegionClearedException?
+ // generate versionTag for the event
+ owner.recordEvent(event);
+ clearOccured = true;
+ }
+ owner.basicInvalidatePart2(newRe, event, clearOccured /*conflict with clear*/, invokeCallbacks);
+ if (!clearOccured) {
+ lruEntryCreate(newRe);
+ incEntryCount(1);
+ }
+ opCompleted = true;
+ didInvalidate = true;
+ invalidatedRe = newRe;
+ // Don't leave an entry in the cache, if we
+ // just wanted to force the distribution and events
+ // for this invalidate
+ if (!forceNewEntry) {
+ removeEntry(event.getKey(), newRe, false);
+ }
+ } // !opCompleted
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ } finally {
+ if (!opCompleted) {
+ removeEntry(event.getKey(), newRe, false);
+ }
+ }
+ } // synchronized newRe
+ } // forceNewEntry
+ else { // !forceNewEntry
+ boolean retry = true;
+ // RegionEntry retryEntry = null;
+ // int retries = -1;
+
+ while (retry) {
+ retry = false;
+ boolean entryExisted = false;
+ RegionEntry re = getEntry(event.getKey());
+ RegionEntry tombstone = null;
+ boolean haveTombstone = false;
+ if (re != null && re.isTombstone()) {
+ tombstone = re;
+ haveTombstone = true;
+ re = null;
+ }
+ if (re == null) {
+ if (!owner.isInitialized()) {
+ // when GII message arrived or processed later than invalidate
+ // message, the entry should be created as placeholder
+ RegionEntry newRe = haveTombstone? tombstone : getEntryFactory().createEntry(owner, event.getKey(),
+ Token.INVALID);
+ synchronized (newRe) {
+ if (haveTombstone && !tombstone.isTombstone()) {
+ // state of the tombstone has changed so we need to retry
+ retry = true;
+ //retryEntry = tombstone; // leave this in place for debugging
+ continue;
+ }
+ re = putEntryIfAbsent(event.getKey(), newRe);
+ if (re == tombstone) {
+ re = null; // pretend we don't have an entry
+ }
+ }
+ } else if (owner.getServerProxy() != null) {
+ Object sync = haveTombstone? tombstone : new Object();
+ synchronized(sync) {
+ if (haveTombstone && !tombstone.isTombstone()) {
+ // bug 45295: state of the tombstone has changed so we need to retry
+ retry = true;
+ //retryEntry = tombstone; // leave this in place for debugging
+ continue;
+ }
+
+ // bug #43287 - send event to server even if it's not in the client (LRU may have evicted it)
+ owner.cacheWriteBeforeInvalidate(event, true, false);
+ if (owner.concurrencyChecksEnabled) {
+ if (event.getVersionTag() == null) {
+ // server did not perform the invalidation, so don't leave an invalid
+ // entry here
+ return false;
+ } else if (tombstone != null) {
+ processVersionTag(tombstone, event);
+ try {
+ if (!tombstone.isTombstone()) {
+ if (isDebugEnabled) {
+ logger.debug("tombstone is no longer a tombstone. {}:event={}", tombstone, event);
+ }
+ }
+ tombstone.setValue(owner, Token.TOMBSTONE);
+ } catch (RegionClearedException e) {
+ // that's okay - when writing a tombstone into a disk, the
+ // region has been cleared (including this tombstone)
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ }
+ // update the tombstone's version to prevent an older CCU/putAll from overwriting it
+ owner.rescheduleTombstone(tombstone, event.getVersionTag());
+ }
+ }
+ }
+ entryExisted = true;
+ }
+ }
+ if (re != null) {
+ // Gester: Race condition in GII
+ // when adding the placeholder for invalidate entry during GII,
+ // if the GII got processed earlier for this entry, then do
+ // normal invalidate operation
+ synchronized (re) {
+ if (!event.isOriginRemote() && event.getOperation().isExpiration()) {
+ // If this expiration started locally then only do it if the RE is not being used by a tx.
+ if (re.isInUseByTransaction()) {
+ return false;
+ }
+ }
+ if (re.isTombstone() || (!re.isRemoved() && !re.isDestroyed())) {
+ entryExisted = true;
+ if (re.isInvalid()) {
+ // was already invalid, do not invoke listeners or increment
+ // stat
+ if (isDebugEnabled) {
+ logger.debug("Invalidate: Entry already invalid: '{}'", event.getKey());
+ }
+ if (event.getVersionTag() != null && owner.getVersionVector() != null) {
+ owner.getVersionVector().recordVersion((InternalDistributedMember) event.getDistributedMember(), event.getVersionTag());
+ }
+ forceInvalidateEvent(event);
+ }
+ else { // previous value not invalid
+ event.setRegionEntry(re);
+ owner.cacheWriteBeforeInvalidate(event, invokeCallbacks, forceNewEntry);
+ if (owner.concurrencyChecksEnabled && event.noVersionReceivedFromServer()) {
+ // server did not perform the invalidation, so don't leave an invalid
+ // entry here
+ if (isDebugEnabled) {
+ logger.debug("returning early because server did not generate a version stamp for this event:{}", event);
+ }
+ return false;
+ }
+ // in case of overflow to disk we need the old value for cqs.
+ if(owner.getFilterProfile().getCqCount() > 0){
+ //use to be getValue and can cause dead lock rdubey.
+ if (re.isValueNull()) {
+ event.setOldValue(re.getValueOnDiskOrBuffer(owner));
+ } else {
+
+ @Retained @Released Object v = re.getValueInVM(owner);
+
+ try {
+ event.setOldValue(v); // OFFHEAP escapes to EntryEventImpl oldValue
+ } finally {
+ OffHeapHelper.release(v);
+ }
+ }
+ }
+ final boolean oldWasTombstone = re.isTombstone();
+ final int oldSize = _getOwner().calculateRegionEntryValueSize(re);
+ try {
+ invalidateEntry(event, re, oldSize);
+ }
+ catch (RegionClearedException rce) {
+ // generate versionTag for the event
+ EntryLogger.logInvalidate(event);
+ _getOwner().recordEvent(event);
+ clearOccured = true;
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ }
+ owner.basicInvalidatePart2(re, event,
+ clearOccured /* conflict with clear */, invokeCallbacks);
+ if (!clearOccured) {
+ if (oldWasTombstone) {
+ lruEntryCreate(re);
+ } else {
+ lruEntryUpdate(re);
+ }
+ }
+ didInvalidate = true;
+ invalidatedRe = re;
+ } // previous value not invalid
+ }
+ } // synchronized re
+ } // re != null
+ else {
+ // At this point, either it's not in GII mode, or the placeholder
+ // is in region, do nothing
+ }
+ if (!entryExisted) {
+ forceInvalidateEvent(event);
+ owner.checkEntryNotFound(event.getKey());
+ }
+ } // while(retry)
+ } // !forceNewEntry
+ } catch( DiskAccessException dae) {
+ invalidatedRe = null;
+ didInvalidate = false;
+ this._getOwner().handleDiskAccessException(dae);
+ throw dae;
+ } finally {
+ releaseCacheModificationLock(owner, event);
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ if (invalidatedRe != null) {
+ owner.basicInvalidatePart3(invalidatedRe, event, invokeCallbacks);
+ }
+ if (didInvalidate && !clearOccured) {
+ try {
+ lruUpdateCallback();
+ } catch( DiskAccessException dae) {
+ this._getOwner().handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+ else if (!didInvalidate){
+ resetThreadLocals();
+ }
+ }
+ return didInvalidate;
+ }
+
+ protected void invalidateNewEntry(EntryEventImpl event,
+ final LocalRegion owner, RegionEntry newRe) throws RegionClearedException {
+ processVersionTag(newRe, event);
+ event.putNewEntry(owner, newRe);
+ owner.recordEvent(event);
+ owner.updateSizeOnCreate(event.getKey(), event.getNewValueBucketSize());
+ }
+
+ protected void invalidateEntry(EntryEventImpl event, RegionEntry re,
+ int oldSize) throws RegionClearedException {
+ processVersionTag(re, event);
+ event.putExistingEntry(_getOwner(), re);
+ EntryLogger.logInvalidate(event);
+ _getOwner().recordEvent(event);
+ _getOwner().updateSizeOnPut(event.getKey(), oldSize, event.getNewValueBucketSize());
+ }
+
+
+ /* (non-Javadoc)
+ * @see com.gemstone.gemfire.internal.cache.RegionMap#updateEntryVersion(com.gemstone.gemfire.internal.cache.EntryEventImpl)
+ */
+ @Override
+ public void updateEntryVersion(EntryEventImpl event) throws EntryNotFoundException {
+
+ final LocalRegion owner = _getOwner();
+ if (owner == null) {
+ // "fix" for bug 32440
+ Assert.assertTrue(false, "The owner for RegionMap " + this
+ + " is null for event " + event);
+
+ }
+
+ DiskRegion dr = owner.getDiskRegion();
+ if (dr != null) {
+ dr.setClearCountReference();
+ }
+
+ lockForCacheModification(owner, event);
+
+ try {
+ RegionEntry re = getEntry(event.getKey());
+
+ boolean entryExisted = false;
+
+ if (re != null) {
+ // process version tag
+ synchronized (re) {
+
+ try {
+ if (re.isTombstone()
+ || (!re.isRemoved() && !re.isDestroyed())) {
+ entryExisted = true;
+ }
+ processVersionTag(re, event);
+ owner.generateAndSetVersionTag(event, re);
+ EntryLogger.logUpdateEntryVersion(event);
+ _getOwner().recordEvent(event);
+ } catch (ConcurrentCacheModificationException ccme) {
+ // Do nothing.
+ }
+
+ }
+ }
+ if (!entryExisted) {
+ owner.checkEntryNotFound(event.getKey());
+ }
+ } catch( DiskAccessException dae) {
+ this._getOwner().handleDiskAccessException(dae);
+ throw dae;
+ } finally {
+ releaseCacheModificationLock(owner, event);
+ if (dr != null) {
+ dr.removeClearCountReference();
+ }
+ }
+ }
+
+ public final void txApplyInvalidate(Object key, Object newValue, boolean didDestroy,
+ TransactionId txId, TXRmtEvent txEvent, boolean localOp,
+ EventID eventId, Object aCallbackArgument,List pendingCallbacks,FilterRoutingInfo filterRoutingInfo,ClientProxyMembershipID bridgeContext, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+// boolean didInvalidate = false;
+ final LocalRegion owner = _getOwner();
+ owner.checkBeforeEntrySync(txEvent);
+
+ EntryEventImpl cbEvent = null;
+ boolean forceNewEntry = !owner.isInitialized() && owner.isAllEvents();
+
+ final boolean hasRemoteOrigin = !((TXId)txId).getMemberId().equals(owner.getMyId());
+ DiskRegion dr = owner.getDiskRegion();
+ // Fix for Bug #44431. We do NOT want to update the region and wait
+ // later for index INIT as region.clear() can cause inconsistency if
+ // happened in parallel as it also does index INIT.
+ IndexManager oqlIndexManager = owner.getIndexManager() ;
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ lockForTXCacheModification(owner, versionTag);
+ try {
+ if (forceNewEntry) {
+ boolean opCompleted = false;
+ RegionEntry newRe = getEntryFactory().createEntry(owner, key,
+ Token.REMOVED_PHASE1);
+ synchronized (newRe) {
+ try {
+ RegionEntry oldRe = putEntryIfAbsent(key, newRe);
+ while (!opCompleted && oldRe != null) {
+ synchronized (oldRe) {
+ if (oldRe.isRemovedPhase2()) {
+ oldRe = putEntryIfAbsent(key, newRe);
+ if (oldRe != null) {
+ owner.getCachePerfStats().incRetries();
+ }
+ }
+ else {
+ opCompleted = true;
+ final boolean oldWasTombstone = oldRe.isTombstone();
+ final int oldSize = owner.calculateRegionEntryValueSize(oldRe);
+ Object oldValue = oldRe.getValueInVM(owner); // OFFHEAP eei
+ try {
+ // Create an entry event only if the calling context is
+ // a receipt of a TXCommitMessage AND there are callbacks
+ // installed
+ // for this region
+ boolean invokeCallbacks = shouldCreateCBEvent(owner, true, owner.isInitialized());
+ boolean cbEventInPending = false;
+ cbEvent = createCBEvent(owner,
+ localOp ? Operation.LOCAL_INVALIDATE : Operation.INVALIDATE,
+ key, newValue, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ cbEvent.setRegionEntry(oldRe);
+ cbEvent.setOldValue(oldValue);
+ if (logger.isDebugEnabled()) {
+ logger.debug("txApplyInvalidate cbEvent={}", cbEvent);
+ }
+
+ txRemoveOldIndexEntry(Operation.INVALIDATE, oldRe);
+ if (didDestroy) {
+ oldRe.txDidDestroy(owner.cacheTimeMillis());
+ }
+ if (txEvent != null) {
+ txEvent.addInvalidate(owner, oldRe, oldRe.getKey(),
+ newValue,aCallbackArgument);
+ }
+ oldRe.setValueResultOfSearch(false);
+ processAndGenerateTXVersionTag(owner, cbEvent, oldRe, txEntryState);
+ boolean clearOccured = false;
+ try {
+ oldRe.setValue(owner, oldRe.prepareValueForCache(owner, newValue, true));
+ EntryLogger.logTXInvalidate(_getOwnerObject(), key);
+ owner.updateSizeOnPut(key, oldSize, 0);
+ if (oldWasTombstone) {
+ owner.unscheduleTombstone(oldRe);
+ }
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ owner.txApplyInvalidatePart2(oldRe, oldRe.getKey(),
+ didDestroy, true, clearOccured);
+ // didInvalidate = true;
+ if (invokeCallbacks) {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(
+ EnumListenerEvent.AFTER_INVALIDATE, cbEvent,
+ true/*callDispatchListenerEvent*/);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ if (!clearOccured) {
+ lruEntryUpdate(oldRe);
+ }
+ if (shouldPerformConcurrencyChecks(owner, cbEvent) && txEntryState != null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ } finally {
+ OffHeapHelper.release(oldValue);
+ }
+ }
+ }
+ }
+ if (!opCompleted) {
+ boolean invokeCallbacks = shouldCreateCBEvent( owner, true /* isInvalidate */, owner.isInitialized());
+ boolean cbEventInPending = false;
+ cbEvent = createCBEvent(owner,
+ localOp ? Operation.LOCAL_INVALIDATE : Operation.INVALIDATE,
+ key, newValue, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ cbEvent.setRegionEntry(newRe);
+ txRemoveOldIndexEntry(Operation.INVALIDATE, newRe);
+ newRe.setValueResultOfSearch(false);
+ boolean clearOccured = false;
+ try {
+ processAndGenerateTXVersionTag(owner, cbEvent, newRe, txEntryState);
+ newRe.setValue(owner, newRe.prepareValueForCache(owner, newValue, true));
+ EntryLogger.logTXInvalidate(_getOwnerObject(), key);
+ owner.updateSizeOnCreate(newRe.getKey(), 0);//we are putting in a new invalidated entry
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ owner.txApplyInvalidatePart2(newRe, newRe.getKey(), didDestroy,
+ true, clearOccured);
+
+ if (invokeCallbacks) {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(
+ EnumListenerEvent.AFTER_INVALIDATE, cbEvent,
+ true/*callDispatchListenerEvent*/);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ opCompleted = true;
+ if (!clearOccured) {
+ lruEntryCreate(newRe);
+ incEntryCount(1);
+ }
+ if (shouldPerformConcurrencyChecks(owner, cbEvent) && txEntryState != null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ }
+ }
+ finally {
+ if (!opCompleted) {
+ removeEntry(key, newRe, false);
+ }
+ }
+ }
+ }
+ else { /* !forceNewEntry */
+ RegionEntry re = getEntry(key);
+ if (re != null) {
+ synchronized (re) {
+ {
+ final int oldSize = owner.calculateRegionEntryValueSize(re);
+ boolean wasTombstone = re.isTombstone();
+ Object oldValue = re.getValueInVM(owner); // OFFHEAP eei
+ // Create an entry event only if the calling context is
+ // a receipt of a TXCommitMessage AND there are callbacks
+ // installed
+ // for this region
+ boolean invokeCallbacks = shouldCreateCBEvent(owner, true, owner.isInitialized());
+ boolean cbEventInPending = false;
+ cbEvent = createCBEvent(owner,
+ localOp ? Operation.LOCAL_INVALIDATE : Operation.INVALIDATE,
+ key, newValue, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ cbEvent.setRegionEntry(re);
+ cbEvent.setOldValue(oldValue);
+ txRemoveOldIndexEntry(Operation.INVALIDATE, re);
+ if (didDestroy) {
+ re.txDidDestroy(owner.cacheTimeMillis());
+ }
+ if (txEvent != null) {
+ txEvent.addInvalidate(owner, re, re.getKey(), newValue,aCallbackArgument);
+ }
+ re.setValueResultOfSearch(false);
+ processAndGenerateTXVersionTag(owner, cbEvent, re, txEntryState);
+ boolean clearOccured = false;
+ try {
+ re.setValue(owner, re.prepareValueForCache(owner, newValue, true));
+ EntryLogger.logTXInvalidate(_getOwnerObject(), key);
+ if (wasTombstone) {
+ owner.unscheduleTombstone(re);
+ }
+ owner.updateSizeOnPut(key, oldSize, 0);
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ owner.txApplyInvalidatePart2(re, re.getKey(), didDestroy, true,
+ clearOccured);
+ // didInvalidate = true;
+ if (invokeCallbacks) {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(
+ EnumListenerEvent.AFTER_INVALIDATE, cbEvent,
+ true/*callDispatchListenerEvent*/);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ if (!clearOccured) {
+ lruEntryUpdate(re);
+ }
+ if (shouldPerformConcurrencyChecks(owner, cbEvent) && txEntryState != null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ }
+ }
+ } else { //re == null
+ // Fix bug#43594
+ // In cases where bucket region is re-created, it may so happen
+ // that the invalidate is already applied on the Initial image
+ // provider, thus causing region entry to be absent.
+ // Notify clients with client events.
+ boolean cbEventInPending = false;
+ cbEvent = createCBEvent(owner,
+ localOp ? Operation.LOCAL_INVALIDATE : Operation.INVALIDATE,
+ key, newValue, txId, txEvent, eventId, aCallbackArgument,
+ filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if (pendingCallbacks == null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_INVALIDATE,
+ cbEvent, false);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ }
+ }
+ }
+ }catch( DiskAccessException dae) {
+ owner.handleDiskAccessException(dae);
+ throw dae;
+ }
+ finally {
+ releaseTXCacheModificationLock(owner, versionTag);
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ }
+ }
+
+ /**
+ * This code needs to be evaluated. It was added quickly to help PR persistence
+ * not to consume as much memory.
+ */
+ public void evictValue(Object key) {
+ final LocalRegion owner = _getOwner();
+ RegionEntry re = getEntry(key);
+ if (re != null) {
+ synchronized (re) {
+ if (!re.isValueNull()) {
+ re.setValueToNull();
+ owner.getDiskRegion().incNumEntriesInVM(-1L);
+ owner.getDiskRegion().incNumOverflowOnDisk(1L);
+ if(owner instanceof BucketRegion)
+ {
+ ((BucketRegion)owner).incNumEntriesInVM(-1L);
+ ((BucketRegion)owner).incNumOverflowOnDisk(1L);
+ }
+ }
+ }
+ }
+ }
+
+ private RegionEntry getOrCreateRegionEntry(Object ownerRegion,
+ EntryEventImpl event, Object value,
+ MapCallbackAdapter valueCreator,
+ boolean onlyExisting, boolean returnTombstone) {
+ Object key = event.getKey();
+ RegionEntry retVal = null;
+ if (event.isFetchFromHDFS()) {
+ retVal = getEntry(event);
+ } else {
+ retVal = getEntryInVM(key);
+ }
+ if (onlyExisting) {
+ if (!returnTombstone && (retVal != null && retVal.isTombstone())) {
+ return null;
+ }
+ return retVal;
+ }
+ if (retVal != null) {
+ return retVal;
+ }
+ if (valueCreator != null) {
+ value = valueCreator.newValue(key, ownerRegion, value, null);
+ }
+ retVal = getEntryFactory().createEntry((RegionEntryContext) ownerRegion, key, value);
+ RegionEntry oldRe = putEntryIfAbsent(key, retVal);
+ if (oldRe != null) {
+ if (retVal instanceof OffHeapRegionEntry) {
+ ((OffHeapRegionEntry) retVal).release();
+ }
+ return oldRe;
+ }
+ return retVal;
+ }
+
+ protected static final MapCallbackAdapter
+ listOfDeltasCreator = new MapCallbackAdapter() {
+ @Override
+ public Object newValue(Object key, Object context, Object createParams,
+ final MapResult result) {
+ return new ListOfDeltas(4);
+ }
+ };
+
+ /**
+ * Neeraj: The below if block is to handle the special
+ * scenario witnessed in Sqlfabric for now. (Though its
+ * a general scenario). The scenario is that the updates start coming
+ * before the base value reaches through GII. In that scenario the updates
+ * essentially the deltas are added to a list and kept as oldValue in the
+ * map and this method returns. When through GII the actual base value arrives
+ * these updates or deltas are applied on it and the new value thus got is put
+ * in the map.
+ * @param event
+ * @param ifOld
+ * @return true if delta was enqued
+ */
+ private boolean enqueDelta(EntryEventImpl event, boolean ifOld) {
+ final IndexUpdater indexManager = getIndexUpdater();
+ LocalRegion owner = _getOwner();
+ if (indexManager != null && !owner.isInitialized() && event.hasDelta()) {
+ boolean isOldValueDelta = true;
+ try {
+ if (ifOld) {
+ final Delta delta = event.getDeltaNewValue();
+ RegionEntry re = getOrCreateRegionEntry(owner, event, null,
+ listOfDeltasCreator, false, false);
+ assert re != null;
+ synchronized (re) {
+ @Retained @Released Object oVal = re.getValueOffHeapOrDiskWithoutFaultIn(owner);
+ if (oVal != null) {
+ try {
+ if (oVal instanceof ListOfDeltas) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("basicPut: adding delta to list of deltas: {}", delta);
+ }
+ ((ListOfDeltas)oVal).merge(delta);
+ @Retained Object newVal = ((AbstractRegionEntry)re).prepareValueForCache(owner, oVal, true);
+ re.setValue(owner, newVal); // TODO:KIRK:48068 prevent orphan
+ }
+ else {
+ isOldValueDelta = false;
+ }
+ }finally {
+ OffHeapHelper.release(oVal);
+ }
+ }
+ else {
+ if (logger.isDebugEnabled()) {
+ logger.debug("basicPut: new list of deltas with delta: {}", delta);
+ }
+ @Retained Object newVal = new ListOfDeltas(delta);
+ // TODO no need to call AbstractRegionMap.prepareValueForCache here?
+ newVal = ((AbstractRegionEntry)re).prepareValueForCache(owner, newVal, true);
+ re.setValue(owner, newVal); // TODO:KIRK:48068 prevent orphan
+ }
+ }
+ }
+ } catch (RegionClearedException ex) {
+ // Neeraj: We can just ignore this exception because we are returning after this block
+ }
+ if (isOldValueDelta) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /*
+ * returns null if the operation fails
+ */
+ public RegionEntry basicPut(EntryEventImpl event,
+ final long lastModified,
+ final boolean ifNew,
+ final boolean ifOld,
+ Object expectedOldValue, // only non-null if ifOld
+ boolean requireOldValue,
+ final boolean overwriteDestroyed)
+ throws CacheWriterException,
+ TimeoutException {
+ final LocalRegion owner = _getOwner();
+ boolean clearOccured = false;
+ if (owner == null) {
+ // "fix" for bug 32440
+ Assert.assertTrue(false, "The owner for RegionMap " + this
+ + " is null for event " + event);
+ }
+ if (logger.isTraceEnabled(LogMarker.LRU_TOMBSTONE_COUNT) && !(owner instanceof HARegion)) {
+ logger.trace(LogMarker.LRU_TOMBSTONE_COUNT,
+ "ARM.basicPut called for {} expectedOldValue={} requireOldValue={} ifNew={} ifOld={} initialized={} overwriteDestroyed={}",
+ event, expectedOldValue, requireOldValue, ifNew, ifOld, owner.isInitialized(), overwriteDestroyed);
+ }
+
+ RegionEntry result = null;
+ long lastModifiedTime = 0;
+ // copy into local var to prevent race condition with setter
+ final CacheWriter cacheWriter = owner.basicGetWriter();
+ final boolean cacheWrite = !event.isOriginRemote() && !event.isNetSearch() && event.isGenerateCallbacks()
+ && (cacheWriter != null
+ || owner.hasServerProxy()
+ || owner.scope.isDistributed());
+ /*
+ * For performance reason, we try to minimize object creation and do as much
+ * work as we can outside of synchronization, especially getting
+ * distribution advice.
+ */
+ final Set netWriteRecipients;
+ if (cacheWrite) {
+ if (cacheWriter == null && owner.scope.isDistributed()) {
+ netWriteRecipients = ((DistributedRegion)owner)
+ .getCacheDistributionAdvisor().adviseNetWrite();
+ }
+ else {
+ netWriteRecipients = null;
+ }
+ }
+ else {
+ netWriteRecipients = null;
+ }
+
+ // mbid: this has been added to maintain consistency between the disk region
+ // and the region map after clear() has been called. This will set the
+ // reference of the diskSegmentRegion as a ThreadLocal so that if the diskRegionSegment
+ // is later changed by another thread, we can do the necessary.
+ boolean uninitialized = !owner.isInitialized();
+ // SqlFabric Changes - BEGIN
+ if (enqueDelta(event, ifOld)) {
+ return null;
+ }
+
+ final IndexUpdater indexManager = getIndexUpdater();
+
+ boolean sqlfIndexLocked = false;
+ // SqlFabric Changes - END
+
+ boolean retrieveOldValueForDelta = event.getDeltaBytes() != null
+ && event.getRawNewValue() == null;
+ lockForCacheModification(owner, event);
+ IndexManager oqlIndexManager = null;
+ try {
+ // take read lock for SQLF index initializations if required; the index
+ // GII lock is for any updates that may come in while index is being
+ // loaded during replay see bug #41377; this will go away once we allow
+ // for indexes to be loaded completely in parallel (#40899); need to
+ // take this lock before the RegionEntry lock else a deadlock can happen
+ // between this thread and index loading thread that will first take the
+ // corresponding write lock on the IndexUpdater
+ if (indexManager != null) {
+ sqlfIndexLocked = indexManager.lockForIndexGII();
+ }
+ // Fix for Bug #44431. We do NOT want to update the region and wait
+ // later for index INIT as region.clear() can cause inconsistency if
+ // happened in parallel as it also does index INIT.
+ oqlIndexManager = owner.getIndexManager() ;
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+
+ // fix for bug #42169, replace must go to server if entry not on client
+ boolean replaceOnClient = event.getOperation() == Operation.REPLACE
+ && owner.getServerProxy() != null;
+ // Rather than having two different blocks for synchronizing oldRe
+ // and newRe, have only one block and synchronize re
+ RegionEntry re = null;
+ boolean eventRecorded = false;
+ boolean onlyExisting = ifOld && !replaceOnClient;
+ re = getOrCreateRegionEntry(owner, event,
+ Token.REMOVED_PHASE1, null, onlyExisting, false);
+ if (re == null) {
+ throwExceptionForSqlFire(event);
+ return null;
+ }
+ while (true) {
+ synchronized (re) {
+ // if the re goes into removed2 state, it will be removed
+ // from the map. otherwise we can append an event to it
+ // and change its state
+ if (re.isRemovedPhase2()) {
+ re = getOrCreateRegionEntry(owner, event,
+ Token.REMOVED_PHASE1, null, onlyExisting, false);
+ _getOwner().getCachePerfStats().incRetries();
+ if (re == null) {
+ // this will happen when onlyExisting is true
+ throwExceptionForSqlFire(event);
+ return null;
+ }
+ continue;
+ } else {
+ @Released Object oldValueForDelta = null;
+ if (retrieveOldValueForDelta) {
+ // defer the lruUpdateCallback to prevent a deadlock (see bug 51121).
+ final boolean disabled = disableLruUpdateCallback();
+ try {
+ // Old value is faulted in from disk if not found in memory.
+ oldValueForDelta = re.getValue(owner); // OFFHEAP: if we are synced on oldRe no issue since we can use ARE's ref
+ } finally {
+ if (disabled) {
+ enableLruUpdateCallback();
+ }
+ }
+ }
+
+ try {
+
+ event.setRegionEntry(re);
+ // set old value in event
+ setOldValueInEvent(event, re, cacheWrite, requireOldValue);
+ if (!continueUpdate(re, event, ifOld, replaceOnClient)) {
+ return null;
+ }
+ // overwrite destroyed?
+ if (!continueOverwriteDestroyed(re, event, overwriteDestroyed, ifNew)) {
+ return null;
+ }
+ // check expectedOldValue
+ if (!satisfiesExpectedOldValue(event, re, expectedOldValue, replaceOnClient)) {
+ return null;
+ }
+ // invoke cacheWriter
+ invokeCacheWriter(re, event, cacheWrite, cacheWriter,
+ netWriteRecipients, requireOldValue, expectedOldValue, replaceOnClient);
+
+ // notify index of an update
+ notifyIndex(re, true);
+ try {
+ try {
+ if ((cacheWrite && event.getOperation().isUpdate()) // if there is a cacheWriter, type of event has already been set
+ || !re.isRemoved()
+ || replaceOnClient) {
+ // update
+ updateEntry(event, requireOldValue, oldValueForDelta, re);
+ } else {
+ // create
+ createEntry(event, owner, re);
+ }
+ owner.recordEvent(event);
+ eventRecorded = true;
+ } catch (RegionClearedException rce) {
+ clearOccured = true;
+ owner.recordEvent(event);
+ } catch (ConcurrentCacheModificationException ccme) {
+ VersionTag tag = event.getVersionTag();
+ if (tag != null && tag.isTimeStampUpdated()) {
+ // Notify gateways of new time-stamp.
+ owner.notifyTimestampsToGateways(event);
+ }
+ throw ccme;
+ }
+ if (uninitialized) {
+ event.inhibitCacheListenerNotification(true);
+ }
+ updateLru(clearOccured, re, event);
+
+ lastModifiedTime = owner.basicPutPart2(event, re,
+ !uninitialized, lastModifiedTime, clearOccured);
+ } finally {
+ notifyIndex(re, false);
+ }
+ result = re;
+ break;
+ } finally {
+ OffHeapHelper.release(oldValueForDelta);
+ if (re != null && !onlyExisting && !isOpComplete(re, event)) {
+ owner.cleanUpOnIncompleteOp(event, re, eventRecorded,
+ false/* updateStats */, replaceOnClient);
+ }
+ else if (re != null && owner.isUsedForPartitionedRegionBucket()) {
+ BucketRegion br = (BucketRegion)owner;
+ CachePerfStats stats = br.getPartitionedRegion().getCachePerfStats();
+ long startTime= stats.startCustomEviction();
+ CustomEvictionAttributes csAttr = br.getCustomEvictionAttributes();
+ // No need to update indexes if entry was faulted in but operation did not succeed.
+ if (csAttr != null && (csAttr.isEvictIncoming() || re.isMarkedForEviction())) {
+
+ if (csAttr.getCriteria().doEvict(event)) {
+ stats.incEvictionsInProgress();
+ // set the flag on event saying the entry should be evicted
+ // and not indexed
+ EntryEventImpl destroyEvent = EntryEventImpl.create (owner, Operation.DESTROY, event.getKey(),
+ null/* newValue */, null, false, owner.getMyId());
+ try {
+
+ destroyEvent.setOldValueFromRegion();
+ destroyEvent.setCustomEviction(true);
+ destroyEvent.setPossibleDuplicate(event.isPossibleDuplicate());
+ if(logger.isDebugEnabled()) {
+ logger.debug("Evicting the entry " + destroyEvent);
+ }
+ if(result != null) {
+ removeEntry(event.getKey(),re, true, destroyEvent,owner, indexUpdater);
+ }
+ else{
+ removeEntry(event.getKey(),re, true, destroyEvent,owner, null);
+ }
+ //mark the region entry for this event as evicted
+ event.setEvicted();
+ stats.incEvictions();
+ if(logger.isDebugEnabled()) {
+ logger.debug("Evicted the entry " + destroyEvent);
+ }
+ //removeEntry(event.getKey(), re);
+ } finally {
+ destroyEvent.release();
+ stats.decEvictionsInProgress();
+ }
+ } else {
+ re.clearMarkedForEviction();
+ }
+ }
+ stats.endCustomEviction(startTime);
+ }
+ } // try
+ }
+ } // sync re
+ }// end while
+ } catch (DiskAccessException dae) {
+ //Asif:Feel that it is safe to destroy the region here as there appears
+ // to be no chance of deadlock during region destruction
+ result = null;
+ this._getOwner().handleDiskAccessException(dae);
+ throw dae;
+ } finally {
+ releaseCacheModificationLock(owner, event);
+ if (sqlfIndexLocked) {
+ indexManager.unlockForIndexGII();
+ }
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ if (result != null) {
+ try {
+ // Note we do distribution after releasing all sync to avoid deadlock
+ final boolean invokeListeners = event.basicGetNewValue() != Token.TOMBSTONE;
+ owner.basicPutPart3(event, result, !uninitialized,
+ lastModifiedTime, invokeListeners, ifNew, ifOld, expectedOldValue, requireOldValue);
+ } catch (EntryExistsException eee) {
+ // SQLFabric changes BEGIN
+ // ignore EntryExistsException in distribution from a non-empty
+ // region since actual check will be done in this put itself
+ // and it can happen in distribution if put comes in from
+ // GII as well as distribution channel
+ if (indexManager != null) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("basicPut: ignoring EntryExistsException in distribution {}", eee);
+ }
+ }
+ else {
+ // can this happen for non-SQLFabric case?
+ throw eee;
+ }
+ // SQLFabric changes END
+ } finally {
+ // bug 32589, post update may throw an exception if exception occurs
+ // for any recipients
+ if (!clearOccured) {
+ try {
+ lruUpdateCallback();
+ } catch( DiskAccessException dae) {
+ //Asif:Feel that it is safe to destroy the region here as there appears
+ // to be no chance of deadlock during region destruction
+ result = null;
+ this._getOwner().handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+ } // finally
+ } else {
+ resetThreadLocals();
+ }
+ } // finally
+
+ return result;
+ }
+
+ /**
+ * If the value in the VM is still REMOVED_PHASE1 Token, then the operation
+ * was not completed (due to cacheWriter exception, concurrentMap operation) etc.
+ */
+ private boolean isOpComplete(RegionEntry re, EntryEventImpl event) {
+ if (re.getValueAsToken() == Token.REMOVED_PHASE1) {
+ return false;
+ }
+ return true;
+ }
+
+ private boolean satisfiesExpectedOldValue(EntryEventImpl event,
+ RegionEntry re, Object expectedOldValue, boolean replaceOnClient) {
+ // replace is propagated to server, so no need to check
+ // satisfiesOldValue on client
+ if (expectedOldValue != null && !replaceOnClient) {
+ ReferenceCountHelper.skipRefCountTracking();
+
+ @Retained @Released Object v = re._getValueRetain(event.getLocalRegion(), true);
+
+ ReferenceCountHelper.unskipRefCountTracking();
+ try {
+ if (!AbstractRegionEntry.checkExpectedOldValue(expectedOldValue, v, event.getLocalRegion())) {
+ return false;
+ }
+ } finally {
+ OffHeapHelper.releaseWithNoTracking(v);
+ }
+ }
+ return true;
+ }
+
+ // Asif: If the new value is an instance of SerializableDelta, then
+ // the old value requirement is a must & it needs to be faulted in
+ // if overflown to disk without affecting LRU? This is needed for
+ // Sql Fabric.
+ // [sumedh] store both the value in VM and the value in VM or disk;
+ // the former is used for updating the VM size calculations, while
+ // the latter is used in other places like passing to
+ // SqlfIndexManager or setting the old value in the event; this is
+ // required since using the latter for updating the size
+ // calculations will be incorrect in case the value was read from
+ // disk but not brought into the VM like what getValueInVMOrDisk
+ // method does when value is not found in VM
+ // PRECONDITION: caller must be synced on re
+ private void setOldValueInEvent(EntryEventImpl event, RegionEntry re, boolean cacheWrite, boolean requireOldValue) {
+ boolean needToSetOldValue = getIndexUpdater() != null || cacheWrite || requireOldValue || event.getOperation().guaranteesOldValue();
+ if (needToSetOldValue) {
+ if (event.hasDelta() || event.getOperation().guaranteesOldValue()
+ || GemFireCacheImpl.sqlfSystem()) {
+ // In these cases we want to even get the old value from disk if it is not in memory
+ ReferenceCountHelper.skipRefCountTracking();
+ @Released Object oldValueInVMOrDisk = re.getValueOffHeapOrDiskWithoutFaultIn(event.getLocalRegion());
+ ReferenceCountHelper.unskipRefCountTracking();
+ try {
+ event.setOldValue(oldValueInVMOrDisk, requireOldValue
+ || GemFireCacheImpl.sqlfSystem());
+ } finally {
+ OffHeapHelper.releaseWithNoTracking(oldValueInVMOrDisk);
+ }
+ } else {
+ // In these cases only need the old value if it is in memory
+ ReferenceCountHelper.skipRefCountTracking();
+
+ @Retained @Released Object oldValueInVM = re._getValueRetain(event.getLocalRegion(), true); // OFFHEAP: re synced so can use its ref.
+
+ ReferenceCountHelper.unskipRefCountTracking();
+ try {
+ event.setOldValue(oldValueInVM,
+ requireOldValue || GemFireCacheImpl.sqlfSystem());
+ } finally {
+ OffHeapHelper.releaseWithNoTracking(oldValueInVM);
+ }
+ }
+ } else {
+ // if the old value is in memory then if it is a GatewaySenderEventImpl then
+ // we want to set the old value.
+ @Unretained Object ov = re._getValue(); // OFFHEAP _getValue is ok since re is synced and we only use it if its a GatewaySenderEventImpl.
+ // Since GatewaySenderEventImpl is never stored in an off-heap region nor a compressed region we don't need to worry about ov being compressed.
+ if (ov instanceof GatewaySenderEventImpl) {
+ event.setOldValue(ov, true);
+ }
+ }
+ }
+
+ /**
+ * Asif: If the system is sqlfabric and the event has delta, then re == null
+ * implies update on non existent row . Throwing ENFE in that case
+ * As returning a boolean etc has other complications in terms of PR reattempt etc
+ */
+ private void throwExceptionForSqlFire(EntryEventImpl event) {
+ if (event.hasDelta() && _getOwner().getGemFireCache().isSqlfSystem()) {
+ throw new EntryNotFoundException(
+ "SqlFabric::No row found for update");
+ }
+ }
+
+ protected void createEntry(EntryEventImpl event, final LocalRegion owner,
+ RegionEntry re) throws RegionClearedException {
+ final boolean wasTombstone = re.isTombstone();
+ processVersionTag(re, event);
+ event.putNewEntry(owner, re);
+ updateSize(event, 0, false, wasTombstone);
+ if (!event.getLocalRegion().isInitialized()) {
+ owner.getImageState().removeDestroyedEntry(event.getKey());
+ }
+ }
+
+ protected void updateEntry(EntryEventImpl event, boolean requireOldValue,
+ Object oldValueForDelta, RegionEntry re) throws RegionClearedException {
+ final int oldSize = event.getLocalRegion().calculateRegionEntryValueSize(re);
+ final boolean wasTombstone = re.isTombstone();
+ processVersionTag(re, event);
+ event.putExistingEntry(event.getLocalRegion(), re, requireOldValue,
+ oldValueForDelta);
+ EntryLogger.logPut(event);
+ updateSize(event, oldSize, true/* isUpdate */, wasTombstone);
+ }
+
+ private void updateLru(boolean clearOccured, RegionEntry re, EntryEventImpl event) {
+ if (!clearOccured) {
+ if (event.getOperation().isCreate()) {
+ lruEntryCreate(re);
+ } else {
+ lruEntryUpdate(re);
+ }
+ }
+ }
+
+ private void updateSize(EntryEventImpl event, int oldSize, boolean isUpdate, boolean wasTombstone) {
+ if (isUpdate && !wasTombstone) {
+ _getOwner().updateSizeOnPut(event.getKey(), oldSize, event.getNewValueBucketSize());
+ } else {
+ _getOwner().updateSizeOnCreate(event.getKey(), event.getNewValueBucketSize());
+ if (!wasTombstone) {
+ incEntryCount(1);
+ }
+ }
+ }
+
+ private void notifyIndex(RegionEntry re, boolean isUpdating) {
+ if (_getOwner().indexMaintenanceSynchronous) {
+ re.setUpdateInProgress(isUpdating);
+ }
+ }
+
+ private void invokeCacheWriter(RegionEntry re, EntryEventImpl event,
+ boolean cacheWrite, CacheWriter cacheWriter, Set netWriteRecipients,
+ boolean requireOldValue, Object expectedOldValue, boolean replaceOnClient) {
+ // invoke listeners only if region is initialized
+ if (_getOwner().isInitialized() && cacheWrite) {
+ // event.setOldValue already called in setOldValueInEvent
+
+ // bug #42638 for replaceOnClient, do not make the event create
+ // or update since replace must propagate to server
+ if (!replaceOnClient) {
+ if (re.isDestroyedOrRemoved()) {
+ event.makeCreate();
+ } else {
+ event.makeUpdate();
+ }
+ }
+ _getOwner().cacheWriteBeforePut(event, netWriteRecipients, cacheWriter,
+ requireOldValue, expectedOldValue);
+ }
+ if (!_getOwner().isInitialized() && !cacheWrite) {
+ // block setting of old value in putNewValueNoSync, don't
+ // need it
+ event.oldValueNotAvailable();
+ }
+ }
+
+ private boolean continueOverwriteDestroyed(RegionEntry re,
+ EntryEventImpl event, boolean overwriteDestroyed, boolean ifNew) {
+ Token oldValueInVM = re.getValueAsToken();
+ // if region is under GII, check if token is destroyed
+ if (!overwriteDestroyed) {
+ if (!_getOwner().isInitialized() && (oldValueInVM == Token.DESTROYED || oldValueInVM == Token.TOMBSTONE)) {
+ event.setOldValueDestroyedToken();
+ return false;
+ }
+ }
+ if (ifNew && !Token.isRemoved(oldValueInVM)) {
+ return false;
+ }
+ return true;
+ }
+
+ private boolean continueUpdate(RegionEntry re, EntryEventImpl event,
+ boolean ifOld, boolean replaceOnClient) {
+ if (ifOld) {
+ // only update, so just do tombstone maintainence and exit
+ if (re.isTombstone() && event.getVersionTag() != null) {
+ // refresh the tombstone so it doesn't time out too soon
+ processVersionTag(re, event);
+ try {
+ re.setValue(_getOwner(), Token.TOMBSTONE);
+ } catch (RegionClearedException e) {
+ // that's okay - when writing a tombstone into a disk, the
+ // region has been cleared (including this tombstone)
+ }
+ _getOwner().rescheduleTombstone(re, re.getVersionStamp().asVersionTag());
+ return false;
+ }
+ if (re.isRemoved() && !replaceOnClient) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ protected boolean destroyEntry(RegionEntry re, EntryEventImpl event,
+ boolean inTokenMode, boolean cacheWrite, @Released Object expectedOldValue,
+ boolean forceDestroy, boolean removeRecoveredEntry)
+ throws CacheWriterException, TimeoutException, EntryNotFoundException,
+ RegionClearedException {
+ processVersionTag(re, event);
+ final int oldSize = _getOwner().calculateRegionEntryValueSize(re);
+ boolean retVal = re.destroy(event.getLocalRegion(), event, inTokenMode,
+ cacheWrite, expectedOldValue, forceDestroy, removeRecoveredEntry);
+ if (retVal) {
+ EntryLogger.logDestroy(event);
+ _getOwner().updateSizeOnRemove(event.getKey(), oldSize);
+ }
+ return retVal;
+ }
+
+ public void txApplyPut(Operation p_putOp, Object key, Object nv,
+ boolean didDestroy, TransactionId txId, TXRmtEvent txEvent,
+ EventID eventId, Object aCallbackArgument,List pendingCallbacks,FilterRoutingInfo filterRoutingInfo,ClientProxyMembershipID bridgeContext, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+ final LocalRegion owner = _getOwner();
+ if (owner == null) {
+ // "fix" for bug 32440
+ Assert.assertTrue(false, "The owner for RegionMap " + this
+ + " is null");
+ }
+
+ Operation putOp = p_putOp;
+
+ owner.checkBeforeEntrySync(txEvent);
+ Object newValue = nv;
+
+ final boolean hasRemoteOrigin = !((TXId)txId).getMemberId().equals(owner.getMyId());
+ final boolean isTXHost = txEntryState != null;
+ final boolean isClientTXOriginator = owner.cache.isClient() && !hasRemoteOrigin;
+ final boolean isRegionReady = owner.isInitialized();
+ EntryEventImpl cbEvent = null;
+ EntryEventImpl sqlfEvent = null;
+ boolean invokeCallbacks = shouldCreateCBEvent(owner, false /*isInvalidate*/, isRegionReady);
+ boolean cbEventInPending = false;
+ cbEvent = createCBEvent(owner, putOp, key, newValue, txId,
+ txEvent, eventId, aCallbackArgument,filterRoutingInfo,bridgeContext, txEntryState, versionTag, tailKey);
+ try {
+ if (logger.isDebugEnabled()) {
+ logger.debug("txApplyPut cbEvent={}", cbEvent);
+ }
+
+
+ if (owner.isUsedForPartitionedRegionBucket()) {
+ newValue = EntryEventImpl.getCachedDeserializable(nv, cbEvent);
+ txHandleWANEvent(owner, cbEvent, txEntryState);
+ }
+
+ if (/*owner.isUsedForPartitionedRegionBucket() && */
+ (getIndexUpdater() != null ||
+ (newValue instanceof com.gemstone.gemfire.internal.cache.delta.Delta))) {
+ sqlfEvent = createCBEvent(owner, putOp, key, newValue, txId,
+ txEvent, eventId, aCallbackArgument,filterRoutingInfo,bridgeContext, txEntryState, versionTag, tailKey);
+ }
+ boolean opCompleted = false;
+ // Fix for Bug #44431. We do NOT want to update the region and wait
+ // later for index INIT as region.clear() can cause inconsistency if
+ // happened in parallel as it also does index INIT.
+ IndexManager oqlIndexManager = owner.getIndexManager() ;
+ if (oqlIndexManager != null) {
+ oqlIndexManager.waitForIndexInit();
+ }
+ lockForTXCacheModification(owner, versionTag);
+ try {
+ if (hasRemoteOrigin && !isTXHost && !isClientTXOriginator) {
+ // If we are not a mirror then only apply the update to existing
+ // entries
+ //
+ // If we are a mirror then then only apply the update to
+ // existing entries when the operation is an update and we
+ // are initialized.
+ // Otherwise use the standard create/update logic
+ if (!owner.isAllEvents() || (!putOp.isCreate() && isRegionReady)) {
+ // At this point we should only apply the update if the entry exists
+ RegionEntry re = getEntry(key); // Fix for bug 32347.
+ if (re != null) {
+ synchronized (re) {
+ if (!re.isRemoved()) {
+ opCompleted = true;
+ putOp = putOp.getCorrespondingUpdateOp();
+ // Net writers are not called for received transaction data
+ final int oldSize = owner.calculateRegionEntryValueSize(re);
+ if (cbEvent != null) {
+ cbEvent.setRegionEntry(re);
+ cbEvent.setOldValue(re.getValueInVM(owner)); // OFFHEAP eei
+ }
+ if (sqlfEvent != null) {
+ sqlfEvent.setOldValue(re.getValueInVM(owner)); // OFFHEAP eei
+ }
+
+ boolean clearOccured = false;
+ // Set RegionEntry updateInProgress
+ if (owner.indexMaintenanceSynchronous) {
+ re.setUpdateInProgress(true);
+ }
+ try {
+ txRemoveOldIndexEntry(putOp, re);
+ if (didDestroy) {
+ re.txDidDestroy(owner.cacheTimeMillis());
+ }
+ if (txEvent != null) {
+ txEvent.addPut(putOp, owner, re, re.getKey(), newValue,aCallbackArgument);
+ }
+ re.setValueResultOfSearch(putOp.isNetSearch());
+ try {
+ // Rahul: applies the delta and sets the new value in
+ // region entry (required for sqlfabric delta).
+ processAndGenerateTXVersionTag(owner, cbEvent, re, txEntryState);
+ if (newValue instanceof com.gemstone.gemfire.internal.cache.delta.Delta
+ && sqlfEvent != null) {
+ //cbEvent.putExistingEntry(owner, re);
+ sqlfEvent.putExistingEntry(owner, re);
+ } else {
+ re.setValue(owner, re.prepareValueForCache(owner, newValue, cbEvent, !putOp.isCreate()));
+ }
+ if (putOp.isCreate()) {
+ owner.updateSizeOnCreate(key, owner.calculateRegionEntryValueSize(re));
+ } else if (putOp.isUpdate()) {
+ // Rahul : fix for 41694. Negative bucket size can also be
+ // an issue with normal GFE Delta and will have to be fixed
+ // in a similar manner and may be this fix the the one for
+ // other delta can be combined.
+ if (sqlfEvent != null) {
+ owner.updateSizeOnPut(key, oldSize, sqlfEvent.getNewValueBucketSize());
+ } else {
+ owner.updateSizeOnPut(key, oldSize, owner.calculateRegionEntryValueSize(re));
+ }
+ }
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ {
+ long lastMod = owner.cacheTimeMillis();
+ EntryLogger.logTXPut(_getOwnerObject(), key, nv);
+ re.updateStatsForPut(lastMod);
+ owner.txApplyPutPart2(re, re.getKey(), newValue, lastMod,
+ false, didDestroy, clearOccured);
+ }
+ } finally {
+ if (re != null && owner.indexMaintenanceSynchronous) {
+ re.setUpdateInProgress(false);
+ }
+ }
+ if (invokeCallbacks) {
+ cbEvent.makeUpdate();
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_UPDATE,
+ cbEvent, hasRemoteOrigin);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ if (!clearOccured) {
+ lruEntryUpdate(re);
+ }
+ }
+ }
+ if (didDestroy && !opCompleted) {
+ owner
+ .txApplyInvalidatePart2(re, re.getKey(), true, false, false /* clear*/);
+ }
+ }
+ if (owner.concurrencyChecksEnabled && txEntryState != null && cbEvent != null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ return;
+ }
+ }
+ RegionEntry newRe = getEntryFactory().createEntry(owner, key,
+ Token.REMOVED_PHASE1);
+ synchronized (newRe) {
+ try {
+ RegionEntry oldRe = putEntryIfAbsent(key, newRe);
+ while (!opCompleted && oldRe != null) {
+ synchronized (oldRe) {
+ if (oldRe.isRemovedPhase2()) {
+ oldRe = putEntryIfAbsent(key, newRe);
+ if (oldRe != null) {
+ owner.getCachePerfStats().incRetries();
+ }
+ }
+ else {
+ opCompleted = true;
+ if (!oldRe.isRemoved()) {
+ putOp = putOp.getCorrespondingUpdateOp();
+ }
+ // Net writers are not called for received transaction data
+ final int oldSize = owner.calculateRegionEntryValueSize(oldRe);
+ final boolean oldIsRemoved = oldRe.isDestroyedOrRemoved();
+ if (cbEvent != null) {
+ cbEvent.setRegionEntry(oldRe);
+ cbEvent.setOldValue(oldRe.getValueInVM(owner)); // OFFHEAP eei
+ }
+ if (sqlfEvent != null) {
+ sqlfEvent.setOldValue(oldRe.getValueInVM(owner)); // OFFHEAP eei
+ }
+ boolean clearOccured = false;
+ // Set RegionEntry updateInProgress
+ if (owner.indexMaintenanceSynchronous) {
+ oldRe.setUpdateInProgress(true);
+ }
+ try {
+ txRemoveOldIndexEntry(putOp, oldRe);
+ if (didDestroy) {
+ oldRe.txDidDestroy(owner.cacheTimeMillis());
+ }
+ if (txEvent != null) {
+ txEvent.addPut(putOp, owner, oldRe, oldRe.getKey(), newValue,aCallbackArgument);
+ }
+ oldRe.setValueResultOfSearch(putOp.isNetSearch());
+ try {
+ processAndGenerateTXVersionTag(owner, cbEvent, oldRe, txEntryState);
+ boolean wasTombstone = oldRe.isTombstone();
+ if (newValue instanceof com.gemstone.gemfire.internal.cache.delta.Delta
+ && sqlfEvent != null ) {
+ //cbEvent.putExistingEntry(owner, oldRe);
+ sqlfEvent.putExistingEntry(owner, oldRe);
+ } else {
+ oldRe.setValue(owner, oldRe.prepareValueForCache(owner, newValue, cbEvent, !putOp.isCreate()));
+ if (wasTombstone) {
+ owner.unscheduleTombstone(oldRe);
+ }
+ }
+ if (putOp.isCreate()) {
+ owner.updateSizeOnCreate(key, owner.calculateRegionEntryValueSize(oldRe));
+ } else if (putOp.isUpdate()) {
+ // Rahul : fix for 41694. Negative bucket size can also be
+ // an issue with normal GFE Delta and will have to be fixed
+ // in a similar manner and may be this fix the the one for
+ // other delta can be combined.
+ if (sqlfEvent != null) {
+ owner.updateSizeOnPut(key, oldSize, sqlfEvent.getNewValueBucketSize());
+ } else {
+ owner.updateSizeOnPut(key, oldSize, owner.calculateRegionEntryValueSize(oldRe));
+ }
+ }
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ {
+ long lastMod = System.currentTimeMillis();
+ EntryLogger.logTXPut(_getOwnerObject(), key, nv);
+ oldRe.updateStatsForPut(lastMod);
+ owner.txApplyPutPart2(oldRe, oldRe.getKey(), newValue,
+ lastMod, false, didDestroy, clearOccured);
+ }
+ } finally {
+ if (oldRe != null && owner.indexMaintenanceSynchronous) {
+ oldRe.setUpdateInProgress(false);
+ }
+ }
+ if (invokeCallbacks) {
+ if (!oldIsRemoved) {
+ cbEvent.makeUpdate();
+ }
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(cbEvent.op.isCreate() ? EnumListenerEvent.AFTER_CREATE : EnumListenerEvent.AFTER_UPDATE,
+ cbEvent, true/*callDispatchListenerEvent*/);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ if (!clearOccured) {
+ lruEntryUpdate(oldRe);
+ }
+ }
+ }
+ }
+ if (!opCompleted) {
+ putOp = putOp.getCorrespondingCreateOp();
+ if (cbEvent != null) {
+ cbEvent.setRegionEntry(newRe);
+ cbEvent.setOldValue(null);
+ }
+ boolean clearOccured = false;
+ // Set RegionEntry updateInProgress
+ if (owner.indexMaintenanceSynchronous) {
+ newRe.setUpdateInProgress(true);
+ }
+ try {
+ txRemoveOldIndexEntry(putOp, newRe);
+ // creating a new entry
+ if (didDestroy) {
+ newRe.txDidDestroy(owner.cacheTimeMillis());
+ }
+ if (txEvent != null) {
+ txEvent.addPut(putOp, owner, newRe, newRe.getKey(), newValue,aCallbackArgument);
+ }
+ newRe.setValueResultOfSearch(putOp.isNetSearch());
+ try {
+
+ processAndGenerateTXVersionTag(owner, cbEvent, newRe, txEntryState);
+ if (sqlfEvent != null ) {
+ sqlfEvent.putNewEntry(owner,newRe);
+ } else {
+ newRe.setValue(owner, newRe.prepareValueForCache(owner, newValue, cbEvent, !putOp.isCreate()));
+ }
+ owner.updateSizeOnCreate(newRe.getKey(), owner.calculateRegionEntryValueSize(newRe));
+ }
+ catch (RegionClearedException rce) {
+ clearOccured = true;
+ }
+ {
+ long lastMod = System.currentTimeMillis();
+ EntryLogger.logTXPut(_getOwnerObject(), key, nv);
+ newRe.updateStatsForPut(lastMod);
+ owner.txApplyPutPart2(newRe, newRe.getKey(), newValue, lastMod,
+ true, didDestroy, clearOccured);
+ }
+ } finally {
+ if (newRe != null && owner.indexMaintenanceSynchronous) {
+ newRe.setUpdateInProgress(false);
+ }
+ }
+ opCompleted = true;
+ if (invokeCallbacks) {
+ cbEvent.makeCreate();
+ cbEvent.setOldValue(null);
+ switchEventOwnerAndOriginRemote(cbEvent, hasRemoteOrigin);
+ if(pendingCallbacks==null) {
+ owner.invokeTXCallbacks(EnumListenerEvent.AFTER_CREATE, cbEvent,
+ true/*callDispatchListenerEvent*/);
+ } else {
+ pendingCallbacks.add(cbEvent);
+ cbEventInPending = true;
+ }
+ }
+ if (!clearOccured) {
+ lruEntryCreate(newRe);
+ incEntryCount(1);
+ }
+ }
+ }
+ finally {
+ if (!opCompleted) {
+ removeEntry(key, newRe, false);
+ }
+ }
+ }
+ if (owner.concurrencyChecksEnabled && txEntryState != null && cbEvent != null) {
+ txEntryState.setVersionTag(cbEvent.getVersionTag());
+ }
+ }catch( DiskAccessException dae) {
+ owner.handleDiskAccessException(dae);
+ throw dae;
+ }
+ finally {
+ releaseTXCacheModificationLock(owner, versionTag);
+ if (oqlIndexManager != null) {
+ oqlIndexManager.countDownIndexUpdaters();
+ }
+ }
+ } finally {
+ if (!cbEventInPending) cbEvent.release();
+ if (sqlfEvent != null) sqlfEvent.release();
+ }
+ }
+
+ private void txHandleWANEvent(final LocalRegion owner, EntryEventImpl cbEvent, TXEntryState txEntryState) {
+ ((BucketRegion)owner).handleWANEvent(cbEvent);
+ if (txEntryState != null) {
+ txEntryState.setTailKey(cbEvent.getTailKey());
+ }
+ }
+
+ /**
+ * called from txApply* methods to process and generate versionTags.
+ */
+ private void processAndGenerateTXVersionTag(final LocalRegion owner,
+ EntryEventImpl cbEvent, RegionEntry re, TXEntryState txEntryState) {
+ if (shouldPerformConcurrencyChecks(owner, cbEvent)) {
+ try {
+ if (txEntryState != null && txEntryState.getRemoteVersionTag() != null) {
+ // to generate a version based on a remote VersionTag, we will
+ // have to put the remote versionTag in the regionEntry
+ VersionTag remoteTag = txEntryState.getRemoteVersionTag();
+ if (re instanceof VersionStamp) {
+ VersionStamp stamp = (VersionStamp) re;
+ stamp.setVersions(remoteTag);
+ }
+ }
+ processVersionTag(re, cbEvent);
+ } catch (ConcurrentCacheModificationException ignore) {
+ // ignore this execption, however invoke callbacks for this operation
+ }
+
+ // For distributed transactions, stuff the next region version generated
+ // in phase-1 commit into the cbEvent so that ARE.generateVersionTag can later
+ // just apply it and not regenerate it in phase-2 commit
+ if (cbEvent != null && txEntryState != null && txEntryState.getDistTxEntryStates() != null) {
+ cbEvent.setNextRegionVersion(txEntryState.getDistTxEntryStates().getRegionVersion());
+ }
+
+ //cbEvent.setNextRegionVersion(txEntryState.getNextRegionVersion());
+ owner.generateAndSetVersionTag(cbEvent, re);
+ }
+ }
+
+ /**
+ * Checks for concurrency checks enabled on Region and that cbEvent is not null.
+ */
+ private boolean shouldPerformConcurrencyChecks(LocalRegion owner, EntryEventImpl cbEvent) {
+ return owner.getConcurrencyChecksEnabled() && cbEvent != null;
+ }
+
+ /**
+ * Switch the event's region from BucketRegion to owning PR and set originRemote to the given value
+ */
+ static EntryEventImpl switchEventOwnerAndOriginRemote(EntryEventImpl event, boolean originRemote) {
+ assert event != null;
+ if (event.getRegion().isUsedForPartitionedRegionBucket()) {
+ LocalRegion pr = event.getRegion().getPartitionedRegion();
+ event.setRegion(pr);
+ }
+ event.setOriginRemote(originRemote);
+ return event;
+ }
+
+ /**
+ * Removing the existing indexed value requires the current value in the cache,
+ * that is the one prior to applying the operation.
+ * @param op
+ * @param entry the RegionEntry that contains the value prior to applying the op
+ */
+ private void txRemoveOldIndexEntry(Operation op, RegionEntry entry) {
+ if ((op.isUpdate() && !entry.isInvalid()) ||
+ op.isInvalidate() || op.isDestroy()) {
+ IndexManager idxManager = _getOwner().getIndexManager();
+ if (idxManager != null) {
+ try {
+ idxManager.updateIndexes(entry,
+ IndexManager.REMOVE_ENTRY,
+ op.isUpdate() ?
+ IndexProtocol.BEFORE_UPDATE_OP :
+ IndexProtocol.OTHER_OP);
+ } catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }
+ }
+
+ public void dumpMap() {
+ logger.debug("dump of concurrent map of size {} for region {}", this._getMap().size(), this._getOwner());
+ for (Iterator it = this._getMap().values().iterator(); it.hasNext(); ) {
+ logger.trace("dumpMap:"+it.next().toString());
+ }
+ }
+
+ static boolean shouldCreateCBEvent( final LocalRegion owner,
+ final boolean isInvalidate, final boolean isInitialized)
+ {
+ LocalRegion lr = owner;
+ boolean isPartitioned = lr.isUsedForPartitionedRegionBucket();
+
+ if(isPartitioned){
+ /* if(!((BucketRegion)lr).getBucketAdvisor().isPrimary()) {
+ if(!BucketRegion.FORCE_LOCAL_LISTENERS_INVOCATION) {
+ return false;
+ }
+ }*/
+ lr = owner.getPartitionedRegion();
+ }
+ if (isInvalidate) { // ignore shouldNotifyGatewayHub check for invalidates
+ return (isPartitioned || isInitialized)
+ && (lr.shouldDispatchListenerEvent()
+ || lr.shouldNotifyBridgeClients()
+ || lr.getConcurrencyChecksEnabled());
+ } else {
+ return (isPartitioned || isInitialized)
+ && (lr.shouldDispatchListenerEvent()
+ || lr.shouldNotifyBridgeClients()
+ || lr.getConcurrencyChecksEnabled());
+ }
+ }
+
+ /** create a callback event for applying a transactional change to the local cache */
+ public static final EntryEventImpl createCBEvent(final LocalRegion re,
+ Operation op, Object key, Object newValue, TransactionId txId,
+ TXRmtEvent txEvent,EventID eventId, Object aCallbackArgument,FilterRoutingInfo filterRoutingInfo,ClientProxyMembershipID bridgeContext, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+ DistributedMember originator = null ;
+ //txId should not be null even on localOrigin
+ Assert.assertTrue(txId != null);
+ originator = ((TXId)txId).getMemberId();
+
+ LocalRegion eventRegion = re;
+ if (eventRegion.isUsedForPartitionedRegionBucket()) {
+ eventRegion = re.getPartitionedRegion();
+ }
+
+ EntryEventImpl retVal = EntryEventImpl.create(
+ re, op, key, newValue,
+ aCallbackArgument,
+ txEntryState == null, originator);
+ boolean returnedRetVal = false;
+ try {
+
+
+ if(bridgeContext!=null) {
+ retVal.setContext(bridgeContext);
+ }
+
+ if (eventRegion.generateEventID()) {
+ retVal.setEventId(eventId);
+ }
+
+ if (versionTag != null) {
+ retVal.setVersionTag(versionTag);
+ }
+
+ retVal.setTailKey(tailKey);
+
+ FilterInfo localRouting = null;
+ boolean computeFilterInfo = false;
+ if (filterRoutingInfo == null) {
+ computeFilterInfo = true;
+ } else {
+ localRouting = filterRoutingInfo.getLocalFilterInfo();
+ if (localRouting != null) {
+ // routing was computed in this VM but may need to perform local interest processing
+ computeFilterInfo = !filterRoutingInfo.hasLocalInterestBeenComputed();
+ } else {
+ // routing was computed elsewhere and is in the "remote" routing table
+ localRouting = filterRoutingInfo.getFilterInfo(re.getMyId());
+ }
+ if (localRouting != null) {
+ if (!computeFilterInfo) {
+ retVal.setLocalFilterInfo(localRouting);
+ }
+ } else {
+ computeFilterInfo = true;
+ }
+ }
+ if (logger.isTraceEnabled()) {
+ logger.trace("createCBEvent filterRouting={} computeFilterInfo={} local routing={}", filterRoutingInfo, computeFilterInfo, localRouting);
+ }
+
+ if (re.isUsedForPartitionedRegionBucket()) {
+ BucketRegion bucket = (BucketRegion)re;
+ if(BucketRegion.FORCE_LOCAL_LISTENERS_INVOCATION || bucket.getBucketAdvisor().isPrimary()) {
+ retVal.setInvokePRCallbacks(true);
+ } else {
+ retVal.setInvokePRCallbacks(false);
+ }
+
+ if (computeFilterInfo) {
+ if (bucket.getBucketAdvisor().isPrimary()) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("createCBEvent computing routing for primary bucket");
+ }
+ FilterProfile fp = ((BucketRegion)re).getPartitionedRegion().getFilterProfile();
+ if (fp != null) {
+ FilterRoutingInfo fri = fp.getFilterRoutingInfoPart2(filterRoutingInfo, retVal);
+ if (fri != null) {
+ retVal.setLocalFilterInfo(fri.getLocalFilterInfo());
+ }
+ }
+ }
+ }
+ } else if (computeFilterInfo) { // not a bucket
+ if (logger.isTraceEnabled()) {
+ logger.trace("createCBEvent computing routing for non-bucket");
+ }
+ FilterProfile fp = re.getFilterProfile();
+ if (fp != null) {
+ retVal.setLocalFilterInfo(fp.getLocalFilterRouting(retVal));
+ }
+ }
+ retVal.setTransactionId(txId);
+ returnedRetVal = true;
+ return retVal;
+ } finally {
+ if (!returnedRetVal) {
+ retVal.release();
+ }
+ }
+ }
+
+ public final void writeSyncIfPresent(Object key, Runnable runner)
+ {
+ RegionEntry re = getEntry(key);
+ if (re != null) {
+ final boolean disabled = disableLruUpdateCallback();
+ try {
+ synchronized (re) {
+ if (!re.isRemoved()) {
+ runner.run();
+ }
+ }
+ }
+ finally {
+ if (disabled) {
+ enableLruUpdateCallback();
+ }
+ try {
+ lruUpdateCallback();
+ }catch(DiskAccessException dae) {
+ this._getOwner().handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+ }
+ }
+
+ public final void removeIfDestroyed(Object key)
+ {
+ LocalRegion owner = _getOwner();
+// boolean makeTombstones = owner.concurrencyChecksEnabled;
+ DiskRegion dr = owner.getDiskRegion();
+ RegionEntry re = getEntry(key);
+ if (re != null) {
+ if (re.isDestroyed()) {
+ synchronized (re) {
+ if (re.isDestroyed()) {
+ // [bruce] destroyed entries aren't in the LRU clock, so they can't be retained here
+// if (makeTombstones) {
+// re.makeTombstone(owner, re.getVersionStamp().asVersionTag());
+// } else {
+ re.removePhase2();
+ removeEntry(key, re, true);
+ }
+ }
+ }
+ }
+// }
+ }
+
+
+ /** get version-generation permission from the region's version vector */
+ private void lockForCacheModification(LocalRegion owner, EntryEventImpl event) {
+ boolean lockedByBulkOp = event.isBulkOpInProgress() && owner.dataPolicy.withReplication();
+ if (!event.isOriginRemote() && !lockedByBulkOp) {
+ RegionVersionVector vector = owner.getVersionVector();
+ if (vector != null) {
+ vector.lockForCacheModification(owner);
+ }
+ }
+ }
+
+ /** release version-generation permission from the region's version vector */
+ private void releaseCacheModificationLock(LocalRegion owner, EntryEventImpl event) {
+ boolean lockedByBulkOp = event.isBulkOpInProgress() && owner.dataPolicy.withReplication();
+ if (!event.isOriginRemote() && !lockedByBulkOp) {
+ RegionVersionVector vector = owner.getVersionVector();
+ if (vector != null) {
+ vector.releaseCacheModificationLock(owner);
+ }
+ }
+ }
+
+ /** get version-generation permission from the region's version vector */
+ private void lockForTXCacheModification(LocalRegion owner, VersionTag tag) {
+ if ( !(tag != null && tag.isFromOtherMember()) ) {
+ RegionVersionVector vector = owner.getVersionVector();
+ if (vector != null) {
+ vector.lockForCacheModification(owner);
+ }
+ }
+ }
+
+ /** release version-generation permission from the region's version vector */
+ private void releaseTXCacheModificationLock(LocalRegion owner, VersionTag tag) {
+ if ( !(tag != null && tag.isFromOtherMember()) ) {
+ RegionVersionVector vector = owner.getVersionVector();
+ if (vector != null) {
+ vector.releaseCacheModificationLock(owner);
+ }
+ }
+ }
+
+ public final void unscheduleTombstone(RegionEntry re) {
+ }
+
+ /**
+ * for testing race conditions between threads trying to apply ops to the
+ * same entry
+ * @param entry the entry to attempt to add to the system
+ */
+ protected final RegionEntry putEntryIfAbsentForTest(RegionEntry entry) {
+ return (RegionEntry)putEntryIfAbsent(entry.getKey(), entry);
+ }
+
+ public boolean isTombstoneNotNeeded(RegionEntry re, int destroyedVersion) {
+ // no need for synchronization - stale values are okay here
+ RegionEntry actualRe = getEntry(re.getKey());
+ // TODO this looks like a problem for regionEntry pooling
+ if (actualRe != re) { // null actualRe is okay here
+ return true; // tombstone was evicted at some point
+ }
+ VersionStamp vs = re.getVersionStamp();
+ if (vs == null) {
+ // if we have no VersionStamp why were we even added as a tombstone?
+ // We used to see an NPE here. See bug 52092.
+ logger.error("Unexpected RegionEntry scheduled as tombstone: re.getClass {} destroyedVersion {}", re.getClass(), destroyedVersion);
+ return true;
+ }
+ int entryVersion = vs.getEntryVersion();
+ boolean isSameTombstone = (entryVersion == destroyedVersion && re.isTombstone());
+ return !isSameTombstone;
+ }
+
+ /** removes a tombstone that has expired locally */
+ public final boolean removeTombstone(RegionEntry re, VersionHolder version, boolean isEviction, boolean isScheduledTombstone) {
+ boolean result = false;
+ int destroyedVersion = version.getEntryVersion();
+
+ synchronized(this._getOwner().getSizeGuard()) { // do this sync first; see bug 51985
+ synchronized (re) {
+ int entryVersion = re.getVersionStamp().getEntryVersion();
+ boolean isTombstone = re.isTombstone();
+ boolean isSameTombstone = (entryVersion == destroyedVersion && isTombstone);
+ if (isSameTombstone || (isTombstone && entryVersion < destroyedVersion)) {
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE_COUNT)) {
+ // logs are at info level for TomstoneService.DEBUG_TOMBSTONE_COUNT so customer doesn't have to use fine level
+ if (isSameTombstone) {
+ // logging this can put tremendous pressure on the log writer in tests
+ // that "wait for silence"
+ logger.trace(LogMarker.TOMBSTONE_COUNT,
+ "removing tombstone for {} with v{} rv{}; count is {}",
+ re.getKey(), destroyedVersion, version.getRegionVersion(), (this._getOwner().getTombstoneCount() - 1));
+ } else {
+ logger.trace(LogMarker.TOMBSTONE_COUNT, "removing entry (v{}) that is older than an expiring tombstone (v{} rv{}) for {}",
+ entryVersion, destroyedVersion, version.getRegionVersion(), re.getKey());
+ }
+ }
+ try {
+ re.setValue(_getOwner(), Token.REMOVED_PHASE2);
+ if (removeTombstone(re)) {
+ result = true;
+ incEntryCount(-1);
+ // Bug 51118: When the method is called by tombstoneGC thread, current 're' is an
+ // expired tombstone. Then we detected an destroyed (due to overwritingOldTombstone()
+ // returns true earlier) tombstone with bigger entry version, it's safe to delete
+ // current tombstone 're' and adjust the tombstone count.
+ // lruEntryDestroy(re); // tombstones are invisible to LRU
+ if (isScheduledTombstone) {
+ _getOwner().incTombstoneCount(-1);
+ }
+ RegionVersionVector vector = _getOwner().getVersionVector();
+ if (vector != null) {
+ vector.recordGCVersion(version.getMemberID(), version.getRegionVersion());
+ }
+ }
+ } catch (RegionClearedException e) {
+ // if the region has been cleared we don't need to remove the tombstone
+ } catch (RegionDestroyedException e) {
+ //if the region has been destroyed, the tombstone is already
+ //gone. Catch an exception to avoid an error from the GC thread.
+ }
+ } else {
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE_COUNT)) {
+ logger.trace(LogMarker.TOMBSTONE_COUNT,
+ "tombstone for {} was resurrected with v{}; destroyed version was v{}; count is {}; entryMap size is {}",
+ re.getKey(), re.getVersionStamp().getEntryVersion(), destroyedVersion, this._getOwner().getTombstoneCount(), size());
+ }
+ }
+ }
+ }
+ return result;
+ }
+
+ protected boolean removeTombstone(RegionEntry re) {
+ return _getMap().remove(re.getKey(), re);
+ }
+
+ // method used for debugging tombstone count issues
+ public boolean verifyTombstoneCount(AtomicInteger numTombstones) {
+ int deadEntries = 0;
+ try {
+ for (Iterator it=_getMap().values().iterator(); it.hasNext(); ) {
+ RegionEntry re = (RegionEntry)it.next();
+ if (re.isTombstone()) {
+ deadEntries++;
+ }
+ }
+ if (deadEntries != numTombstones.get()) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("tombstone count ({}) does not match actual number of tombstones ({})",
+ numTombstones, deadEntries, new Exception());
+ }
+ return false;
+ } else {
+ if (logger.isDebugEnabled()) {
+ logger.debug("tombstone count verified");
+ }
+ }
+ } catch (Exception e) {
+ // ignore
+ }
+ return true;
+ }
+}
+
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
new file mode 100644
index 000000000000..ad6c40cd17dd
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+
+/**
+ * Used to fetch a record's raw bytes and user bits.
+ * The actual data length in byte array may be less than
+ * the size of the byte array itself. An integer field contains
+ * the valid length. This class is used exclusively by the Oplog Compactor
+ * for rolling the entries. The reason for this class is to reuse the
+ * underlying byte array for rolling multiple entries there by
+ * reducing the garbage.
+ * @since 5.5
+ */
+public class BytesAndBitsForCompactor {
+ /**
+ * If dataChunk is set then ignore the "data" and "validLength" fields.
+ * The dataChunk field is unretained so it can only be used while the RegionEntry is still synced.
+ * When done with the dataChunk, null it out if you want to reuse the byte[] later.
+ */
+ private @Unretained Chunk dataChunk;
+ private byte[] data;
+ private byte userBits=0;
+ // length of the data present in the byte array
+ private int validLength;
+ private static final byte[] INIT_FOR_WRAPPER = new byte[0];
+ // boolean indicating if the object can be reused.
+ // Typically if the data stores the reference of a value byte [] directly
+ // from the RegionEntry than this byte array cannot be reused for
+ //storing another entry's data
+ private boolean isReusable ;
+
+ public BytesAndBitsForCompactor() {
+ this.data = INIT_FOR_WRAPPER;
+ //this.userBits = userBits;
+ this.validLength = INIT_FOR_WRAPPER.length;
+ this.isReusable = true;
+ }
+
+
+ public final Chunk getDataChunk() {
+ return this.dataChunk;
+ }
+ public final byte[] getBytes() {
+ return this.data;
+ }
+ public final byte getBits() {
+ return this.userBits;
+ }
+
+ public final int getValidLength() {
+ return this.validLength;
+ }
+
+ public boolean isReusable() {
+ return this.isReusable;
+ }
+
+ /**
+ *
+ * @param data byte array storing the data
+ * @param userBits byte with appropriate bits set
+ * @param validLength The number of bytes representing the data , starting from 0 as offset
+ * @param isReusable true if this object is safe for reuse as a data holder
+ */
+ public void setData(byte[] data, byte userBits, int validLength, boolean isReusable) {
+ this.data = data;
+ this.userBits = userBits;
+ this.validLength = validLength;
+ this.isReusable = isReusable;
+ }
+ public void setChunkData(Chunk c, byte userBits) {
+ this.dataChunk = c;
+ this.userBits = userBits;
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
new file mode 100644
index 000000000000..07d370b5a85a
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
@@ -0,0 +1,2027 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.cache.DiskAccessException;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.ByteArrayDataInput;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.ValueWrapper;
+import com.gemstone.gemfire.internal.cache.DiskStoreImpl.AsyncDiskEntry;
+import com.gemstone.gemfire.internal.cache.lru.EnableLRU;
+import com.gemstone.gemfire.internal.cache.lru.LRUClockNode;
+import com.gemstone.gemfire.internal.cache.lru.LRUEntry;
+import com.gemstone.gemfire.internal.cache.persistence.BytesAndBits;
+import com.gemstone.gemfire.internal.cache.persistence.DiskRecoveryStore;
+import com.gemstone.gemfire.internal.cache.persistence.DiskRegionView;
+import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
+import com.gemstone.gemfire.internal.offheap.Releasable;
+import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.internal.offheap.annotations.Released;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+import com.gemstone.gemfire.internal.util.BlobHelper;
+
+/**
+ * Represents an entry in an {@link RegionMap} whose value may be
+ * stored on disk. This interface provides accessor and mutator
+ * methods for a disk entry's state. This allows us to abstract all
+ * of the interesting behavior into a {@linkplain DiskEntry.Helper
+ * helper class} that we only need to implement once.
+ *
+ *
+ *
+ * Each DiskEntry
has a unique id
that is
+ * used by the {@link DiskRegion} to identify the key/value pair.
+ * Before the disk entry is written to disk, the value of the
+ * id
is {@link DiskRegion#INVALID_ID invalid}. Once the
+ * object has been written to disk, the id
is a positive
+ * number. If the value is {@linkplain Helper#update updated}, then the
+ * id
is negated to signify that the value on disk is
+ * dirty.
+ *
+ * @see DiskRegion
+ *
+ *
+ * @since 3.2
+ */
+public interface DiskEntry extends RegionEntry {
+ /**
+ * Sets the value with a {@link RegionEntryContext}.
+ * @param context the value's context.
+ * @param value an entry value.
+ */
+ public void setValueWithContext(RegionEntryContext context,Object value);
+
+ /**
+ * In some cases we need to do something just before we drop the value
+ * from a DiskEntry that is being moved (i.e. overflowed) to disk.
+ * @param context
+ */
+ public void handleValueOverflow(RegionEntryContext context);
+
+ /**
+ * In some cases we need to do something just after we unset the value
+ * from a DiskEntry that has been moved (i.e. overflowed) to disk.
+ * @param context
+ */
+ public void afterValueOverflow(RegionEntryContext context);
+
+ /**
+ * Returns true if the DiskEntry value is equal to {@link Token#DESTROYED}, {@link Token#REMOVED_PHASE1}, or {@link Token#REMOVED_PHASE2}.
+ */
+ public boolean isRemovedFromDisk();
+
+ /**
+ * Returns the id of this DiskEntry
+ */
+ public DiskId getDiskId();
+
+ public void _removePhase1();
+
+ public int updateAsyncEntrySize(EnableLRU capacityController);
+
+ public DiskEntry getPrev();
+ public DiskEntry getNext();
+ public void setPrev(DiskEntry v);
+ public void setNext(DiskEntry v);
+
+ /**
+ * Used as the entry value if it was invalidated.
+ */
+ public static final byte[] INVALID_BYTES = new byte[0];
+ /**
+ * Used as the entry value if it was locally invalidated.
+ */
+ public static final byte[] LOCAL_INVALID_BYTES = new byte[0];
+ /**
+ * Used as the entry value if it was tombstone.
+ */
+ public static final byte[] TOMBSTONE_BYTES = new byte[0];
+
+ /////////////////////// Inner Classes //////////////////////
+
+ /**
+ * A Helper class for performing functions common to all
+ * DiskEntry
s.
+ */
+ public static class Helper {
+ private static final Logger logger = LogService.getLogger();
+
+ /**
+ * Testing purpose only
+ * Get the value of an entry that is on disk without faulting
+ * it in and without looking in the io buffer.
+ * @since 3.2.1
+ */
+ static Object getValueOnDisk(DiskEntry entry, DiskRegion dr) {
+ DiskId id = entry.getDiskId();
+ if (id == null) {
+ return null;
+ }
+ dr.acquireReadLock();
+ try {
+ synchronized (id) {
+ if (id == null
+ || (dr.isBackup() && id.getKeyId() == DiskRegion.INVALID_ID)
+ || (!entry.isValueNull() && id.needsToBeWritten() && !EntryBits.isRecoveredFromDisk(id.getUserBits()))/*fix for bug 41942*/) {
+ return null;
+ }
+
+ return dr.getNoBuffer(id);
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ }
+
+ /**
+ * Get the serialized value directly from disk. Returned object may be
+ * a {@link CachedDeserializable}. Goes straight to disk without faulting
+ * into memory. Only looks at the disk storage, not at heap storage.
+ * @param entry the entry used to identify the value to fetch
+ * @param dr the persistent storage from which to fetch the value
+ * @return either null, byte array, or CacheDeserializable
+ * @since gemfire57_hotfix
+ */
+ public static Object getSerializedValueOnDisk(
+ DiskEntry entry, DiskRegion dr) {
+ DiskId did = entry.getDiskId();
+ if (did == null) {
+ return null;
+ }
+ dr.acquireReadLock();
+ try {
+ synchronized (did) {
+ if (did == null
+ || (dr.isBackup() && did.getKeyId() == DiskRegion.INVALID_ID)) {
+ return null;
+ } else if (!entry.isValueNull() && did.needsToBeWritten() && !EntryBits.isRecoveredFromDisk(did.getUserBits())/*fix for bug 41942*/) {
+ return null;
+ }
+ return dr.getSerializedData(did);
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ }
+
+
+ /**
+ * Get the value of an entry that is on disk without
+ * faulting it in . It checks for the presence in the buffer also.
+ * This method is used for concurrent map operations, SQLFabric and CQ processing
+ *
+ * @throws DiskAccessException
+ * @since 5.1
+ */
+ static Object getValueOnDiskOrBuffer(DiskEntry entry, DiskRegion dr, RegionEntryContext context) {
+ @Released Object v = getOffHeapValueOnDiskOrBuffer(entry, dr, context);
+ if (v instanceof CachedDeserializable) {
+ if (v instanceof Chunk) {
+ @Released Chunk ohv = (Chunk) v;
+ try {
+ v = ohv.getDeserializedValue(null, null);
+ if (v == ohv) {
+ throw new IllegalStateException("sqlf tried to use getValueOnDiskOrBuffer");
+ }
+ } finally {
+ ohv.release(); // OFFHEAP the offheap ref is decremented here
+ }
+ } else {
+ v = ((CachedDeserializable)v).getDeserializedValue(null, null);
+ }
+ }
+ return v;
+ }
+
+ @Retained
+ static Object getOffHeapValueOnDiskOrBuffer(DiskEntry entry, DiskRegion dr, RegionEntryContext context) {
+ DiskId did = entry.getDiskId();
+ Object syncObj = did;
+ if (syncObj == null) {
+ syncObj = entry;
+ }
+ if (syncObj == did) {
+ dr.acquireReadLock();
+ }
+ try {
+ synchronized (syncObj) {
+ if (did != null && did.isPendingAsync()) {
+ @Retained Object v = entry._getValueRetain(context, true); // TODO:KIRK:OK Rusty had Object v = entry.getValueWithContext(context);
+ if (Token.isRemovedFromDisk(v)) {
+ v = null;
+ }
+ return v;
+ }
+ if (did == null
+ || ( dr.isBackup() && did.getKeyId() == DiskRegion.INVALID_ID)
+ || (!entry.isValueNull() && did.needsToBeWritten() && !EntryBits.isRecoveredFromDisk(did.getUserBits()))/*fix for bug 41942*/) {
+ return null;
+ }
+
+ return dr.getSerializedData(did);
+ }
+ } finally {
+ if (syncObj == did) {
+ dr.releaseReadLock();
+ }
+ }
+ }
+
+ /**
+ * Returns false if the entry is INVALID (or LOCAL_INVALID). Determines this
+ * without faulting in the value from disk.
+ *
+ * @since 3.2.1
+ */
+ /* TODO prpersist - Do we need this method? It was added by the sqlf merge
+ static boolean isValid(DiskEntry entry, DiskRegion dr) {
+ synchronized (entry) {
+ if (entry.isRecovered()) {
+ // We have a recovered entry whose value is still on disk.
+ // So take a peek at it without faulting it in.
+ //long id = entry.getDiskId().getKeyId();
+ //entry.getDiskId().setKeyId(-id);
+ byte bits = dr.getBits(entry.getDiskId());
+ //TODO Asif:Check if resetting is needed
+ return !EntryBits.isInvalid(bits) && !EntryBits.isLocalInvalid(bits);
+ }
+ }
+ }*/
+
+ static boolean isOverflowedToDisk(DiskEntry de, DiskRegion dr, DistributedRegion.DiskPosition dp,RegionEntryContext context) {
+ Object v = null;
+ DiskId did;
+ synchronized (de) {
+ did = de.getDiskId();
+ }
+ Object syncObj = did;
+ if (syncObj == null) {
+ syncObj = de;
+ }
+ if (syncObj == did) {
+ dr.acquireReadLock();
+ }
+ try {
+ synchronized (syncObj) {
+ if (de.isValueNull()) {
+ if (did == null) {
+ synchronized (de) {
+ did = de.getDiskId();
+ }
+ assert did != null;
+ return isOverflowedToDisk(de, dr, dp, context);
+ } else {
+ dp.setPosition(did.getOplogId(), did.getOffsetInOplog());
+ return true;
+ }
+ } else {
+ return false;
+ }
+ }
+ } finally {
+ if (syncObj == did) {
+ dr.releaseReadLock();
+ }
+ }
+ }
+
+ /**
+ * Get the value of an entry that is on disk without faulting
+ * it in.
+ * @since 3.2.1
+ */
+ static boolean fillInValue(DiskEntry de, InitialImageOperation.Entry entry,
+ DiskRegion dr, DM mgr, ByteArrayDataInput in, RegionEntryContext context) {
+ @Retained @Released Object v = null;
+ DiskId did;
+ synchronized (de) {
+ did = de.getDiskId();
+ }
+ Object syncObj = did;
+ if (syncObj == null) {
+ syncObj = de;
+ }
+ if (syncObj == did) {
+ dr.acquireReadLock();
+ }
+ try {
+ synchronized (syncObj) {
+ entry.setLastModified(mgr, de.getLastModified());
+
+ ReferenceCountHelper.setReferenceCountOwner(entry);
+ v = de._getValueRetain(context, true); // OFFHEAP copied to heap entry; todo allow entry to refer to offheap since it will be copied to network.
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ if (v == null) {
+ if (did == null) {
+ // fix for bug 41449
+ synchronized (de) {
+ did = de.getDiskId();
+ }
+ assert did != null;
+ // do recursive call to get readLock on did
+ return fillInValue(de, entry, dr, mgr, in, context);
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("DiskEntry.Helper.fillInValue, key={}; getting value from disk, disk id={}", entry.key, did);
+ }
+ BytesAndBits bb = null;
+ try {
+ bb = dr.getBytesAndBits(did, false);
+ }catch(DiskAccessException dae){
+ return false;
+ }
+ if (EntryBits.isInvalid(bb.getBits())) {
+ entry.setInvalid();
+ }
+ else if (EntryBits.isLocalInvalid(bb.getBits())) {
+ entry.setLocalInvalid();
+ }
+ else if (EntryBits.isTombstone(bb.getBits())) {
+ entry.setTombstone();
+ }
+ else {
+ entry.value = bb.getBytes();
+ entry.setSerialized(EntryBits.isSerialized(bb.getBits()));
+ }
+ return true;
+ }
+ }
+ } finally {
+ if (syncObj == did) {
+ dr.releaseReadLock();
+ }
+ }
+ final boolean isEagerDeserialize = entry.isEagerDeserialize();
+ if (isEagerDeserialize) {
+ entry.clearEagerDeserialize();
+ }
+ if (Token.isRemovedFromDisk(v)) {
+ // fix for bug 31757
+ return false;
+ } else if (v instanceof CachedDeserializable) {
+ try {
+ if (v instanceof StoredObject && !((StoredObject) v).isSerialized()) {
+ entry.setSerialized(false);
+ entry.value = ((StoredObject) v).getDeserializedForReading();
+
+ //For SQLFire we prefer eager deserialized
+// if(v instanceof ByteSource) {
+// entry.setEagerDeserialize();
+// }
+ } else {
+ // don't serialize here if it is not already serialized
+
+ Object tmp = ((CachedDeserializable)v).getValue();
+ //For SQLFire we prefer eager deserialized
+// if(v instanceof ByteSource) {
+// entry.setEagerDeserialize();
+// }
+ if (tmp instanceof byte[]) {
+ byte[] bb = (byte[])tmp;
+ entry.value = bb;
+ entry.setSerialized(true);
+ }
+ else if (isEagerDeserialize && tmp instanceof byte[][]) {
+ // optimize for byte[][] since it will need to be eagerly deserialized
+ // for SQLFabric
+ entry.value = tmp;
+ entry.setEagerDeserialize();
+ entry.setSerialized(true);
+ }
+ else {
+ try {
+ HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
+ BlobHelper.serializeTo(tmp, hdos);
+ hdos.trim();
+ entry.value = hdos;
+ entry.setSerialized(true);
+ } catch (IOException e) {
+ RuntimeException e2 = new IllegalArgumentException(LocalizedStrings.DiskEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING.toLocalizedString());
+ e2.initCause(e);
+ throw e2;
+ }
+ }
+ }
+ } finally {
+ // If v == entry.value then v is assumed to be an OffHeapByteSource
+ // and release() will be called on v after the bytes have been read from
+ // off-heap.
+ if (v != entry.value) {
+ OffHeapHelper.releaseWithNoTracking(v);
+ }
+ }
+ }
+ else if (v instanceof byte[]) {
+ entry.value = v;
+ entry.setSerialized(false);
+ }
+ else if (isEagerDeserialize && v instanceof byte[][]) {
+ // optimize for byte[][] since it will need to be eagerly deserialized
+ // for SQLFabric
+ entry.value = v;
+ entry.setEagerDeserialize();
+ }
+ else if (v == Token.INVALID) {
+ entry.setInvalid();
+ }
+ else if (v == Token.LOCAL_INVALID) {
+ // fix for bug 31107
+ entry.setLocalInvalid();
+ } else if (v == Token.TOMBSTONE) {
+ entry.setTombstone();
+ }
+ else {
+ Object preparedValue = v;
+ if (preparedValue != null) {
+ preparedValue = AbstractRegionEntry.prepareValueForGII(preparedValue);
+ if (preparedValue == null) {
+ return false;
+ }
+ }
+ if (CachedDeserializableFactory.preferObject()) {
+ entry.value = preparedValue;
+ entry.setEagerDeserialize();
+ }
+ else {
+ try {
+ HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
+ BlobHelper.serializeTo(preparedValue, hdos);
+ hdos.trim();
+ entry.value = hdos;
+ entry.setSerialized(true);
+ } catch (IOException e) {
+ RuntimeException e2 = new IllegalArgumentException(LocalizedStrings.DiskEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING.toLocalizedString());
+ e2.initCause(e);
+ throw e2;
+ }
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Used to initialize a new disk entry
+ */
+ static void initialize(DiskEntry entry, DiskRecoveryStore r, Object newValue) {
+ DiskRegionView drv = null;
+ if (r instanceof LocalRegion) {
+ drv = ((LocalRegion)r).getDiskRegion();
+ } else if (r instanceof DiskRegionView) {
+ drv = (DiskRegionView)r;
+ }
+ if (drv == null) {
+ throw new IllegalArgumentException(LocalizedStrings.DiskEntry_DISK_REGION_IS_NULL.toLocalizedString());
+ }
+
+ if (newValue == null || Token.isRemovedFromDisk(newValue)) {
+ // it is not in vm and it is not on disk
+ DiskId did = entry.getDiskId();
+ if (did != null) {
+ did.setKeyId(DiskRegion.INVALID_ID);
+ }
+ }
+ else if (newValue instanceof RecoveredEntry) {
+ // Set the id directly, the value will also be set if RECOVER_VALUES
+ RecoveredEntry re = (RecoveredEntry)newValue;
+ DiskId did = entry.getDiskId();
+ did.setOplogId(re.getOplogId());
+ did.setOffsetInOplog(re.getOffsetInOplog());
+ did.setKeyId(re.getRecoveredKeyId());
+ did.setUserBits(re.getUserBits());
+ did.setValueLength(re.getValueLength());
+ if (re.getRecoveredKeyId() < 0) {
+ drv.incNumOverflowOnDisk(1L);
+ drv.incNumOverflowBytesOnDisk(did.getValueLength());
+ incrementBucketStats(r, 0/*InVM*/, 1/*OnDisk*/, did.getValueLength());
+ }
+ else {
+ entry.setValueWithContext(drv, entry.prepareValueForCache((RegionEntryContext) r,
+ re.getValue(), false));
+ drv.incNumEntriesInVM(1L);
+ incrementBucketStats(r, 1/*InVM*/, 0/*OnDisk*/, 0);
+ }
+ }
+ else {
+ DiskId did = entry.getDiskId();
+ if (did != null) {
+ did.setKeyId(DiskRegion.INVALID_ID);
+ }
+ drv.incNumEntriesInVM(1L);
+ incrementBucketStats(r, 1/*InVM*/, 0/*OnDisk*/, 0);
+ }
+ }
+
+ private static final ValueWrapper INVALID_VW = new ByteArrayValueWrapper(true, INVALID_BYTES);
+ private static final ValueWrapper LOCAL_INVALID_VW = new ByteArrayValueWrapper(true, LOCAL_INVALID_BYTES);
+ private static final ValueWrapper TOMBSTONE_VW = new ByteArrayValueWrapper(true, TOMBSTONE_BYTES);
+
+ public static interface ValueWrapper {
+ public boolean isSerialized();
+ public int getLength();
+ public byte getUserBits();
+ public void sendTo(ByteBuffer bb, Flushable flushable) throws IOException;
+ public String getBytesAsString();
+ }
+ public static interface Flushable {
+ public void flush() throws IOException;
+
+ public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException;
+ }
+ public static class ByteArrayValueWrapper implements ValueWrapper {
+ public final boolean isSerializedObject;
+ public final byte[] bytes;
+
+ public ByteArrayValueWrapper(boolean isSerializedObject, byte[] bytes) {
+ this.isSerializedObject = isSerializedObject;
+ this.bytes = bytes;
+ }
+
+ @Override
+ public boolean isSerialized() {
+ return this.isSerializedObject;
+ }
+
+ @Override
+ public int getLength() {
+ return (this.bytes != null) ? this.bytes.length : 0;
+ }
+
+ private boolean isInvalidToken() {
+ return this == INVALID_VW;
+ }
+
+ private boolean isLocalInvalidToken() {
+ return this == LOCAL_INVALID_VW;
+ }
+
+ private boolean isTombstoneToken() {
+ return this == TOMBSTONE_VW;
+ }
+
+ @Override
+ public byte getUserBits() {
+ byte userBits = 0x0;
+ if (isSerialized()) {
+ if (isTombstoneToken()) {
+ userBits = EntryBits.setTombstone(userBits, true);
+ } else if (isInvalidToken()) {
+ userBits = EntryBits.setInvalid(userBits, true);
+ } else if (isLocalInvalidToken()) {
+ userBits = EntryBits.setLocalInvalid(userBits, true);
+ } else {
+ if (this.bytes == null) {
+ throw new IllegalStateException("userBits==1 and value is null");
+ } else if (this.bytes.length == 0) {
+ throw new IllegalStateException("userBits==1 and value is zero length");
+ }
+ userBits = EntryBits.setSerialized(userBits, true);
+ }
+ }
+ return userBits;
+ }
+
+ @Override
+ public void sendTo(ByteBuffer bb, Flushable flushable) throws IOException {
+ int offset = 0;
+ final int maxOffset = getLength();
+ while (offset < maxOffset) {
+ int bytesThisTime = maxOffset - offset;
+ boolean needsFlush = false;
+ if (bytesThisTime > bb.remaining()) {
+ needsFlush = true;
+ bytesThisTime = bb.remaining();
+ }
+ bb.put(this.bytes, offset, bytesThisTime);
+ offset += bytesThisTime;
+ if (needsFlush) {
+ flushable.flush();
+ }
+ }
+ }
+
+ @Override
+ public String getBytesAsString() {
+ if (this.bytes == null) {
+ return "null";
+ }
+ StringBuffer sb = new StringBuffer();
+ int len = getLength();
+ for (int i = 0; i < len; i++) {
+ sb.append(this.bytes[i]).append(", ");
+ }
+ return sb.toString();
+ }
+ }
+
+ /**
+ * This class is a bit of a hack used by the compactor.
+ * For the compactor always copies to a byte[] so
+ * this class is just a simple wrapper.
+ * It is possible that the length of the byte array is greater
+ * than the actual length of the wrapped data.
+ * At the time we create this we are all done with isSerialized
+ * and userBits so those methods are not supported.
+ */
+ public static class CompactorValueWrapper extends ByteArrayValueWrapper {
+ private final int length;
+
+ public CompactorValueWrapper(byte[] bytes, int length) {
+ super(false, bytes);
+ this.length = length;
+ }
+
+ @Override
+ public boolean isSerialized() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int getLength() {
+ return this.length;
+ }
+
+ @Override
+ public byte getUserBits() {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ /**
+ * Note that the Chunk this ValueWrapper is created with
+ * is unretained so it must be used before the owner of
+ * the chunk releases it.
+ * Since the RegionEntry that has the value we are writing to
+ * disk has it retained we are ok as long as this ValueWrapper's
+ * life ends before the RegionEntry sync is released.
+ * Note that this class is only used with uncompressed chunks.
+ */
+ public static class ChunkValueWrapper implements ValueWrapper {
+ private final @Unretained Chunk chunk;
+ public ChunkValueWrapper(Chunk c) {
+ assert !c.isCompressed();
+ this.chunk = c;
+ }
+ @Override
+ public boolean isSerialized() {
+ return this.chunk.isSerialized();
+ }
+ @Override
+ public int getLength() {
+ return this.chunk.getDataSize();
+ }
+ @Override
+ public byte getUserBits() {
+ byte userBits = 0x0;
+ if (isSerialized()) {
+ userBits = EntryBits.setSerialized(userBits, true);
+ }
+ return userBits;
+ }
+ @Override
+ public void sendTo(ByteBuffer bb, Flushable flushable) throws IOException {
+ final int maxOffset = getLength();
+ if (maxOffset == 0) {
+ return;
+ }
+ if (maxOffset > bb.capacity()) {
+ ByteBuffer chunkbb = this.chunk.createDirectByteBuffer();
+ if (chunkbb != null) {
+ flushable.flush(bb, chunkbb);
+ return;
+ }
+ }
+ final long bbAddress = Chunk.getDirectByteBufferAddress(bb);
+ if (bbAddress != 0L) {
+ int bytesRemaining = maxOffset;
+ int availableSpace = bb.remaining();
+ long addrToWrite = bbAddress + bb.position();
+ long addrToRead = this.chunk.getAddressForReading(0, maxOffset);
+ if (bytesRemaining > availableSpace) {
+ do {
+ UnsafeMemoryChunk.copyMemory(addrToRead, addrToWrite, availableSpace);
+ bb.position(bb.position()+availableSpace);
+ addrToRead += availableSpace;
+ bytesRemaining -= availableSpace;
+ flushable.flush();
+ addrToWrite = bbAddress + bb.position();
+ availableSpace = bb.remaining();
+ } while (bytesRemaining > availableSpace);
+ }
+ UnsafeMemoryChunk.copyMemory(addrToRead, addrToWrite, bytesRemaining);
+ bb.position(bb.position()+bytesRemaining);
+ } else {
+ long addr = this.chunk.getAddressForReading(0, maxOffset);
+ final long endAddr = addr + maxOffset;
+ while (addr != endAddr) {
+ bb.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+ addr++;
+ if (!bb.hasRemaining()) {
+ flushable.flush();
+ }
+ }
+ }
+ }
+ @Override
+ public String getBytesAsString() {
+ return this.chunk.getStringForm();
+ }
+ }
+
+ public static ValueWrapper createValueWrapper(Object value, EntryEventImpl event) {
+ if (value == Token.INVALID) {
+ // even though it is not serialized we say it is because
+ // bytes will never be an empty array when it is serialized
+ // so that gives us a way to specify the invalid value
+ // given a byte array and a boolean flag.
+ return INVALID_VW;
+ }
+ else if (value == Token.LOCAL_INVALID) {
+ // even though it is not serialized we say it is because
+ // bytes will never be an empty array when it is serialized
+ // so that gives us a way to specify the local-invalid value
+ // given a byte array and a boolean flag.
+ return LOCAL_INVALID_VW;
+ }
+ else if (value == Token.TOMBSTONE) {
+ return TOMBSTONE_VW;
+ }
+ else {
+ boolean isSerializedObject = true;
+ byte[] bytes;
+ if (value instanceof CachedDeserializable) {
+ CachedDeserializable proxy = (CachedDeserializable)value;
+ if (proxy instanceof Chunk) {
+ return new ChunkValueWrapper((Chunk) proxy);
+ }
+ if (proxy instanceof StoredObject) {
+ StoredObject ohproxy = (StoredObject) proxy;
+ isSerializedObject = ohproxy.isSerialized();
+ if (isSerializedObject) {
+ bytes = ohproxy.getSerializedValue();
+ } else {
+ bytes = (byte[]) ohproxy.getDeserializedForReading();
+ }
+ } else {
+ bytes = proxy.getSerializedValue();
+ }
+ if (event != null && isSerializedObject) {
+ event.setCachedSerializedNewValue(bytes);
+ }
+ }
+ else if (value instanceof byte[]) {
+ isSerializedObject = false;
+ bytes = (byte[])value;
+ }
+ else {
+ Assert.assertTrue(!Token.isRemovedFromDisk(value));
+ if (event != null && event.getCachedSerializedNewValue() != null) {
+ bytes = event.getCachedSerializedNewValue();
+ } else {
+ bytes = EntryEventImpl.serialize(value);
+ if (bytes.length == 0) {
+ throw new IllegalStateException("serializing <" + value + "> produced empty byte array");
+ }
+ if (event != null) {
+ event.setCachedSerializedNewValue(bytes);
+ }
+ }
+ }
+ return new ByteArrayValueWrapper(isSerializedObject, bytes);
+ }
+ }
+ public static ValueWrapper createValueWrapperFromEntry(DiskEntry entry, LocalRegion region, EntryEventImpl event) {
+ if (event != null) {
+ // For off-heap it should be faster to pass a reference to the
+ // StoredObject instead of using the cached byte[] (unless it is also compressed).
+ // Since NIO is used if the chunk of memory is large we can write it
+ // to the file with using the off-heap memory with no extra copying.
+ // So we give preference to getRawNewValue over getCachedSerializedNewValue
+ Object rawValue = null;
+ if (!event.hasDelta()) {
+ // We don't do this for the delta case because getRawNewValue returns delta
+ // and we want to write the entire new value to disk.
+ rawValue = event.getRawNewValue();
+ if (rawValue instanceof Chunk) {
+ return new ChunkValueWrapper((Chunk) rawValue);
+ }
+ }
+ if (event.getCachedSerializedNewValue() != null) {
+ return new ByteArrayValueWrapper(true, event.getCachedSerializedNewValue());
+ }
+ if (rawValue != null) {
+ return createValueWrapper(rawValue, event);
+ }
+ }
+ // TODO OFFHEAP: No need to retain since we hold the sync on entry but we need a flavor of _getValue that will decompress
+ @Retained Object value = entry._getValueRetain(region, true);
+ try {
+ return createValueWrapper(value, event);
+ } finally {
+ OffHeapHelper.release(value);
+ }
+ }
+
+ private static void writeToDisk(DiskEntry entry, LocalRegion region, boolean async) throws RegionClearedException {
+ writeToDisk(entry, region, async, null);
+ }
+
+ /**
+ * Writes the key/value object stored in the given entry to disk
+ * @throws RegionClearedException
+ *
+ * @see DiskRegion#put
+ */
+ private static void writeToDisk(DiskEntry entry, LocalRegion region, boolean async, EntryEventImpl event) throws RegionClearedException {
+ writeBytesToDisk(entry, region, async, createValueWrapperFromEntry(entry, region, event));
+ }
+
+ private static void writeBytesToDisk(DiskEntry entry, LocalRegion region, boolean async, ValueWrapper vw) throws RegionClearedException {
+ // @todo does the following unmark need to be called when an async
+ // write is scheduled or is it ok for doAsyncFlush to do it?
+ entry.getDiskId().unmarkForWriting();
+ region.getDiskRegion().put(entry, region, vw, async);
+ }
+
+ public static void update(DiskEntry entry, LocalRegion region, Object newValue) throws RegionClearedException {
+ update(entry, region, newValue, null);
+ }
+ /**
+ * Updates the value of the disk entry with a new value. This allows us to
+ * free up disk space in the non-backup case.
+ *
+ * @throws RegionClearedException
+ */
+ public static void update(DiskEntry entry, LocalRegion region, Object newValue, EntryEventImpl event) throws RegionClearedException {
+ DiskRegion dr = region.getDiskRegion();
+ if (newValue == null) {
+ throw new NullPointerException(LocalizedStrings.DiskEntry_ENTRYS_VALUE_SHOULD_NOT_BE_NULL.toLocalizedString());
+ }
+
+ //If we have concurrency checks enabled for a persistent region, we need
+ //to add an entry to the async queue for every update to maintain the RVV
+ boolean maintainRVV = region.concurrencyChecksEnabled && dr.isBackup();
+
+ Token oldValue = null;
+ int oldValueLength = 0;
+ boolean scheduleAsync = false;
+ boolean callRemoveFromDisk = false;
+ DiskId did = entry.getDiskId();
+ VersionTag tag = null;
+ Object syncObj = did;
+ if (syncObj == null) {
+ syncObj = entry;
+ }
+ if (syncObj == did) {
+ dr.acquireReadLock();
+ }
+ try {
+ synchronized (syncObj) {
+ oldValue = entry.getValueAsToken();
+ if (Token.isRemovedFromDisk(newValue)) {
+ if (dr.isBackup()) {
+ dr.testIsRecoveredAndClear(did); // fixes bug 41409
+ }
+ RuntimeException rte = null;
+ try {
+ if (!Token.isRemovedFromDisk(oldValue)) {
+ // removeFromDisk takes care of oldValueLength
+ if (dr.isSync()) {
+ removeFromDisk(entry, region, false);
+ } else {
+ callRemoveFromDisk = true; // do it outside the sync
+ }
+ }
+ } catch (RuntimeException e) {
+ rte = e;
+ throw e;
+ }
+ finally {
+ if (rte != null && (rte instanceof CacheClosedException)) {
+ // 47616: not to set the value to be removedFromDisk since it failed to persist
+ } else {
+ // Asif Ensure that the value is rightly set despite clear so
+ // that it can be distributed correctly
+ entry.setValueWithContext(region, newValue); // OFFHEAP newValue was already preparedForCache
+ }
+ }
+ }
+ else if (newValue instanceof RecoveredEntry) {
+ // Now that oplog creates are immediately put in cache
+ // a later oplog modify will get us here
+ RecoveredEntry re = (RecoveredEntry)newValue;
+ long oldKeyId = did.getKeyId();
+ long oldOplogId = did.getOplogId();
+ long newOplogId = re.getOplogId();
+ if (newOplogId != oldOplogId) {
+ did.setOplogId(newOplogId);
+ re.setOplogId(oldOplogId); // so caller knows oldoplog id
+ }
+ did.setOffsetInOplog(re.getOffsetInOplog());
+ // id already set
+ did.setUserBits(re.getUserBits());
+ oldValueLength = did.getValueLength();
+ did.setValueLength(re.getValueLength());
+ // The following undo and then do fixes bug 41849
+ // First, undo the stats done for the previous recovered value
+ if (oldKeyId < 0) {
+ dr.incNumOverflowOnDisk(-1L);
+ dr.incNumOverflowBytesOnDisk(-oldValueLength);
+ incrementBucketStats(region, 0/*InVM*/, -1/*OnDisk*/, -oldValueLength);
+ } else {
+ dr.incNumEntriesInVM(-1L);
+ incrementBucketStats(region, -1/*InVM*/, 0/*OnDisk*/, 0);
+ }
+ // Second, do the stats done for the current recovered value
+ if (re.getRecoveredKeyId() < 0) {
+ if (!entry.isValueNull()) {
+ try {
+ entry.handleValueOverflow(region);
+ entry.setValueWithContext(region, null); // fixes bug 41119
+ }finally {
+ entry.afterValueOverflow(region);
+ }
+
+ }
+ dr.incNumOverflowOnDisk(1L);
+ dr.incNumOverflowBytesOnDisk(did.getValueLength());
+ incrementBucketStats(region, 0/*InVM*/, 1/*OnDisk*/,
+ did.getValueLength());
+ } else {
+ entry.setValueWithContext(region, entry.prepareValueForCache(region, re.getValue(), false));
+ dr.incNumEntriesInVM(1L);
+ incrementBucketStats(region, 1/*InVM*/, 0/*OnDisk*/, 0);
+ }
+ }
+ else {
+ //The new value in the entry needs to be set after the disk writing
+ // has succeeded. If not , for GemFireXD , it is possible that other thread
+ // may pick this transient value from region entry ( which for
+ //offheap will eventually be released ) as index key,
+ //given that this operation is bound to fail in case of
+ //disk access exception.
+
+ //entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
+
+ if(did != null && did.isPendingAsync()) {
+ //if the entry was not yet written to disk, we didn't update
+ //the bytes on disk.
+ oldValueLength = 0;
+ } else {
+ oldValueLength = getValueLength(did);
+ }
+
+ if (dr.isBackup()) {
+ dr.testIsRecoveredAndClear(did); // fixes bug 41409
+ if (dr.isSync()) {
+ //In case of compression the value is being set first
+ // because atleast for now , GemFireXD does not support compression
+ // if and when it does support, this needs to be taken care of else
+ // we risk Bug 48965
+ if (AbstractRegionEntry.isCompressible(dr, newValue)) {
+ entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
+
+ // newValue is prepared and compressed. We can't write compressed values to disk.
+ writeToDisk(entry, region, false, event);
+ } else {
+ writeBytesToDisk(entry, region, false, createValueWrapper(newValue, event));
+ entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
+ }
+
+ } else if (did.isPendingAsync() && !maintainRVV) {
+ entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
+
+ // nothing needs to be done except
+ // fixing up LRU stats
+ // @todo fixup LRU stats if needed
+ // I'm not sure anything needs to be done here.
+ // If we have overflow and it decided to evict this entry
+ // how do we handle that case when we are async?
+ // Seems like the eviction code needs to leave the value
+ // in memory until the pendingAsync is done.
+ } else {
+ //if the entry is not async, we need to schedule it
+ //for regions with concurrency checks enabled, we add an entry
+ //to the queue for every entry.
+ scheduleAsync = true;
+ did.setPendingAsync(true);
+ VersionStamp stamp = entry.getVersionStamp();
+ if(stamp != null) {
+ tag = stamp.asVersionTag();
+ }
+ entry.setValueWithContext(region, newValue);
+ }
+ } else if (did != null) {
+ entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
+
+ // Mark the id as needing to be written
+ // The disk remove that this section used to do caused bug 30961
+ // @todo this seems wrong. How does leaving it on disk fix the bug?
+ did.markForWriting();
+ //did.setValueSerializedSize(0);
+ }else {
+ entry.setValueWithContext(region, newValue);
+ }
+
+ if (Token.isRemovedFromDisk(oldValue)) {
+ // Note we now initialize entries removed and then set their
+ // value once we find no existing entry.
+ // So this is the normal path for a brand new entry.
+ dr.incNumEntriesInVM(1L);
+ incrementBucketStats(region, 1/*InVM*/, 0/*OnDisk*/, 0);
+ }
+ }
+ if (entry instanceof LRUEntry) {
+ LRUEntry le = (LRUEntry)entry;
+ boolean wasEvicted = le.testEvicted();
+ le.unsetEvicted();
+ if (!Token.isRemovedFromDisk(newValue)) {
+ if (oldValue == null
+ // added null check for bug 41759
+ || wasEvicted && did != null && did.isPendingAsync()) {
+ // Note we do not append this entry because that will be
+ // done by lruEntryUpdate
+ dr.incNumEntriesInVM(1L);
+ dr.incNumOverflowOnDisk(-1L);
+ dr.incNumOverflowBytesOnDisk(-oldValueLength);
+ incrementBucketStats(region, 1/*InVM*/, -1/*OnDisk*/, -oldValueLength);
+ }
+ }
+ }
+ }
+ } finally {
+ if (syncObj == did) {
+ dr.releaseReadLock();
+ }
+ }
+ if (callRemoveFromDisk) {
+ removeFromDisk(entry, region, false, oldValue == null, false);
+ } else if (scheduleAsync && did.isPendingAsync()) {
+ // this needs to be done outside the above sync
+ scheduleAsyncWrite(new AsyncDiskEntry(region, entry, tag));
+ }
+ }
+
+ private static int getValueLength(DiskId did) {
+ int result = 0;
+ if (did != null) {
+ synchronized (did) {
+ result = did.getValueLength();
+ }
+ }
+ return result;
+ }
+
+ public static void updateRecoveredEntry(PlaceHolderDiskRegion drv,
+ DiskEntry entry,
+ RecoveredEntry newValue,RegionEntryContext context)
+ {
+ if (newValue == null) {
+ throw new NullPointerException(LocalizedStrings.DiskEntry_ENTRYS_VALUE_SHOULD_NOT_BE_NULL.toLocalizedString());
+ }
+ DiskId did = entry.getDiskId();
+ synchronized (did) {
+ boolean oldValueWasNull = entry.isValueNull();
+ int oldValueLength = did.getValueLength();
+ // Now that oplog creates are immediately put in cache
+ // a later oplog modify will get us here
+ long oldOplogId = did.getOplogId();
+ long newOplogId = newValue.getOplogId();
+ if (newOplogId != oldOplogId) {
+ did.setOplogId(newOplogId);
+ newValue.setOplogId(oldOplogId); // so caller knows oldoplog id
+ }
+ did.setOffsetInOplog(newValue.getOffsetInOplog());
+ // id already set
+ did.setUserBits(newValue.getUserBits());
+ did.setValueLength(newValue.getValueLength());
+ if (newValue.getRecoveredKeyId() >= 0) {
+ entry.setValueWithContext(context, entry.prepareValueForCache(drv, newValue.getValue(),
+ false));
+ } else {
+ if (!oldValueWasNull) {
+ try {
+ entry.handleValueOverflow(context);
+ entry.setValueWithContext(context,null); // fixes bug 41119
+ }finally {
+ entry.afterValueOverflow(context);
+ }
+ }
+ }
+ if (entry instanceof LRUEntry) {
+ LRUEntry le = (LRUEntry)entry;
+ assert !le.testEvicted();
+ // we don't allow eviction during recovery
+ if (oldValueWasNull) {
+ // Note we do not append this entry because that will be
+ // done by lruEntryUpdate
+ drv.incNumEntriesInVM(1L);
+ drv.incNumOverflowOnDisk(-1L);
+ drv.incNumOverflowBytesOnDisk(-oldValueLength);
+ //No need to call incrementBucketStats here because we don't have
+ //a real bucket region, this is during recovery from disk.
+ }
+ }
+ }
+ }
+
+ public static Object getValueInVMOrDiskWithoutFaultIn(DiskEntry entry, LocalRegion region) {
+ Object result = OffHeapHelper.copyAndReleaseIfNeeded(getValueOffHeapOrDiskWithoutFaultIn(entry, region));
+ if (result instanceof CachedDeserializable) {
+ result = ((CachedDeserializable)result).getDeserializedValue(null, null);
+ }
+ if (result instanceof StoredObject) {
+ ((StoredObject) result).release();
+ throw new IllegalStateException("sqlf tried to use getValueInVMOrDiskWithoutFaultIn");
+ }
+ return result;
+ }
+
+ @Retained
+ public static Object getValueOffHeapOrDiskWithoutFaultIn(DiskEntry entry, LocalRegion region) {
+ @Retained Object v = entry._getValueRetain(region, true); // TODO:KIRK:OK Object v = entry.getValueWithContext(region);
+ if (v == null || Token.isRemovedFromDisk(v)
+ && !region.isIndexCreationThread()) {
+ synchronized (entry) {
+ v = entry._getValueRetain(region, true); // TODO:KIRK:OK v = entry.getValueWithContext(region);
+ if (v == null) {
+ v = Helper.getOffHeapValueOnDiskOrBuffer(entry, region.getDiskRegion(),region);
+ }
+ }
+ }
+ if (Token.isRemovedFromDisk(v)) {
+ // fix for bug 31800
+ v = null;
+// } else if (v instanceof ByteSource) {
+// // If the ByteSource contains a Delta or ListOfDelta then we want to deserialize it
+// Object deserVal = ((CachedDeserializable)v).getDeserializedForReading();
+// if (deserVal != v) {
+// OffHeapHelper.release(v);
+// v = deserVal;
+// }
+ }
+ return v;
+ }
+
+ /**
+ *
+ * @param entry
+ * @param region
+ * @return Value
+ * @throws DiskAccessException
+ */
+ public static Object faultInValue(DiskEntry entry, LocalRegion region) {
+ return faultInValue(entry, region, false);
+ }
+ @Retained
+ public static Object faultInValueRetain(DiskEntry entry, LocalRegion region) {
+ return faultInValue(entry, region, true);
+ }
+ /**
+ * @param retainResult if true then the result may be a retained off-heap reference
+ */
+ @Retained
+ private static Object faultInValue(DiskEntry entry, LocalRegion region, boolean retainResult)
+ {
+ DiskRegion dr = region.getDiskRegion();
+ @Retained Object v = entry._getValueRetain(region, true); // TODO:KIRK:OK Object v = entry.getValueWithContext(region);
+ boolean lruFaultedIn = false;
+ boolean done = false;
+ try {
+ //Asif: If the entry is instance of LRU then DidkRegion cannot be null.
+ //Since SqlFabric is accessing this method direcly & it passes the owning region,
+ //if the region happens to be persistent PR type, the owning region passed is PR,
+ // but it will have DiskRegion as null. SqlFabric takes care of passing owning region
+ // as BucketRegion in case of Overflow type entry. This is fix for Bug # 41804
+ if ( entry instanceof LRUEntry && !dr.isSync() ) {
+ synchronized (entry) {
+ DiskId did = entry.getDiskId();
+ if (did != null && did.isPendingAsync()) {
+ done = true;
+ // See if it is pending async because of a faultOut.
+ // If so then if we are not a backup then we can unschedule the pending async.
+ // In either case we need to do the lruFaultIn logic.
+ boolean evicted = ((LRUEntry)entry).testEvicted();
+ if (evicted) {
+ if (!dr.isBackup()) {
+ // @todo do we also need a bit that tells us if it is in the async queue?
+ // Seems like we could end up adding it to the queue multiple times.
+ did.setPendingAsync(false);
+ }
+ // since it was evicted fix the stats here
+ dr.incNumEntriesInVM(1L);
+ dr.incNumOverflowOnDisk(-1L);
+ // no need to dec overflowBytesOnDisk because it was not inced in this case.
+ incrementBucketStats(region, 1/*InVM*/, -1/*OnDisk*/, 0);
+ }
+ lruEntryFaultIn((LRUEntry) entry, region);
+ lruFaultedIn = true;
+ }
+ }
+ }
+ if (!done
+ && (v == null || Token.isRemovedFromDisk(v) && !region.isIndexCreationThread())) {
+ synchronized (entry) {
+ v = entry._getValueRetain(region, true); // TODO:KIRK:OK v = entry.getValueWithContext(region);
+ if (v == null) {
+ v = readValueFromDisk(entry, region);
+ if (entry instanceof LRUEntry) {
+ if (v != null && !Token.isInvalid(v)) {
+ lruEntryFaultIn((LRUEntry) entry, region);
+
+ lruFaultedIn = true;
+ }
+ }
+ }
+ }
+ }
+ } finally {
+ if (!retainResult) {
+ v = OffHeapHelper.copyAndReleaseIfNeeded(v);
+ // At this point v should be either a heap object
+ }
+ }
+ if (Token.isRemoved(v)) {
+ // fix for bug 31800
+ v = null;
+ } else {
+ ((RegionEntry)entry).setRecentlyUsed();
+ }
+ if (lruFaultedIn) {
+ lruUpdateCallback(region);
+ }
+ return v; // OFFHEAP: the value ends up being returned by RegionEntry.getValue
+ }
+
+ public static void recoverValue(DiskEntry entry, long oplogId, DiskRecoveryStore recoveryStore, ByteArrayDataInput in) {
+ boolean lruFaultedIn = false;
+ synchronized (entry) {
+ if (entry.isValueNull()) {
+ DiskId did = entry.getDiskId();
+ if (did != null) {
+ Object value = null;
+ DiskRecoveryStore region = recoveryStore;
+ DiskRegionView dr = region.getDiskRegionView();
+ dr.acquireReadLock();
+ try {
+ synchronized (did) {
+ // don't read if the oplog has changed.
+ if (oplogId == did.getOplogId()) {
+ value = getValueFromDisk(dr, did, in);
+ if (value != null) {
+ setValueOnFaultIn(value, did, entry, dr, region);
+ }
+ }
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ if (entry instanceof LRUEntry) {
+ if (value != null && !Token.isInvalid(value)) {
+ lruEntryFaultIn((LRUEntry) entry, recoveryStore);
+ lruFaultedIn = true;
+ }
+ }
+ }
+ }
+ }
+ if (lruFaultedIn) {
+ lruUpdateCallback(recoveryStore);
+ }
+ }
+
+ /**
+ * Caller must have "did" synced.
+ */
+ private static Object getValueFromDisk(DiskRegionView dr, DiskId did, ByteArrayDataInput in) {
+ Object value;
+ if (dr.isBackup() && did.getKeyId() == DiskRegion.INVALID_ID) {
+ // must have been destroyed
+ value = null;
+ } else {
+ if (did.isKeyIdNegative()) {
+ did.setKeyId(- did.getKeyId());
+ }
+ // if a bucket region then create a CachedDeserializable here instead of object
+ value = dr.getRaw(did); // fix bug 40192
+ if (value instanceof BytesAndBits) {
+ BytesAndBits bb = (BytesAndBits)value;
+ if (EntryBits.isInvalid(bb.getBits())) {
+ value = Token.INVALID;
+ } else if (EntryBits.isLocalInvalid(bb.getBits())) {
+ value = Token.LOCAL_INVALID;
+ } else if (EntryBits.isTombstone(bb.getBits())) {
+ value = Token.TOMBSTONE;
+ } else if (EntryBits.isSerialized(bb.getBits())) {
+ value = readSerializedValue(bb.getBytes(), bb.getVersion(), in, false);
+ } else {
+ value = readRawValue(bb.getBytes(), bb.getVersion(), in);
+ }
+ }
+ }
+ return value;
+ }
+
+ private static void lruUpdateCallback(DiskRecoveryStore recoveryStore) {
+ /*
+ * Used conditional check to see if
+ * if its a LIFO Enabled,
+ * yes then disable lruUpdateCallback()
+ * and called updateStats()
+ * its keep track of actual entries
+ * present in memory - useful when
+ * checking capacity constraint
+ */
+ try {
+ if (recoveryStore.getEvictionAttributes() != null
+ && recoveryStore.getEvictionAttributes().getAlgorithm().isLIFO()) {
+ ((VMLRURegionMap) recoveryStore.getRegionMap()).updateStats();
+ return;
+ }
+ // this must be done after releasing synchronization
+ recoveryStore.getRegionMap().lruUpdateCallback();
+ }catch( DiskAccessException dae) {
+ recoveryStore.handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+
+ private static void lruEntryFaultIn(LRUEntry entry, DiskRecoveryStore recoveryStore) {
+ RegionMap rm = (RegionMap)recoveryStore.getRegionMap();
+ try {
+ rm.lruEntryFaultIn((LRUEntry) entry);
+ }catch(DiskAccessException dae) {
+ recoveryStore.handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+
+ /**
+ * Returns the value of this map entry, reading it from disk, if necessary.
+ * Sets the value in the entry.
+ * This is only called by the faultIn code once it has determined that
+ * the value is no longer in memory.
+ * return the result will only be off-heap if the value is a sqlf ByteSource. Otherwise result will be on-heap.
+ * Caller must have "entry" synced.
+ */
+ @Retained
+ private static Object readValueFromDisk(DiskEntry entry, DiskRecoveryStore region) {
+
+ DiskRegionView dr = region.getDiskRegionView();
+ DiskId did = entry.getDiskId();
+ if (did == null) {
+ return null;
+ }
+ dr.acquireReadLock();
+ try {
+ synchronized (did) {
+ Object value = getValueFromDisk(dr, did, null);
+ if (value == null) return null;
+ @Unretained Object preparedValue = setValueOnFaultIn(value, did, entry, dr, region);
+ // For Sqlfire we want to return the offheap representation.
+ // So we need to retain it for the caller to release.
+ /*if (preparedValue instanceof ByteSource) {
+ // This is the only case in which we return a retained off-heap ref.
+ ((ByteSource)preparedValue).retain();
+ return preparedValue;
+ } else */{
+ return value;
+ }
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ }
+
+ /**
+ * Caller must have "entry" and "did" synced and "dr" readLocked.
+ * @return the unretained result must be used by the caller before it releases the sync on "entry".
+ */
+ @Unretained
+ private static Object setValueOnFaultIn(Object value, DiskId did, DiskEntry entry, DiskRegionView dr, DiskRecoveryStore region) {
+// dr.getOwner().getCache().getLogger().info("DEBUG: faulting in entry with key " + entry.getKey());
+ int bytesOnDisk = getValueLength(did);
+ // Retained by the prepareValueForCache call for the region entry.
+ // NOTE that we return this value unretained because the retain is owned by the region entry not the caller.
+ @Retained Object preparedValue = entry.prepareValueForCache((RegionEntryContext) region, value,
+ false);
+ region.updateSizeOnFaultIn(entry.getKey(), region.calculateValueSize(preparedValue), bytesOnDisk);
+ //did.setValueSerializedSize(0);
+ // I think the following assertion is true but need to run
+ // a regression with it. Reenable this post 6.5
+ //Assert.assertTrue(entry._getValue() == null);
+ entry.setValueWithContext((RegionEntryContext) region, preparedValue);
+ dr.incNumEntriesInVM(1L);
+ dr.incNumOverflowOnDisk(-1L);
+ dr.incNumOverflowBytesOnDisk(-bytesOnDisk);
+ incrementBucketStats(region, 1/*InVM*/, -1/*OnDisk*/, -bytesOnDisk);
+ return preparedValue;
+ }
+
+ static Object readSerializedValue(byte[] valueBytes, Version version,
+ ByteArrayDataInput in, boolean forceDeserialize) {
+ if (forceDeserialize) {
+ // deserialize checking for product version change
+ return EntryEventImpl.deserialize(valueBytes, version, in);
+ }
+ else {
+ // TODO: upgrades: is there a case where GemFire values are internal
+ // ones that need to be upgraded transparently; probably messages
+ // being persisted (gateway events?)
+ return CachedDeserializableFactory.create(valueBytes);
+ }
+ }
+
+ static Object readRawValue(byte[] valueBytes, Version version,
+ ByteArrayDataInput in) {
+ /*
+ final StaticSystemCallbacks sysCb;
+ if (version != null && (sysCb = GemFireCacheImpl.FactoryStatics
+ .systemCallbacks) != null) {
+ // may need to change serialized shape for SQLFire
+ return sysCb.fromVersion(valueBytes, false, version, in);
+ }
+ else */ {
+ return valueBytes;
+ }
+ }
+
+ public static void incrementBucketStats(Object owner,
+ int entriesInVmDelta,
+ int overflowOnDiskDelta,
+ int overflowBytesOnDiskDelta) {
+ if (owner instanceof BucketRegion) {
+ ((BucketRegion)owner).incNumEntriesInVM(entriesInVmDelta);
+ ((BucketRegion)owner).incNumOverflowOnDisk(overflowOnDiskDelta);
+ ((BucketRegion)owner).incNumOverflowBytesOnDisk(overflowBytesOnDiskDelta);
+ } else if (owner instanceof DiskRegionView) {
+ ((DiskRegionView)owner).incNumOverflowBytesOnDisk(overflowBytesOnDiskDelta);
+ }
+ }
+
+ /**
+ * Writes the value of this DiskEntry
to disk and
+ * null
s out the reference to the value to free up VM space.
+ *
+ * Note that if the value had already been written to disk, it is not
+ * written again.
+ *
+ * Caller must synchronize on entry and it is assumed the entry is evicted
+ *
+ * see #writeToDisk
+ * @throws RegionClearedException
+ */
+ public static int overflowToDisk(DiskEntry entry, LocalRegion region, EnableLRU ccHelper) throws RegionClearedException {
+ {
+ Token entryVal = entry.getValueAsToken();
+ if (entryVal == null || Token.isRemovedFromDisk(entryVal)) {
+ // Note it could be removed token now because
+ // freeAllEntriesOnDisk is not able to sync on entry
+ return 0;
+ }
+ }
+ DiskRegion dr = region.getDiskRegion();
+ final int oldSize = region.calculateRegionEntryValueSize(entry);;
+ //Asif:Get diskID . If it is null, it implies it is
+ // overflow only mode.
+ //long id = entry.getDiskId().getKeyId();
+ DiskId did = entry.getDiskId();
+ if (did == null) {
+ ((LRUEntry)entry).setDelayedDiskId(region);
+ did = entry.getDiskId();
+ }
+
+ // Notify the SQLFire IndexManager if present
+ /* final IndexUpdater indexUpdater = region.getIndexUpdater();
+ if(indexUpdater != null && dr.isSync()) {
+ indexUpdater.onOverflowToDisk(entry);
+ }*/
+
+ int change = 0;
+ boolean scheduledAsyncHere = false;
+ dr.acquireReadLock();
+ try {
+ synchronized (did) {
+ // check for a concurrent freeAllEntriesOnDisk
+ if (entry.isRemovedFromDisk()) {
+ return 0;
+ }
+
+ //TODO:Asif: Check if we need to overflow even when id is = 0
+ boolean wasAlreadyPendingAsync = did.isPendingAsync();
+ if (did.needsToBeWritten()) {
+ if (dr.isSync()) {
+ writeToDisk(entry, region, false);
+ } else if (!wasAlreadyPendingAsync) {
+ scheduledAsyncHere = true;
+ did.setPendingAsync(true);
+ } else {
+ // it may have been scheduled to be written (isBackup==true)
+ // and now we are faulting it out
+ }
+ }
+
+ boolean movedValueToDisk = false; // added for bug 41849
+
+ // If async then if it does not need to be written (because it already was)
+ // then treat it like the sync case. This fixes bug 41310
+ if (scheduledAsyncHere || wasAlreadyPendingAsync) {
+ // we call _setValue(null) after it is actually written to disk
+ change = entry.updateAsyncEntrySize(ccHelper);
+ // do the stats when it is actually written to disk
+ } else {
+ region.updateSizeOnEvict(entry.getKey(), oldSize);
+ //did.setValueSerializedSize(byteSizeOnDisk);
+ try {
+ entry.handleValueOverflow(region);
+ entry.setValueWithContext(region,null);
+ }finally {
+ entry.afterValueOverflow(region);
+ }
+ movedValueToDisk = true;
+ change = ((LRUClockNode)entry).updateEntrySize(ccHelper);
+ }
+ int valueLength = 0;
+ if (movedValueToDisk) {
+ valueLength = getValueLength(did);
+ }
+ dr.incNumEntriesInVM(-1L);
+ dr.incNumOverflowOnDisk(1L);
+ dr.incNumOverflowBytesOnDisk(valueLength);
+ incrementBucketStats(region, -1/*InVM*/, 1/*OnDisk*/, valueLength);
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ if (scheduledAsyncHere && did.isPendingAsync()) {
+ // this needs to be done outside the above sync
+ // the version tag is null here because this method only needs
+ // to write to disk for overflow only regions, which do not need
+ // to maintain an RVV on disk.
+ scheduleAsyncWrite(new AsyncDiskEntry(region, entry, null));
+ }
+ return change;
+ }
+
+ private static void scheduleAsyncWrite(AsyncDiskEntry ade) {
+ DiskRegion dr = ade.region.getDiskRegion();
+ dr.scheduleAsyncWrite(ade);
+ }
+
+
+ public static void handleFullAsyncQueue(DiskEntry entry, LocalRegion region, VersionTag tag) {
+ DiskRegion dr = region.getDiskRegion();
+ DiskId did = entry.getDiskId();
+ synchronized (entry) {
+ dr.acquireReadLock();
+ try {
+ synchronized (did) {
+ if (did.isPendingAsync()) {
+ did.setPendingAsync(false);
+ final Token entryVal = entry.getValueAsToken();
+ final int entryValSize = region.calculateRegionEntryValueSize(entry);
+ boolean remove = false;
+ try {
+ if (Token.isRemovedFromDisk(entryVal)) {
+ // onDisk was already deced so just do the valueLength here
+ dr.incNumOverflowBytesOnDisk(-did.getValueLength());
+ incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
+ -did.getValueLength());
+ dr.remove(region, entry, true, false);
+ if (dr.isBackup()) {
+ did.setKeyId(DiskRegion.INVALID_ID); // fix for bug 41340
+ }
+ remove = true;
+ } else if (Token.isInvalid(entryVal) && !dr.isBackup()) {
+ // no need to write invalid to disk if overflow only
+ } else if (entryVal != null) {
+ writeToDisk(entry, region, true);
+ } else {
+ //if we have a version tag we need to record the operation
+ //to update the RVV
+ if(tag != null) {
+ DiskEntry.Helper.doAsyncFlush(tag, region);
+ }
+ return;
+ }
+ assert !dr.isSync();
+ // Only setValue to null if this was an evict.
+ // We could just be a backup that is writing async.
+ if (!remove
+ && !Token.isInvalid(entryVal)
+ && entry instanceof LRUEntry
+ && ((LRUEntry)entry).testEvicted()) {
+ // Moved this here to fix bug 40116.
+ region.updateSizeOnEvict(entry.getKey(), entryValSize);
+ // note the old size was already accounted for
+ // onDisk was already inced so just do the valueLength here
+ dr.incNumOverflowBytesOnDisk(did.getValueLength());
+ incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
+ did.getValueLength());
+ try {
+ entry.handleValueOverflow(region);
+ entry.setValueWithContext(region,null);
+ }finally {
+ entry.afterValueOverflow(region);
+ }
+ }
+
+ //See if we the entry we wrote to disk has the same tag
+ //as this entry. If not, write the tag as a conflicting operation.
+ //to update the RVV.
+ VersionStamp stamp = entry.getVersionStamp();
+ if(tag != null && stamp != null
+ && (stamp.getMemberID() != tag.getMemberID()
+ || stamp.getRegionVersion() != tag.getRegionVersion())) {
+ DiskEntry.Helper.doAsyncFlush(tag, region);
+ }
+ } catch (RegionClearedException ignore) {
+ // no need to do the op since it was clobbered by a region clear
+ }
+ } else {
+ //if we have a version tag we need to record the operation
+ //to update the RVV, even if we don't write the entry
+ if(tag != null) {
+ DiskEntry.Helper.doAsyncFlush(tag, region);
+ }
+ }
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ } // sync entry
+ }
+
+ public static void doAsyncFlush(VersionTag tag, LocalRegion region) {
+ if (region.isThisRegionBeingClosedOrDestroyed()) return;
+ DiskRegion dr = region.getDiskRegion();
+ if (!dr.isBackup()) {
+ return;
+ }
+ assert !dr.isSync();
+ dr.acquireReadLock();
+ try {
+ dr.getDiskStore().putVersionTagOnly(region, tag, true);
+ } finally {
+ dr.releaseReadLock();
+ }
+ }
+
+ /**
+ * Flush an entry that was previously scheduled to be written to disk.
+ * @param tag
+ * @since prPersistSprint1
+ */
+ public static void doAsyncFlush(DiskEntry entry, LocalRegion region, VersionTag tag) {
+ if (region.isThisRegionBeingClosedOrDestroyed()) return;
+ DiskRegion dr = region.getDiskRegion();
+ dr.setClearCountReference();
+ synchronized (entry) { // fixes 40116
+ // If I don't sync the entry and this method ends up doing an eviction
+ // thus setting value to null
+ // some other thread is free to fetch the value while the entry is synced
+ // and think it has removed it or replaced it. This results in updateSizeOn*
+ // being called twice for the same value (once when it is evicted and once
+ // when it is removed/updated).
+ try {
+ dr.acquireReadLock();
+ try {
+ DiskId did = entry.getDiskId();
+ synchronized (did) {
+ if (did.isPendingAsync()) {
+ did.setPendingAsync(false);
+ final Token entryVal = entry.getValueAsToken();
+ final int entryValSize = region.calculateRegionEntryValueSize(entry);
+ boolean remove = false;
+ try {
+ if (Token.isRemovedFromDisk(entryVal)) {
+ if (region.isThisRegionBeingClosedOrDestroyed()) return;
+ // onDisk was already deced so just do the valueLength here
+ dr.incNumOverflowBytesOnDisk(-did.getValueLength());
+ incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
+ -did.getValueLength());
+ dr.remove(region, entry, true, false);
+ if (dr.isBackup()) {
+ did.setKeyId(DiskRegion.INVALID_ID); // fix for bug 41340
+ }
+ remove = true;
+ } else if ((Token.isInvalid(entryVal) || entryVal == Token.TOMBSTONE) && !dr.isBackup()) {
+ // no need to write invalid or tombstones to disk if overflow only
+ } else if (entryVal != null) {
+ writeToDisk(entry, region, true);
+ } else {
+ // @todo why would we have a null value here?
+ // I'm seeing it show up in tests:
+// java.lang.IllegalArgumentException: Must not serialize null in this context.
+// at com.gemstone.gemfire.internal.cache.EntryEventImpl.serialize(EntryEventImpl.java:1024)
+// at com.gemstone.gemfire.internal.cache.DiskEntry$Helper.writeToDisk(DiskEntry.java:351)
+// at com.gemstone.gemfire.internal.cache.DiskEntry$Helper.doAsyncFlush(DiskEntry.java:683)
+// at com.gemstone.gemfire.internal.cache.DiskRegion$FlusherThread.run(DiskRegion.java:1055)
+ //if we have a version tag we need to record the operation
+ //to update the RVV
+ if(tag != null) {
+ DiskEntry.Helper.doAsyncFlush(tag, region);
+ }
+ return;
+ }
+ assert !dr.isSync();
+ // Only setValue to null if this was an evict.
+ // We could just be a backup that is writing async.
+ if (!remove
+ && !Token.isInvalid(entryVal)
+ && (entryVal != Token.TOMBSTONE)
+ && entry instanceof LRUEntry
+ && ((LRUEntry)entry).testEvicted()) {
+ // Moved this here to fix bug 40116.
+ region.updateSizeOnEvict(entry.getKey(), entryValSize);
+ // note the old size was already accounted for
+ // onDisk was already inced so just do the valueLength here
+ dr.incNumOverflowBytesOnDisk(did.getValueLength());
+ incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
+ did.getValueLength());
+ try {
+ entry.handleValueOverflow(region);
+ entry.setValueWithContext(region,null);
+ }finally {
+ entry.afterValueOverflow(region);
+ }
+ }
+ } catch (RegionClearedException ignore) {
+ // no need to do the op since it was clobbered by a region clear
+ }
+
+ //See if we the entry we wrote to disk has the same tag
+ //as this entry. If not, write the tag as a conflicting operation.
+ //to update the RVV.
+ VersionStamp stamp = entry.getVersionStamp();
+ if(tag != null && stamp != null
+ && (stamp.getMemberID() != tag.getMemberID()
+ || stamp.getRegionVersion() != tag.getRegionVersion())) {
+ DiskEntry.Helper.doAsyncFlush(tag, region);
+ }
+ } else {
+ //if we have a version tag we need to record the operation
+ //to update the RVV
+ if(tag != null) {
+ DiskEntry.Helper.doAsyncFlush(tag, region);
+ }
+ }
+ }
+ } finally {
+ dr.releaseReadLock();
+ }
+ } finally {
+ dr.removeClearCountReference();
+ }
+ } // sync entry
+ }
+
+ /**
+ * Removes the key/value pair in the given entry from disk
+ *
+ * @throws RegionClearedException If the operation is aborted due to a clear
+ * @see DiskRegion#remove
+ */
+ public static void removeFromDisk(DiskEntry entry, LocalRegion region, boolean isClear) throws RegionClearedException {
+ removeFromDisk(entry, region, true, false, isClear);
+ }
+ private static void removeFromDisk(DiskEntry entry, LocalRegion region,
+ boolean checkValue, boolean valueWasNull, boolean isClear) throws RegionClearedException {
+ DiskRegion dr = region.getDiskRegion();
+
+ //If we have concurrency checks enabled for a persistent region, we need
+ //to add an entry to the async queue for every update to maintain the RVV
+ boolean maintainRVV = region.concurrencyChecksEnabled && dr.isBackup();
+
+ DiskId did = entry.getDiskId();
+ VersionTag tag = null;
+ Object syncObj = did;
+ if (did == null) {
+ syncObj = entry;
+ }
+ boolean scheduledAsyncHere = false;
+ if (syncObj == did) {
+ dr.acquireReadLock();
+ }
+ try {
+ synchronized (syncObj) {
+
+ if (did == null || (dr.isBackup() && did.getKeyId()== DiskRegion.INVALID_ID)) {
+ // Not on disk yet
+ dr.incNumEntriesInVM(-1L);
+ incrementBucketStats(region, -1/*InVM*/, 0/*OnDisk*/, 0);
+ dr.unscheduleAsyncWrite(did);
+ return;
+ }
+ //Asif: This will convert the -ve OplogKeyId to positive as part of fixing
+ //Bug # 39989
+ did.unmarkForWriting();
+
+ //System.out.println("DEBUG: removeFromDisk doing remove(" + id + ")");
+ int oldValueLength = 0;
+ if (dr.isSync() || isClear) {
+ oldValueLength = did.getValueLength();
+ dr.remove(region, entry, false, isClear);
+ if (dr.isBackup()) {
+ did.setKeyId(DiskRegion.INVALID_ID); // fix for bug 41340
+ }
+ //If this is a clear, we should unschedule the async write for this
+ //entry
+ did.setPendingAsync(false);
+ } else {
+ if (!did.isPendingAsync() || maintainRVV) {
+ scheduledAsyncHere = true;
+ did.setPendingAsync(true);
+ VersionStamp stamp = entry.getVersionStamp();
+ if(stamp != null) {
+ tag = stamp.asVersionTag();
+ }
+ }
+ }
+ if (checkValue) {
+ valueWasNull = entry.isValueNull();
+ entry._removePhase1();
+ }
+ if (valueWasNull) {
+ dr.incNumOverflowOnDisk(-1L);
+ dr.incNumOverflowBytesOnDisk(-oldValueLength);
+ incrementBucketStats(region, 0/*InVM*/, -1/*OnDisk*/, -oldValueLength);
+ }
+ else {
+ dr.incNumEntriesInVM(-1L);
+ incrementBucketStats(region, -1/*InVM*/, 0/*OnDisk*/, 0);
+ if (!dr.isSync()) {
+ // we are going to do an async remove of an entry that is not currently
+ // overflowed to disk so we don't want to count its value length as being
+ // on disk when we finally do the async op. So we clear it here.
+ did.setValueLength(0);
+ }
+ }
+ }
+ } finally {
+ if (syncObj == did) {
+ dr.releaseReadLock();
+ }
+ }
+ if (scheduledAsyncHere && did.isPendingAsync()) {
+ // do this outside the sync
+ scheduleAsyncWrite(new AsyncDiskEntry(region, entry, tag));
+ }
+ }
+
+ /**
+ * @param entry
+ * @param region
+ * @param tag
+ */
+ public static void updateVersionOnly(DiskEntry entry, LocalRegion region,
+ VersionTag tag) {
+ DiskRegion dr = region.getDiskRegion();
+ if (!dr.isBackup()) {
+ return;
+ }
+
+ assert tag != null && tag.getMemberID()!=null;
+ boolean scheduleAsync = false;
+ DiskId did = entry.getDiskId();
+ Object syncObj = did;
+ if (syncObj == null) {
+ syncObj = entry;
+ }
+ if (syncObj == did) {
+ dr.acquireReadLock();
+ }
+ try {
+ synchronized (syncObj) {
+ if (dr.isSync()) {
+ dr.getDiskStore().putVersionTagOnly(region, tag, false);
+ } else {
+ scheduleAsync = true;
+ }
+ }
+ } finally {
+ if (syncObj == did) {
+ dr.releaseReadLock();
+ }
+ }
+ if (scheduleAsync) {
+ // this needs to be done outside the above sync
+ scheduleAsyncWrite(new AsyncDiskEntry(region, tag));
+ }
+ }
+
+ }
+
+ /**
+ * A marker object for an entry that has been recovered from disk.
+ * It is handled specially when it is placed in a region.
+ */
+ public static class RecoveredEntry {
+
+ /** The disk id of the entry being recovered */
+ private final long recoveredKeyId;
+
+ /** The value of the recovered entry */
+ private final Object value;
+
+ private final long offsetInOplog;
+ private final byte userBits;
+ private final int valueLength;
+
+ private long oplogId;
+ private VersionTag tag;
+
+ /**
+ * Only for this constructor, the value is not loaded into the region & it is lying
+ * on the oplogs. Since Oplogs rely on DiskId to furnish user bits so as to correctly
+ * interpret bytes, the userbit needs to be set correctly here.
+ */
+ public RecoveredEntry(long keyId, long oplogId, long offsetInOplog,
+ byte userBits, int valueLength) {
+ this(-keyId, oplogId, offsetInOplog, userBits, valueLength, null);
+ }
+
+ public RecoveredEntry(long keyId, long oplogId, long offsetInOplog,
+ byte userBits, int valueLength, Object value) {
+ this.recoveredKeyId = keyId;
+ this.value = value;
+ this.oplogId = oplogId;
+ this.offsetInOplog = offsetInOplog;
+ this.userBits = EntryBits.setRecoveredFromDisk(userBits, true);
+ this.valueLength = valueLength;
+ }
+
+ /**
+ * Returns the disk id of the entry being recovered
+ */
+ public long getRecoveredKeyId() {
+ return this.recoveredKeyId;
+ }
+ /**
+ * Returns the value of the recovered entry. Note that if the
+ * disk id is < 0 then the value has not been faulted in and
+ * this method will return null.
+ */
+ public Object getValue() {
+ return this.value;
+ }
+ /**
+ *
+ * @return byte indicating the user bits. The correct value is returned only in the specific case of
+ * entry recovered from oplog ( & not rolled to Htree) & the RECOVER_VALUES flag is false . In other cases
+ * the exact value is not needed
+ */
+ public byte getUserBits() {
+ return this.userBits;
+ }
+ public int getValueLength() {
+ return this.valueLength;
+ }
+ public long getOffsetInOplog() {
+ return offsetInOplog;
+ }
+ public long getOplogId() {
+ return this.oplogId;
+ }
+
+ public void setOplogId(long v) {
+ this.oplogId = v;
+ }
+ public VersionTag getVersionTag() {
+ return this.tag;
+ }
+ public void setVersionTag(VersionTag tag) {
+ this.tag = tag;
+ }
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
new file mode 100644
index 000000000000..fcfc2f333cf2
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -0,0 +1,4309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ABSTRACT_REGION_ENTRY_FILL_IN_VALUE;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.InvalidDeltaException;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.cache.CacheListener;
+import com.gemstone.gemfire.cache.CacheLoader;
+import com.gemstone.gemfire.cache.CacheLoaderException;
+import com.gemstone.gemfire.cache.CacheWriter;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskAccessException;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.LossAction;
+import com.gemstone.gemfire.cache.MembershipAttributes;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.RegionAccessException;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.RegionDistributionException;
+import com.gemstone.gemfire.cache.RegionMembershipListener;
+import com.gemstone.gemfire.cache.ResumptionAction;
+import com.gemstone.gemfire.cache.RoleException;
+import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.persistence.PersistentReplicatesOfflineException;
+import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
+import com.gemstone.gemfire.cache.wan.GatewaySender;
+import com.gemstone.gemfire.distributed.DistributedLockService;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.LockServiceDestroyedException;
+import com.gemstone.gemfire.distributed.Role;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisee;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.Profile;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.ProfileVisitor;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.MembershipListener;
+import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
+import com.gemstone.gemfire.distributed.internal.locks.DLockRemoteToken;
+import com.gemstone.gemfire.distributed.internal.locks.DLockService;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor.CacheProfile;
+import com.gemstone.gemfire.internal.cache.InitialImageOperation.GIIStatus;
+import com.gemstone.gemfire.internal.cache.RemoteFetchVersionMessage.FetchVersionResponse;
+import com.gemstone.gemfire.internal.cache.control.InternalResourceManager.ResourceType;
+import com.gemstone.gemfire.internal.cache.control.MemoryEvent;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionExecutor;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionResultSender;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionResultWaiter;
+import com.gemstone.gemfire.internal.cache.execute.FunctionStats;
+import com.gemstone.gemfire.internal.cache.execute.LocalResultCollector;
+import com.gemstone.gemfire.internal.cache.execute.RegionFunctionContextImpl;
+import com.gemstone.gemfire.internal.cache.execute.ServerToClientFunctionResultSender;
+import com.gemstone.gemfire.internal.cache.lru.LRUEntry;
+import com.gemstone.gemfire.internal.cache.persistence.CreatePersistentRegionProcessor;
+import com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisor;
+import com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisorImpl;
+import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
+import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberManager;
+import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberView;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tier.sockets.VersionedObjectList;
+import com.gemstone.gemfire.internal.cache.versions.ConcurrentCacheModificationException;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
+import com.gemstone.gemfire.internal.cache.versions.VersionSource;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
+import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySenderEventProcessor;
+import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueConfigurationException;
+import com.gemstone.gemfire.internal.cache.wan.GatewaySenderConfigurationException;
+import com.gemstone.gemfire.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderQueue;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.annotations.Released;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.sequencelog.RegionLogger;
+import com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch;
+import com.gemstone.gemfire.i18n.StringId;
+/**
+ *
+ */
+@SuppressWarnings("deprecation")
+public class DistributedRegion extends LocalRegion implements
+ CacheDistributionAdvisee
+{
+ private static final Logger logger = LogService.getLogger();
+
+ /** causes cache profile to be added to afterRemoteRegionCreate notification for testing */
+ public static boolean TEST_HOOK_ADD_PROFILE = false;
+
+ /** Used to sync accesses to this.dlockService to allow lazy construction */
+ private final Object dlockMonitor = new Object();
+
+ final CacheDistributionAdvisor distAdvisor;
+
+ /**
+ * @guarded.By {@link #dlockMonitor}
+ */
+ private DistributedLockService dlockService;
+
+ protected final AdvisorListener advisorListener = new AdvisorListener();
+
+ /** Set of currently missing required roles */
+ protected final HashSet missingRequiredRoles = new HashSet();
+
+ /** True if this region is currently missing any required roles */
+ protected volatile boolean isMissingRequiredRoles = false;
+
+ /**
+ * True if this region is has any required roles defined and the LossAction is
+ * either NO_ACCESS or LIMITED_ACCESS. Reliability checks will only happen if
+ * this is true.
+ */
+ private final boolean requiresReliabilityCheck;
+
+ /**
+ * Provides a queue for reliable message delivery
+ *
+ * @since 5.0
+ */
+ protected final ReliableMessageQueue rmq;
+
+ /**
+ * Latch that is opened after initialization waits for required roles up to
+ * the member-timeout .
+ */
+ protected final StoppableCountDownLatch initializationLatchAfterMemberTimeout;
+
+ private final PersistenceAdvisor persistenceAdvisor;
+
+ private final PersistentMemberID persistentId;
+
+ /**
+ * This boolean is set to false when this region
+ * is non-persistent, but there are persistent members in the distributed system
+ * to which all region modifications should be forwarded
+ * see bug 45186
+ */
+ private volatile boolean generateVersionTag = true;
+
+ /** Tests can set this to true and ignore reliability triggered reconnects */
+ public static boolean ignoreReconnect = false;
+
+ /**
+ * Lock to prevent multiple threads on this member from performing
+ * a clear at the same time.
+ */
+ private final Object clearLock = new Object();
+
+ private static AtomicBoolean loggedNetworkPartitionWarning = new AtomicBoolean(false);
+
+ /** Creates a new instance of DistributedRegion */
+ protected DistributedRegion(String regionName, RegionAttributes attrs,
+ LocalRegion parentRegion, GemFireCacheImpl cache,
+ InternalRegionArguments internalRegionArgs) {
+ super(regionName, attrs, parentRegion, cache, internalRegionArgs);
+ this.initializationLatchAfterMemberTimeout = new StoppableCountDownLatch(
+ getCancelCriterion(), 1);
+ this.distAdvisor = createDistributionAdvisor(internalRegionArgs);
+
+ if (getDistributionManager().getConfig().getEnableNetworkPartitionDetection()
+ && !isInternalRegion() && !attrs.getScope().isAck() && !doesNotDistribute() && attrs.getDataPolicy().withStorage()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributedRegion_REGION_0_1_SPLITBRAIN_CONFIG_WARNING,
+ new Object[] { regionName, attrs.getScope() }));
+ }
+ if (!getDistributionManager().getConfig().getEnableNetworkPartitionDetection()
+ && attrs.getDataPolicy().withPersistence() && !loggedNetworkPartitionWarning.getAndSet(true)) {
+ logger.warn(LocalizedMessage.create(
+ LocalizedStrings.DistributedRegion_REGION_0_ENABLE_NETWORK_PARTITION_WARNING,
+ new Object[] { regionName, attrs.getScope() }));
+ }
+
+ boolean setRequiresReliabilityCheck = attrs.getMembershipAttributes()
+ .hasRequiredRoles()
+ &&
+ // note that the following includes NO_ACCESS, LIMITED_ACCESS,
+ !attrs.getMembershipAttributes().getLossAction().isAllAccess()
+ && !attrs.getMembershipAttributes().getLossAction().isReconnect();
+
+ // this optimization is safe for as long as Roles and Required Roles are
+ // immutable
+ // if this VM fulfills all required roles, make requiresReliabilityCheck
+ // false
+ Set reqRoles = new HashSet(attrs.getMembershipAttributes()
+ .getRequiredRoles());
+ reqRoles.removeAll(getSystem().getDistributedMember().getRoles());
+ if (reqRoles.isEmpty()) {
+ setRequiresReliabilityCheck = false;
+ }
+
+ this.requiresReliabilityCheck = setRequiresReliabilityCheck;
+
+ {
+ ReliableMessageQueue tmp = null;
+ if (this.requiresReliabilityCheck) {
+ // if
+ // (attrs.getMembershipAttributes().getLossAction().isAllAccessWithQueuing())
+ // {
+ // tmp = cache.getReliableMessageQueueFactory().create(this);
+ // }
+ }
+ this.rmq = tmp;
+ }
+
+ if(internalRegionArgs.isUsedForPartitionedRegionBucket()) {
+ this.persistenceAdvisor = internalRegionArgs.getPersistenceAdvisor();
+ } else if (this.allowsPersistence()){
+ //TODO prpersist - using this lock service is a hack. Maybe? Or maybe
+ //it's ok if we have one (rarely used) lock service for many operations?
+ //What does the resource manager do?
+ DistributedLockService dl = cache.getPartitionedRegionLockService();
+ try {
+ //TODO prpersist - this is just a quick and dirty storage mechanism so that
+ //I can test the storage.
+ DiskRegionStats diskStats;
+ PersistentMemberView storage;
+ if(getDataPolicy().withPersistence()) {
+ storage = getDiskRegion();
+ diskStats = getDiskRegion().getStats();
+ } else {
+ storage = new InMemoryPersistentMemberView();
+ diskStats = null;
+ }
+ PersistentMemberManager memberManager = cache.getPersistentMemberManager();
+ this.persistenceAdvisor = new PersistenceAdvisorImpl(distAdvisor, dl, storage, this.getFullPath(), diskStats, memberManager);
+ } catch (Exception e) {
+ throw new InternalGemFireError("Couldn't recover persistence");
+ }
+ } else {
+ this.persistenceAdvisor = null;
+ }
+ if(this.persistenceAdvisor != null) {
+ this.persistentId = persistenceAdvisor.generatePersistentID();
+ } else {
+ this.persistentId = null;
+ }
+
+ }
+
+ @Override
+ public void createEventTracker() {
+ this.eventTracker = new EventTracker(this);
+ this.eventTracker.start();
+ }
+
+ /**
+ * Intended for used during construction of a DistributedRegion
+ *
+ * @return the advisor to be used by the region
+ */
+ protected CacheDistributionAdvisor createDistributionAdvisor(InternalRegionArguments internalRegionArgs) {
+ return CacheDistributionAdvisor.createCacheDistributionAdvisor(this); // Warning: potential early escape of object before full construction
+ }
+
+ /**
+ * Does this region support persistence?
+ */
+ public boolean allowsPersistence() {
+ return true;
+ }
+
+ @Override
+ public boolean requiresOneHopForMissingEntry(EntryEventImpl event) {
+ // received from another member - don't use one-hop
+ if (event.isOriginRemote()) {
+ return false;
+ }
+ // local ops aren't distributed
+ if (event.getOperation().isLocal()) {
+ return false;
+ }
+ // if it already has a valid version tag it can go out with a DistributedCacheOperation
+ if (event.getVersionTag() != null && event.getVersionTag().getRegionVersion() > 0) {
+ return false;
+ }
+ // if we're not allowed to generate a version tag we need to send it to someone who can
+ if (!this.generateVersionTag) {
+ return true;
+ }
+ return this.concurrencyChecksEnabled &&
+ (this.srp == null) &&
+ !isTX() &&
+ this.scope.isDistributed() &&
+ !this.dataPolicy.withReplication();
+ }
+
+
+ /**
+ * @see LocalRegion#virtualPut(EntryEventImpl, boolean, boolean, Object,
+ * boolean, long, boolean)
+ */
+ @Override
+ protected
+ boolean virtualPut(EntryEventImpl event,
+ boolean ifNew,
+ boolean ifOld,
+ Object expectedOldValue,
+ boolean requireOldValue,
+ long lastModified,
+ boolean overwriteDestroyed)
+ throws TimeoutException,
+ CacheWriterException {
+ final boolean isTraceEnabled = logger.isTraceEnabled();
+
+ Lock dlock = null;
+ if (this.scope.isGlobal() && // lock only applies to global scope
+ !event.isOriginRemote() && // only if operation originating locally
+ !event.isNetSearch() && // search and load processor handles own locking
+ !event.isNetLoad() &&
+ // @todo darrel/kirk: what about putAll?
+ !event.isLocalLoad() &&
+ !event.isSingleHopPutOp()) { // Single Hop Op means dlock is already taken at origin node.
+ dlock = this.getDistributedLockIfGlobal(event.getKey());
+ }
+ if (isTraceEnabled) {
+ logger.trace("virtualPut invoked for event {}", event);
+ }
+ try {
+ if (!hasSeenEvent(event)) {
+ if (this.requiresOneHopForMissingEntry(event)) {
+ // bug #45704: see if a one-hop must be done for this operation
+ RegionEntry re = getRegionEntry(event.getKey());
+ if (re == null /*|| re.isTombstone()*/ || !this.generateVersionTag) {
+ if (!event.isBulkOpInProgress() || this.dataPolicy.withStorage()) {
+ // putAll will send a single one-hop for empty regions. for other missing entries
+ // we need to get a valid version number before modifying the local cache
+ boolean didDistribute = RemotePutMessage.distribute(event, lastModified,
+ false, false, expectedOldValue, requireOldValue, !this.generateVersionTag);
+
+ if (!didDistribute && isTraceEnabled) {
+ logger.trace("Unable to perform one-hop messaging");
+ }
+ if (!this.generateVersionTag && !didDistribute) {
+ throw new PersistentReplicatesOfflineException();
+ }
+ if (didDistribute) {
+ if (isTraceEnabled) {
+ logger.trace("Event after remotePut operation: {}", event);
+ }
+ if (event.getVersionTag() == null) {
+ // if the event wasn't applied by the one-hop replicate it will not have a version tag
+ // and so should not be applied to this cache
+ return false;
+ }
+ }
+ }
+ }
+ }
+ return super.virtualPut(event,
+ ifNew,
+ ifOld,
+ expectedOldValue,
+ requireOldValue,
+ lastModified,
+ overwriteDestroyed);
+ }
+ else {
+ if (event.getDeltaBytes() != null && event.getRawNewValue() == null) {
+ // This means that this event has delta bytes but no full value.
+ // Request the full value of this event.
+ // The value in this vm may not be same as this event's value.
+ throw new InvalidDeltaException(
+ "Cache encountered replay of event containing delta bytes for key "
+ + event.getKey());
+ }
+ // if the listeners have already seen this event, then it has already
+ // been successfully applied to the cache. Distributed messages and
+ // return
+ if (isTraceEnabled) {
+ logger.trace("DR.virtualPut: this cache has already seen this event {}", event);
+ }
+
+ // Gester, Fix 39014: when hasSeenEvent, put will still distribute
+ // event, but putAll did not. We add the logic back here, not to put
+ // back into DR.distributeUpdate() because we moved this part up into
+ // LR.basicPutPart3 in purpose. Reviewed by Bruce.
+ if (event.isBulkOpInProgress() && !event.isOriginRemote()) {
+ event.getPutAllOperation().addEntry(event, true);
+ }
+
+ /* doing this so that other VMs will apply this no matter what. If it
+ * is an "update" they will not apply it if they don't have the key.
+ * Because this is probably a retry, it will never get applied to this
+ * local AbstractRegionMap, and so will never be flipped to a 'create'
+ */
+ event.makeCreate();
+ if (!getConcurrencyChecksEnabled() || event.hasValidVersionTag()) {
+ distributeUpdate(event, lastModified, ifNew, ifOld, expectedOldValue, requireOldValue);
+ event.invokeCallbacks(this,true, true);
+ }
+ return true;
+ }
+ }
+ finally {
+ if (dlock != null) {
+ dlock.unlock();
+ }
+ }
+ }
+
+ @Override
+ protected RegionEntry basicPutEntry(EntryEventImpl event, long lastModified)
+ throws TimeoutException, CacheWriterException {
+
+ final boolean isTraceEnabled = logger.isTraceEnabled();
+
+ if (isTraceEnabled) {
+ logger.trace("basicPutEntry invoked for event {}", event);
+ }
+ if (this.requiresOneHopForMissingEntry(event)) {
+ // bug #45704: see if a one-hop must be done for this operation
+ RegionEntry re = getRegionEntry(event.getKey());
+ if (re == null /*|| re.isTombstone()*/ || !this.generateVersionTag) {
+ final boolean ifNew = false;
+ final boolean ifOld = false;
+ boolean didDistribute = RemotePutMessage.distribute(event, lastModified,
+ ifNew, ifOld, null, false, !this.generateVersionTag);
+ if (!this.generateVersionTag && !didDistribute) {
+ throw new PersistentReplicatesOfflineException();
+ }
+ if (didDistribute && isTraceEnabled) {
+ logger.trace("Event after remotePut for basicPutEntry: {}", event);
+ }
+ }
+ }
+ return super.basicPutEntry(event, lastModified);
+ }
+
+ @Override
+ public void performPutAllEntry(EntryEventImpl event) {
+ /*
+ * force shared data view so that we just do the virtual op, accruing things in the put all operation for later
+ */
+ if(isTX()) {
+ event.getPutAllOperation().addEntry(event);
+ } else {
+ getSharedDataView().putEntry(event, false, false, null, false, 0L, false);
+ }
+ }
+
+ @Override
+ public void performRemoveAllEntry(EntryEventImpl event) {
+ // force shared data view so that we just do the virtual op, accruing things in the bulk operation for later
+ if(isTX()) {
+ event.getRemoveAllOperation().addEntry(event);
+ } else {
+ basicDestroy(event, true, null);
+ //getSharedDataView().destroyExistingEntry(event, true, null);
+ }
+ }
+
+ /**
+ * distribution and listener notification
+ */
+ @Override
+ public void basicPutPart3(EntryEventImpl event, RegionEntry entry,
+ boolean isInitialized, long lastModified, boolean invokeCallbacks,
+ boolean ifNew, boolean ifOld, Object expectedOldValue,
+ boolean requireOldValue) {
+
+ distributeUpdate(event, lastModified, false, false, null, false);
+ super.basicPutPart3(event, entry, isInitialized, lastModified,
+ invokeCallbacks, ifNew, ifOld, expectedOldValue, requireOldValue);
+ }
+
+ /** distribute an update operation */
+ protected void distributeUpdate(EntryEventImpl event, long lastModified, boolean ifNew, boolean ifOld, Object expectedOldValue, boolean requireOldValue) {
+ // an update from a netSearch is not distributed
+ if (!event.isOriginRemote() && !event.isNetSearch() && !event.isBulkOpInProgress()) {
+ boolean distribute = true;
+ if (event.getInhibitDistribution()) {
+ // this has already been distributed by a one-hop operation
+ distribute = false;
+ }
+ if (distribute) {
+ UpdateOperation op = new UpdateOperation(event, lastModified);
+ if (logger.isTraceEnabled()) {
+ logger.trace("distributing operation for event : {} : for region : {}", event, this.getName());
+ }
+ op.distribute();
+ }
+ }
+ }
+
+ protected void setGeneratedVersionTag(boolean generateVersionTag) {
+ // there is at-least one other persistent member, so turn on concurrencyChecks
+ enableConcurrencyChecks();
+
+ this.generateVersionTag = generateVersionTag;
+ }
+
+ protected boolean getGenerateVersionTag() {
+ return this.generateVersionTag;
+ }
+
+ @Override
+ protected boolean shouldGenerateVersionTag(RegionEntry entry, EntryEventImpl event) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("shouldGenerateVersionTag this.generateVersionTag={} ccenabled={} dataPolicy={} event:{}",
+ this.generateVersionTag, this.concurrencyChecksEnabled, this.dataPolicy, event);
+ }
+ if (!this.concurrencyChecksEnabled || this.dataPolicy == DataPolicy.EMPTY || !this.generateVersionTag) {
+ return false;
+ }
+ if (this.srp != null) { // client
+ return false;
+ }
+ if (event.getVersionTag() != null && !event.getVersionTag().isGatewayTag()) {
+ return false;
+ }
+ if (event.getOperation().isLocal()) { // bug #45402 - localDestroy generated a version tag
+ return false;
+ }
+ if (!event.isOriginRemote() && this.dataPolicy.withReplication()) {
+ return true;
+ }
+ if (!this.dataPolicy.withReplication() && !this.dataPolicy.withPersistence()) {
+ if (!entry.getVersionStamp().hasValidVersion()) {
+ // do not generate a version stamp in a region that has no replication if it's not based
+ // on an existing version from a replicate region
+ return false;
+ }
+ return true;
+ }
+ if (!event.isOriginRemote() && event.getDistributedMember() != null) {
+ if (!event.getDistributedMember().equals(this.getMyId())) {
+ return event.getVersionTag() == null; // one-hop remote message
+ }
+ }
+ return false;
+ }
+ /**
+ * Throws RegionAccessException if required roles are missing and the
+ * LossAction is NO_ACCESS
+ *
+ * @throws RegionAccessException
+ * if required roles are missing and the LossAction is NO_ACCESS
+ */
+ @Override
+ protected void checkForNoAccess()
+ {
+ if (this.requiresReliabilityCheck && this.isMissingRequiredRoles) {
+ if (getMembershipAttributes().getLossAction().isNoAccess()) {
+ synchronized (this.missingRequiredRoles) {
+ if (!this.isMissingRequiredRoles)
+ return;
+ Set roles = Collections.unmodifiableSet(new HashSet(
+ this.missingRequiredRoles));
+ throw new RegionAccessException(LocalizedStrings.DistributedRegion_OPERATION_IS_DISALLOWED_BY_LOSSACTION_0_BECAUSE_THESE_REQUIRED_ROLES_ARE_MISSING_1.toLocalizedString(new Object[] {getMembershipAttributes().getLossAction(), roles}), getFullPath(), roles);
+ }
+ }
+ }
+ }
+
+ /**
+ * Throws RegionAccessException is required roles are missing and the
+ * LossAction is either NO_ACCESS or LIMITED_ACCESS.
+ *
+ * @throws RegionAccessException
+ * if required roles are missing and the LossAction is either
+ * NO_ACCESS or LIMITED_ACCESS
+ */
+ @Override
+ protected void checkForLimitedOrNoAccess()
+ {
+ if (this.requiresReliabilityCheck && this.isMissingRequiredRoles) {
+ if (getMembershipAttributes().getLossAction().isNoAccess()
+ || getMembershipAttributes().getLossAction().isLimitedAccess()) {
+ synchronized (this.missingRequiredRoles) {
+ if (!this.isMissingRequiredRoles)
+ return;
+ Set roles = Collections.unmodifiableSet(new HashSet(
+ this.missingRequiredRoles));
+ Assert.assertTrue(!roles.isEmpty());
+ throw new RegionAccessException(LocalizedStrings.DistributedRegion_OPERATION_IS_DISALLOWED_BY_LOSSACTION_0_BECAUSE_THESE_REQUIRED_ROLES_ARE_MISSING_1
+ .toLocalizedString(new Object[] { getMembershipAttributes().getLossAction(), roles}), getFullPath(), roles);
+ }
+ }
+ }
+ }
+
+ @Override
+ protected void handleReliableDistribution(ReliableDistributionData data,
+ Set successfulRecipients) {
+ handleReliableDistribution(data, successfulRecipients,
+ Collections.EMPTY_SET, Collections.EMPTY_SET);
+ }
+
+ protected void handleReliableDistribution(ReliableDistributionData data,
+ Set successfulRecipients, Set otherRecipients1, Set otherRecipients2)
+ {
+ if (this.requiresReliabilityCheck) {
+ MembershipAttributes ra = getMembershipAttributes();
+ Set recipients = successfulRecipients;
+ // determine the successful roles
+ Set roles = new HashSet();
+ for (Iterator iter = recipients.iterator(); iter.hasNext();) {
+ InternalDistributedMember mbr = (InternalDistributedMember)iter.next();
+ if (mbr != null) {
+ roles.addAll(mbr.getRoles());
+ }
+ }
+ for (Iterator iter = otherRecipients1.iterator(); iter.hasNext();) {
+ InternalDistributedMember mbr = (InternalDistributedMember)iter.next();
+ if (mbr != null) {
+ roles.addAll(mbr.getRoles());
+ }
+ }
+ for (Iterator iter = otherRecipients2.iterator(); iter.hasNext();) {
+ InternalDistributedMember mbr = (InternalDistributedMember)iter.next();
+ if (mbr != null) {
+ roles.addAll(mbr.getRoles());
+ }
+ }
+ // determine the missing roles
+ Set failedRoles = new HashSet(ra.getRequiredRoles());
+ failedRoles.removeAll(roles);
+ if (failedRoles.isEmpty())
+ return;
+// if (rp.isAllAccessWithQueuing()) {
+// this.rmq.add(data, failedRoles);
+// } else {
+
+ throw new RegionDistributionException(LocalizedStrings.DistributedRegion_OPERATION_DISTRIBUTION_MAY_HAVE_FAILED_TO_NOTIFY_THESE_REQUIRED_ROLES_0.toLocalizedString(failedRoles), getFullPath(), failedRoles);
+// }
+ }
+ }
+
+ /**
+ *
+ * Called when we do a distributed operation and don't have anyone to
+ * distributed it too. Since this is only called when no distribution was done
+ * (i.e. no recipients) we do not check isMissingRequiredRoles because it
+ * might not longer be true due to race conditions
+ *
+ * @return false if this region has at least one required role and queuing is
+ * configured. Returns true if sending to no one is ok.
+ * @throws RoleException
+ * if a required role is missing and the LossAction is either
+ * NO_ACCESS or LIMITED_ACCESS.
+ * @since 5.0
+ */
+ protected boolean isNoDistributionOk()
+ {
+ if (this.requiresReliabilityCheck) {
+ MembershipAttributes ra = getMembershipAttributes();
+ // if (ra.getLossAction().isAllAccessWithQueuing()) {
+ // return !ra.hasRequiredRoles();
+ // } else {
+ Set failedRoles = ra.getRequiredRoles();
+ throw new RegionDistributionException(LocalizedStrings.DistributedRegion_OPERATION_DISTRIBUTION_WAS_NOT_DONE_TO_THESE_REQUIRED_ROLES_0.toLocalizedString(failedRoles), getFullPath(), failedRoles);
+ // }
+ }
+ return true;
+ }
+
+ /**
+ * returns true if this Region does not distribute its operations to other
+ * members.
+ * @since 6.0
+ * @see HARegion#localDestroyNoCallbacks(Object)
+ */
+ public boolean doesNotDistribute() {
+ return false;
+ }
+
+
+ @Override
+ public boolean shouldSyncForCrashedMember(InternalDistributedMember id) {
+ return !doesNotDistribute() && super.shouldSyncForCrashedMember(id);
+ }
+
+
+ /**
+ * Adjust the specified set of recipients by removing any of them that are
+ * currently having their data queued.
+ *
+ * @param recipients
+ * the set of recipients that a message is to be distributed too.
+ * Recipients that are currently having their data queued will be
+ * removed from this set.
+ * @return the set, possibly null, of recipients that are currently having
+ * their data queued.
+ * @since 5.0
+ */
+ protected Set adjustForQueuing(Set recipients)
+ {
+ Set result = null;
+ // if (this.requiresReliabilityCheck) {
+ // MembershipAttributes ra = getMembershipAttributes();
+ // if (ra.getLossAction().isAllAccessWithQueuing()) {
+ // Set currentQueuedRoles = this.rmq.getQueuingRoles();
+ // if (currentQueuedRoles != null) {
+ // // foreach recipient see if any of his roles are queued and if
+ // // they are remove him from recipients and add him to result
+ // Iterator it = recipients.iterator();
+ // while (it.hasNext()) {
+ // DistributedMember dm = (DistributedMember)it.next();
+ // Set dmRoles = dm.getRoles();
+ // if (!dmRoles.isEmpty()) {
+ // if (intersects(dmRoles, currentQueuedRoles)) {
+ // it.remove(); // fix for bug 34447
+ // if (result == null) {
+ // result = new HashSet();
+ // }
+ // result.add(dm);
+ // }
+ // }
+ // }
+ // }
+ // }
+ // }
+ return result;
+ }
+
+ /**
+ * Returns true if the two sets intersect
+ *
+ * @param a
+ * a non-null non-empty set
+ * @param b
+ * a non-null non-empty set
+ * @return true if sets a and b intersect; false if not
+ * @since 5.0
+ */
+ public static boolean intersects(Set a, Set b)
+ {
+ Iterator it;
+ Set target;
+ if (a.size() <= b.size()) {
+ it = a.iterator();
+ target = b;
+ }
+ else {
+ it = b.iterator();
+ target = a;
+ }
+ while (it.hasNext()) {
+ if (target.contains(it.next()))
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public boolean requiresReliabilityCheck()
+ {
+ return this.requiresReliabilityCheck;
+ }
+
+ /**
+ * Returns true if the ExpiryTask is currently allowed to expire.
+ *
+ * If the region is in NO_ACCESS due to reliability configuration, then no
+ * expiration actions are allowed.
+ *
+ * If the region is in LIMITED_ACCESS due to reliability configuration, then
+ * only non-distributed expiration actions are allowed.
+ */
+ @Override
+ protected boolean isExpirationAllowed(ExpiryTask expiry)
+ {
+ if (this.requiresReliabilityCheck && this.isMissingRequiredRoles) {
+ if (getMembershipAttributes().getLossAction().isNoAccess()) {
+ return false;
+ }
+ if (getMembershipAttributes().getLossAction().isLimitedAccess()
+ && expiry.isDistributedAction()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Performs the resumption action when reliability is resumed.
+ *
+ * @return true if asynchronous resumption is triggered
+ */
+ protected boolean resumeReliability(InternalDistributedMember id,
+ Set newlyAcquiredRoles)
+ {
+ boolean async = false;
+ try {
+ ResumptionAction ra = getMembershipAttributes().getResumptionAction();
+ if (ra.isNone()) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Reliability resumption for action of none");
+ }
+ resumeExpiration();
+ }
+ else if (ra.isReinitialize()) {
+ async = true;
+ asyncResumeReliability(id, newlyAcquiredRoles);
+ }
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_UNEXPECTED_EXCEPTION), e);
+ }
+ return async;
+ }
+
+ /**
+ * Handles asynchronous ResumptionActions such as region reinitialize.
+ */
+ private void asyncResumeReliability(final InternalDistributedMember id,
+ final Set newlyAcquiredRoles)
+ throws RejectedExecutionException {
+ final ResumptionAction ra = getMembershipAttributes().getResumptionAction();
+ getDistributionManager().getWaitingThreadPool().execute(new Runnable() {
+ public void run()
+ {
+ try {
+ if (ra.isReinitialize()) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Reliability resumption for action of reinitialize");
+ }
+ if (!isDestroyed() && !cache.isClosed()) {
+ RegionEventImpl event = new RegionEventImpl(
+ DistributedRegion.this, Operation.REGION_REINITIALIZE, null,
+ false, getMyId(), generateEventID());
+ reinitialize(null, event);
+ }
+ synchronized (missingRequiredRoles) {
+ // any number of threads may be waiting on missingRequiredRoles
+ missingRequiredRoles.notifyAll();
+ if (hasListener() && id != null) {
+ // fire afterRoleGain event
+ RoleEventImpl relEvent = new RoleEventImpl(
+ DistributedRegion.this, Operation.REGION_CREATE, null,
+ true, id, newlyAcquiredRoles);
+ dispatchListenerEvent(EnumListenerEvent.AFTER_ROLE_GAIN,
+ relEvent);
+ }
+ }
+ }
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_UNEXPECTED_EXCEPTION), e);
+ }
+ }
+ });
+ }
+
+ /** Reschedules expiry tasks when reliability is resumed. */
+ private void resumeExpiration()
+ {
+ boolean isNoAccess = getMembershipAttributes().getLossAction().isNoAccess();
+ boolean isLimitedAccess = getMembershipAttributes().getLossAction()
+ .isLimitedAccess();
+ if (!(isNoAccess || isLimitedAccess)) {
+ return; // early out: expiration was never affected by reliability
+ }
+
+ if (getEntryTimeToLive().getTimeout() > 0
+ && (isNoAccess || (isLimitedAccess && getEntryTimeToLive().getAction()
+ .isDistributed()))) {
+ rescheduleEntryExpiryTasks();
+ }
+ else
+ if (getEntryIdleTimeout().getTimeout() > 0
+ && (isNoAccess || (isLimitedAccess && getEntryIdleTimeout().getAction()
+ .isDistributed()))) {
+ rescheduleEntryExpiryTasks();
+ }
+ else
+ if (getCustomEntryTimeToLive() != null || getCustomEntryIdleTimeout() != null) {
+ // Force all entries to be rescheduled
+ rescheduleEntryExpiryTasks();
+ }
+
+ if (getRegionTimeToLive().getTimeout() > 0
+ && (isNoAccess || (isLimitedAccess && getRegionTimeToLive().getAction()
+ .isDistributed()))) {
+ addTTLExpiryTask();
+ }
+ if (getRegionIdleTimeout().getTimeout() > 0
+ && (isNoAccess || (isLimitedAccess && getRegionIdleTimeout()
+ .getAction().isDistributed()))) {
+ addIdleExpiryTask();
+ }
+ }
+
+ /**
+ * A boolean used to indicate if its the intialization time i.e the
+ * distributed Region is created for the first time. The variable is used at
+ * the time of lost reliablility.
+ */
+ private boolean isInitializingThread = false;
+
+ /**
+ * Called when reliability is lost. If MembershipAttributes are configured
+ * with {@link LossAction#RECONNECT}then DistributedSystem reconnect will be
+ * called asynchronously.
+ *
+ * @return true if asynchronous resumption is triggered
+ */
+ protected boolean lostReliability(final InternalDistributedMember id,
+ final Set newlyMissingRoles)
+ {
+ if (DistributedRegion.ignoreReconnect)
+ return false;
+ boolean async = false;
+ try {
+ if (getMembershipAttributes().getLossAction().isReconnect()) {
+ async = true;
+ if (isInitializingThread) {
+ doLostReliability(true, id, newlyMissingRoles);
+ }
+ else {
+ doLostReliability(false, id, newlyMissingRoles);
+ }
+ // we don't do this in the waiting pool because we're going to
+ // disconnect
+ // the distributed system, and it will wait for the pool to empty
+ /*
+ * moved to a new method called doLostReliablity. Thread t = new
+ * Thread("Reconnect Distributed System") { public void run() { try { //
+ * TODO: may need to check isReconnecting and checkReadiness...
+ * initializationLatchAfterMemberTimeout.await(); // TODO:
+ * call reconnect here
+ * getSystem().tryReconnect((GemFireCache)getCache()); // added for
+ * reconnect. synchronized (missingRequiredRoles) { // any number of
+ * threads may be waiting on missingRequiredRoles
+ * missingRequiredRoles.notifyAll(); // need to fire an event if id is
+ * not null if (hasListener() && id != null) { RoleEventImpl relEvent =
+ * new RoleEventImpl( DistributedRegion.this, Operation.CACHE_RECONNECT,
+ * null, true, id, newlyMissingRoles); dispatchListenerEvent(
+ * EnumListenerEvent.AFTER_ROLE_LOSS, relEvent); } } } catch (Exception
+ * e) { } } };
+ * t.setDaemon(true); t.start();
+ */
+ }
+ }
+ catch (CancelException cce) {
+ throw cce;
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_UNEXPECTED_EXCEPTION), e);
+ }
+ return async;
+ }
+
+ private void doLostReliability(boolean isInitializing,
+ final InternalDistributedMember id, final Set newlyMissingRoles)
+ {
+ try {
+ if (!isInitializing) {
+ // moved code to a new thread.
+ Thread t = new Thread(LocalizedStrings.DistributedRegion_RECONNECT_DISTRIBUTED_SYSTEM.toLocalizedString()) {
+ @Override
+ public void run()
+ {
+ try {
+ // TODO: may need to check isReconnecting and checkReadiness...
+ if (logger.isDebugEnabled()) {
+ logger.debug("Reliability loss with policy of reconnect and membership thread doing reconnect");
+ }
+ initializationLatchAfterMemberTimeout.await();
+ getSystem().tryReconnect(false, "Role Loss", getCache());
+ synchronized (missingRequiredRoles) {
+ // any number of threads may be waiting on missingRequiredRoles
+ missingRequiredRoles.notifyAll();
+ // need to fire an event if id is not null
+ if (hasListener() && id != null) {
+ RoleEventImpl relEvent = new RoleEventImpl(
+ DistributedRegion.this, Operation.CACHE_RECONNECT, null,
+ true, id, newlyMissingRoles);
+ dispatchListenerEvent(EnumListenerEvent.AFTER_ROLE_LOSS,
+ relEvent);
+ }
+ }
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_UNEXPECTED_EXCEPTION), e);
+ }
+ }
+ };
+ t.setDaemon(true);
+ t.start();
+
+ }
+ else {
+ getSystem().tryReconnect(false, "Role Loss", getCache()); // added for
+ // reconnect.
+ synchronized (missingRequiredRoles) {
+ // any number of threads may be waiting on missingRequiredRoles
+ missingRequiredRoles.notifyAll();
+ // need to fire an event if id is not null
+ if (hasListener() && id != null) {
+ RoleEventImpl relEvent = new RoleEventImpl(DistributedRegion.this,
+ Operation.CACHE_RECONNECT, null, true, id, newlyMissingRoles);
+ dispatchListenerEvent(EnumListenerEvent.AFTER_ROLE_LOSS, relEvent);
+ }
+ }
+ // } catch (CancelException cce){
+
+ // }
+
+ }
+ }
+ catch (CancelException ignor) {
+ throw ignor;
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_UNEXPECTED_EXCEPTION), e);
+ }
+
+ }
+
+ protected void lockCheckReadiness()
+ {
+ // fix for bug 32610
+ cache.getCancelCriterion().checkCancelInProgress(null);
+ checkReadiness();
+ }
+
+ @Override
+ public final Object validatedDestroy(Object key, EntryEventImpl event)
+ throws TimeoutException, EntryNotFoundException, CacheWriterException {
+ Lock dlock = this.getDistributedLockIfGlobal(key);
+ try {
+ return super.validatedDestroy(key, event);
+ } finally {
+ if (dlock != null) {
+ dlock.unlock();
+ }
+ }
+ }
+
+ /**
+ * @see LocalRegion#localDestroyNoCallbacks(Object)
+ */
+ @Override
+ public void localDestroyNoCallbacks(Object key)
+ {
+ super.localDestroyNoCallbacks(key);
+ if (getScope().isGlobal()) {
+ try {
+ this.getLockService().freeResources(key);
+ }
+ catch (LockServiceDestroyedException ignore) {
+ }
+ }
+ }
+
+ /**
+ * @see LocalRegion#localDestroy(Object, Object)
+ */
+ @Override
+ public void localDestroy(Object key, Object aCallbackArgument)
+ throws EntryNotFoundException
+ {
+ super.localDestroy(key, aCallbackArgument);
+ if (getScope().isGlobal()) {
+ try {
+ this.getLockService().freeResources(key);
+ }
+ catch (LockServiceDestroyedException ignore) {
+ }
+ }
+ }
+
+ /**
+ * @see LocalRegion#invalidate(Object, Object)
+ */
+ @Override
+ public void invalidate(Object key, Object aCallbackArgument)
+ throws TimeoutException, EntryNotFoundException
+ {
+ validateKey(key);
+ validateCallbackArg(aCallbackArgument);
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ Lock dlock = this.getDistributedLockIfGlobal(key);
+ try {
+ super.validatedInvalidate(key, aCallbackArgument);
+ }
+ finally {
+ if (dlock != null)
+ dlock.unlock();
+ }
+ }
+
+ @Override
+ public Lock getRegionDistributedLock() throws IllegalStateException
+ {
+ lockCheckReadiness();
+ checkForLimitedOrNoAccess();
+ if (!this.scope.isGlobal()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_DISTRIBUTION_LOCKS_ARE_ONLY_SUPPORTED_FOR_REGIONS_WITH_GLOBAL_SCOPE_NOT_0.toLocalizedString(this.scope));
+ }
+ return new RegionDistributedLock();
+ }
+
+ @Override
+ public Lock getDistributedLock(Object key) throws IllegalStateException
+ {
+ validateKey(key);
+ lockCheckReadiness();
+ checkForLimitedOrNoAccess();
+ if (!this.scope.isGlobal()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_DISTRIBUTION_LOCKS_ARE_ONLY_SUPPORTED_FOR_REGIONS_WITH_GLOBAL_SCOPE_NOT_0.toLocalizedString(this.scope));
+ }
+ if (isLockingSuspendedByCurrentThread()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_THIS_THREAD_HAS_SUSPENDED_ALL_LOCKING_FOR_THIS_REGION.toLocalizedString());
+ }
+ return new DistributedLock(key);
+ }
+
+ /**
+ * Called while NOT holding lock on parent's subregions
+ *
+ * @throws IllegalStateException
+ * if region is not compatible with a region in another VM.
+ *
+ * @see LocalRegion#initialize(InputStream, InternalDistributedMember, InternalRegionArguments)
+ */
+ @Override
+ protected void initialize(InputStream snapshotInputStream,
+ InternalDistributedMember imageTarget, InternalRegionArguments internalRegionArgs) throws TimeoutException,
+ IOException, ClassNotFoundException
+ {
+ Assert.assertTrue(!isInitialized());
+ if (logger.isDebugEnabled()) {
+ logger.debug("DistributedRegion.initialize BEGIN: {}", getFullPath());
+ }
+
+ // if we're versioning entries we need a region-level version vector
+ if (this.scope.isDistributed() && this.concurrencyChecksEnabled) {
+ createVersionVector();
+ }
+
+ if (this.scope.isGlobal()) {
+ getLockService(); // create lock service eagerly now
+ }
+
+ final IndexUpdater indexUpdater = getIndexUpdater();
+ boolean sqlfGIILockTaken = false;
+ // this try block is to release the SQLF GII lock in finally
+ // which should be done after bucket status will be set
+ // properly in LocalRegion#initialize()
+ try {
+ try {
+ try {
+ // take the GII lock to avoid missing entries while updating the
+ // index list for SQLFabric (#41330 and others)
+ if (indexUpdater != null) {
+ indexUpdater.lockForGII();
+ sqlfGIILockTaken = true;
+ }
+
+ PersistentMemberID persistentId = null;
+ boolean recoverFromDisk = isRecoveryNeeded();
+ DiskRegion dskRgn = getDiskRegion();
+ if (recoverFromDisk) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("DistributedRegion.getInitialImageAndRecovery: Starting Recovery");
+ }
+ dskRgn.initializeOwner(this); // do recovery
+ if (logger.isDebugEnabled()) {
+ logger.debug("DistributedRegion.getInitialImageAndRecovery: Finished Recovery");
+ }
+ persistentId = dskRgn.getMyPersistentID();
+ }
+
+ // Create OQL indexes before starting GII.
+ createOQLIndexes(internalRegionArgs, recoverFromDisk);
+
+ if (getDataPolicy().withReplication()
+ || getDataPolicy().withPreloaded()) {
+ getInitialImageAndRecovery(snapshotInputStream, imageTarget,
+ internalRegionArgs, recoverFromDisk, persistentId);
+ }
+ else {
+ new CreateRegionProcessor(this).initializeRegion();
+ if (snapshotInputStream != null) {
+ releaseBeforeGetInitialImageLatch();
+ loadSnapshotDuringInitialization(snapshotInputStream);
+ }
+ }
+ }
+ catch (DiskAccessException dae) {
+ this.handleDiskAccessException(dae, true);
+ throw dae;
+ }
+
+ initMembershipRoles();
+ isInitializingThread = false;
+ super.initialize(null, null, null); // makes sure all latches are released if they haven't been already
+ } finally {
+ if (this.eventTracker != null) {
+ this.eventTracker.setInitialized();
+ }
+ }
+ } finally {
+ if (sqlfGIILockTaken) {
+ indexUpdater.unlockForGII();
+ }
+ }
+ }
+
+ @Override
+ public void initialized() {
+ new UpdateAttributesProcessor(this).distribute(false);
+ }
+
+ /** True if GII was impacted by missing required roles */
+ private boolean giiMissingRequiredRoles = false;
+
+ /**
+ * A reference counter to protected the memoryThresholdReached boolean
+ */
+ private final Set memoryThresholdReachedMembers =
+ new CopyOnWriteArraySet();
+
+ private ConcurrentParallelGatewaySenderQueue hdfsQueue;
+
+ /** Sets and returns giiMissingRequiredRoles */
+ private boolean checkInitialImageForReliability(
+ InternalDistributedMember imageTarget,
+ CacheDistributionAdvisor.InitialImageAdvice advice)
+ {
+ // assumption: required roles are interesting to GII only if Reinitialize...
+// if (true)
+ return false;
+// if (getMembershipAttributes().hasRequiredRoles()
+// && getMembershipAttributes().getResumptionAction().isReinitialize()) {
+// // are any required roles missing for GII with Reinitialize?
+// Set missingRR = new HashSet(getMembershipAttributes().getRequiredRoles());
+// missingRR.removeAll(getSystem().getDistributedMember().getRoles());
+// for (Iterator iter = advice.replicates.iterator(); iter.hasNext();) {
+// DistributedMember member = (DistributedMember)iter.next();
+// missingRR.removeAll(member.getRoles());
+// }
+// for (Iterator iter = advice.others.iterator(); iter.hasNext();) {
+// DistributedMember member = (DistributedMember)iter.next();
+// missingRR.removeAll(member.getRoles());
+// }
+// for (Iterator iter = advice.preloaded.iterator(); iter.hasNext();) {
+// DistributedMember member = (DistributedMember)iter.next();
+// missingRR.removeAll(member.getRoles());
+// }
+// if (!missingRR.isEmpty()) {
+// // entering immediate loss condition, which will cause reinit on resume
+// this.giiMissingRequiredRoles = true;
+// }
+// }
+// return this.giiMissingRequiredRoles;
+ }
+
+ private void getInitialImageAndRecovery(InputStream snapshotInputStream,
+ InternalDistributedMember imageSrc, InternalRegionArguments internalRegionArgs,
+ boolean recoverFromDisk, PersistentMemberID persistentId) throws TimeoutException
+ {
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributedRegion_INITIALIZING_REGION_0, this.getName()));
+
+ ImageState imgState = getImageState();
+ imgState.init();
+ boolean targetRecreated = internalRegionArgs.getRecreateFlag();
+ Boolean isCBool = (Boolean)isConversion.get();
+ boolean isForConversion = isCBool!=null?isCBool.booleanValue():false;
+
+ if (recoverFromDisk && snapshotInputStream != null && !isForConversion) {
+ throw new InternalGemFireError(LocalizedStrings.DistributedRegion_IF_LOADING_A_SNAPSHOT_THEN_SHOULD_NOT_BE_RECOVERING_ISRECOVERING_0_SNAPSHOTSTREAM_1.toLocalizedString(new Object[] {Boolean.valueOf(recoverFromDisk), snapshotInputStream}));
+ }
+
+ ProfileExchangeProcessor targetProvider;
+ if (dataPolicy.withPersistence()) {
+ targetProvider = new CreatePersistentRegionProcessor(this,
+ getPersistenceAdvisor(), recoverFromDisk);
+ }
+ else {
+ // this will go in the advisor profile
+ targetProvider = new CreateRegionProcessor(this);
+ }
+ imgState.setInRecovery(false);
+ RegionVersionVector recovered_rvv = null;
+ if (dataPolicy.withPersistence()) {
+ recovered_rvv = (this.getVersionVector()==null?null:this.getVersionVector().getCloneForTransmission());
+ }
+ // initializeRegion will send out our profile
+ targetProvider.initializeRegion();
+
+ if(persistenceAdvisor != null) {
+ persistenceAdvisor.initialize();
+ }
+
+ // Register listener here so that the remote members are known
+ // since registering calls initializeCriticalMembers (which needs to know about
+ // remote members
+ if (!isInternalRegion()) {
+ if (!this.isDestroyed) {
+ cache.getResourceManager().addResourceListener(ResourceType.MEMORY, this);
+ }
+ }
+
+ releaseBeforeGetInitialImageLatch();
+
+ // allow GII to invoke test hooks. Do this just after releasing the
+ // before-gii latch for bug #48962. See ConcurrentLeaveDuringGIIDUnitTest
+ InitialImageOperation.beforeGetInitialImage(this);
+
+ if (snapshotInputStream != null) {
+ try {
+ if (logger.isDebugEnabled()) {
+ logger.debug("DistributedRegion.getInitialImageAndRecovery: About to load snapshot, isInitialized={}; {}",
+ isInitialized(), getFullPath());
+ }
+ loadSnapshotDuringInitialization(snapshotInputStream);
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e); // @todo change this exception?
+ }
+ catch (ClassNotFoundException e) {
+ throw new RuntimeException(e); // @todo change this exception?
+ }
+ cleanUpDestroyedTokensAndMarkGIIComplete(GIIStatus.NO_GII);
+ return;
+ }
+
+ // No snapshot provided, use the imageTarget(s)
+
+ // if we were given a recommended imageTarget, use that first, and
+ // treat it like it is a replicate (regardless of whether it actually is
+ // or not)
+
+ InitialImageOperation iiop = new InitialImageOperation(this, this.entries);
+ // [defunct] Special case GII for PR admin regions (which are always
+ // replicates and always writers
+ // bruce: this was commented out after adding the GIIAckRequest logic to
+ // force
+ // consistency before the gii operation begins
+ // if (isUsedForPartitionedRegionAdmin() ||
+ // isUsedForPartitionedRegionBucket()) {
+ // releaseBeforeGetInitialImageLatch();
+ // iiop.getFromAll(this.distAdvisor.adviseGeneric(), false);
+ // cleanUpDestroyedTokens();
+ // return;
+ // }
+
+
+ CacheDistributionAdvisor.InitialImageAdvice advice = null;
+ boolean done = false;
+ while(!done && !isDestroyed()) {
+ advice = targetProvider.getInitialImageAdvice(advice);
+ checkInitialImageForReliability(imageSrc, advice);
+ boolean attemptGetFromOne =
+ imageSrc != null // we were given a specific member
+ || this.dataPolicy.withPreloaded()
+ && !advice.preloaded.isEmpty() // this is a preloaded region
+ || (!advice.replicates.isEmpty());
+ // That is: if we have 0 or 1 giiProvider then we can do a getFromOne gii;
+ // if we have 2 or more giiProviders then we must do a getFromAll gii.
+
+ if (attemptGetFromOne) {
+ if (recoverFromDisk) {
+ if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER){
+ CacheObserverHolder.getInstance().afterMarkingGIIStarted();
+ }
+ }
+ {
+ // If we have an imageSrc and the target is reinitializing mark the
+ // getInitialImage so that it will wait until the target region is fully initialized
+ // before responding to the get image request. Otherwise, the
+ // source may respond with no data because it is still initializing,
+ // e.g. loading a snapshot.
+
+ // Plan A: use specified imageSrc, if specified
+ if (imageSrc != null) {
+ try {
+ GIIStatus ret = iiop.getFromOne(Collections.singleton(imageSrc),
+ targetRecreated, advice, recoverFromDisk, recovered_rvv);
+ if (GIIStatus.didGII(ret)) {
+ this.giiMissingRequiredRoles = false;
+ cleanUpDestroyedTokensAndMarkGIIComplete(ret);
+ done = true;
+ return;
+ }
+ } finally {
+ imageSrc = null;
+ }
+ }
+
+ // Plan C: use a replicate, if one exists
+ GIIStatus ret = iiop.getFromOne(advice.replicates, false, advice, recoverFromDisk, recovered_rvv);
+ if (GIIStatus.didGII(ret)) {
+ cleanUpDestroyedTokensAndMarkGIIComplete(ret);
+ done = true;
+ return;
+ }
+
+ // Plan D: if this is a PRELOADED region, fetch from another PRELOADED
+ if (this.dataPolicy.isPreloaded()) {
+ GIIStatus ret_preload = iiop.getFromOne(advice.preloaded, false, advice, recoverFromDisk, recovered_rvv);
+ if (GIIStatus.didGII(ret_preload)) {
+ cleanUpDestroyedTokensAndMarkGIIComplete(ret_preload);
+ done = true;
+ return;
+ }
+ } // isPreloaded
+ }
+
+ //If we got to this point, we failed in the GII. Cleanup
+ //any partial image we received
+ cleanUpAfterFailedGII(recoverFromDisk);
+
+ } // attemptGetFromOne
+ else {
+ if(!isDestroyed()) {
+ if(recoverFromDisk) {
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributedRegion_INITIALIZED_FROM_DISK,
+ new Object[] {this.getFullPath(), persistentId, getPersistentID()}));
+ if(persistentId != null) {
+ RegionLogger.logRecovery(this.getFullPath(), persistentId,
+ getDistributionManager().getDistributionManagerId());
+ }
+ } else {
+ RegionLogger.logCreate(this.getFullPath(),
+ getDistributionManager().getDistributionManagerId());
+
+ if (getPersistentID() != null) {
+ RegionLogger.logPersistence(this.getFullPath(),
+ getDistributionManager().getDistributionManagerId(),
+ getPersistentID());
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributedRegion_NEW_PERSISTENT_REGION_CREATED,
+ new Object[] {this.getFullPath(), getPersistentID()}));
+ }
+ }
+
+ /* no more union GII
+ // do union getInitialImage
+ Set rest = new HashSet();
+ rest.addAll(advice.others);
+ rest.addAll(advice.preloaded);
+ // push profile w/ recovery flag turned off at same time that we
+ // do a union getInitialImage
+ boolean pushProfile = recoverFromDisk;
+ iiop.getFromAll(rest, pushProfile);
+ */
+ cleanUpDestroyedTokensAndMarkGIIComplete(GIIStatus.NO_GII);
+ done = true;
+ return;
+ }
+ break;
+ }
+ }
+
+ return;
+ }
+
+ private void synchronizeWith(InternalDistributedMember target,
+ VersionSource idToRecover) {
+ InitialImageOperation op = new InitialImageOperation(this, this.entries);
+ op.synchronizeWith(target, idToRecover, null);
+ }
+
+ /**
+ * If this region has concurrency controls enabled this will pull any missing
+ * changes from other replicates using InitialImageOperation and a filtered
+ * chunking protocol.
+ */
+ public void synchronizeForLostMember(InternalDistributedMember
+ lostMember, VersionSource lostVersionID) {
+ if (this.concurrencyChecksEnabled == false) {
+ return;
+ }
+ CacheDistributionAdvisor advisor = getCacheDistributionAdvisor();
+ Set targets = advisor.adviseInitializedReplicates();
+ for (InternalDistributedMember target: targets) {
+ synchronizeWith(target, lostVersionID, lostMember);
+ }
+ }
+
+ /**
+ * synchronize with another member wrt messages from the given "lost" member.
+ * This can be used when a primary bucket crashes to ensure that interrupted
+ * message distribution is mended.
+ */
+ private void synchronizeWith(InternalDistributedMember target,
+ VersionSource versionMember, InternalDistributedMember lostMember) {
+ InitialImageOperation op = new InitialImageOperation(this, this.entries);
+ op.synchronizeWith(target, versionMember, lostMember);
+ }
+
+ /**
+ * invoked just before an initial image is requested from another member
+ */
+ /** remove any partial entries received in a failed GII */
+ protected void cleanUpAfterFailedGII(boolean recoverFromDisk) {
+ DiskRegion dskRgn = getDiskRegion();
+ //if we have a persistent region, instead of deleting everything on disk,
+ //we will just reset the "recovered from disk" flag. After
+ //the next GII we will delete these entries if they do not come
+ //in as part of the GII.
+ if (recoverFromDisk && dskRgn != null && dskRgn.isBackup()) {
+ dskRgn.resetRecoveredEntries(this);
+ return;
+ }
+
+ if (!this.entries.isEmpty()) {
+ closeEntries();
+ if (getDiskRegion() != null) {
+ getDiskRegion().clear(this, null);
+ }
+ // clear the left-members and version-tags sets in imageState
+ getImageState().getLeftMembers();
+ getImageState().getVersionTags();
+ // Clear OQL indexes
+ if (this.indexManager != null) {
+ try {
+ this.indexManager.rerunIndexCreationQuery();
+ } catch (Exception ex){
+ if (logger.isDebugEnabled()) {
+ logger.debug("Exception while clearing indexes after GII failure.", ex);
+ }
+ }
+ }
+ }
+ }
+
+ private void initMembershipRoles()
+ {
+ synchronized (this.advisorListener) {
+ // hold sync to prevent listener from changing initial members
+ Set others = this.distAdvisor
+ .addMembershipListenerAndAdviseGeneric(this.advisorListener);
+ this.advisorListener.addMembers(others);
+ // initialize missing required roles with initial member info
+ if (getMembershipAttributes().hasRequiredRoles()) {
+ // AdvisorListener will also sync on missingRequiredRoles
+ synchronized (this.missingRequiredRoles) {
+ this.missingRequiredRoles.addAll(getMembershipAttributes()
+ .getRequiredRoles());
+ // remove all the roles we are playing since they will never be
+ // missing
+ this.missingRequiredRoles.removeAll(getSystem()
+ .getDistributedMember().getRoles());
+ for (Iterator iter = others.iterator(); iter.hasNext();) {
+ DistributedMember other = (DistributedMember)iter.next();
+ this.missingRequiredRoles.removeAll(other.getRoles());
+ }
+ }
+ }
+ }
+ if (getMembershipAttributes().hasRequiredRoles()) {
+ // wait up to memberTimeout for required roles...
+// boolean requiredRolesAreMissing = false;
+ int memberTimeout = getSystem().getConfig().getMemberTimeout();
+ if (logger.isDebugEnabled()) {
+ logger.debug("Waiting up to {} for required roles.", memberTimeout);
+ }
+ try {
+ if (this.giiMissingRequiredRoles) {
+ // force reliability loss and possibly resumption
+ isInitializingThread = true;
+ synchronized (this.advisorListener) {
+ synchronized (this.missingRequiredRoles) {
+ // forcing state of loss because of bad GII
+ this.isMissingRequiredRoles = true;
+ getCachePerfStats().incReliableRegionsMissing(1);
+ if (getMembershipAttributes().getLossAction().isAllAccess())
+ getCachePerfStats().incReliableRegionsMissingFullAccess(1); // rahul
+ else if (getMembershipAttributes().getLossAction()
+ .isLimitedAccess())
+ getCachePerfStats().incReliableRegionsMissingLimitedAccess(1);
+ else if (getMembershipAttributes().getLossAction().isNoAccess())
+ getCachePerfStats().incReliableRegionsMissingNoAccess(1);
+ // pur code to increment the stats.
+ if (logger.isDebugEnabled()) {
+ logger.debug("GetInitialImage had missing required roles.");
+ }
+ // TODO: will this work with RECONNECT and REINITIALIZE?
+ isInitializingThread = true;
+ lostReliability(null, null);
+ if (this.missingRequiredRoles.isEmpty()) {
+ // all required roles are present so force resumption
+ this.isMissingRequiredRoles = false;
+ getCachePerfStats().incReliableRegionsMissing(-1);
+ if (getMembershipAttributes().getLossAction().isAllAccess())
+ getCachePerfStats().incReliableRegionsMissingFullAccess(-1); // rahul
+ else if (getMembershipAttributes().getLossAction()
+ .isLimitedAccess())
+ getCachePerfStats()
+ .incReliableRegionsMissingLimitedAccess(-1);
+ else if (getMembershipAttributes().getLossAction().isNoAccess())
+ getCachePerfStats().incReliableRegionsMissingNoAccess(-1);
+ // pur code to increment the stats.
+ boolean async = resumeReliability(null, null);
+ if (async) {
+ advisorListener.destroyed = true;
+ }
+ }
+ }
+ }
+ }
+ else {
+ if (!getSystem().isLoner()) {
+ waitForRequiredRoles(memberTimeout);
+ }
+ synchronized (this.advisorListener) {
+ synchronized (this.missingRequiredRoles) {
+ if (this.missingRequiredRoles.isEmpty()) {
+ Assert.assertTrue(!this.isMissingRequiredRoles);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Initialization completed with all required roles present.");
+ }
+ }
+ else {
+ // starting in state of loss...
+ this.isMissingRequiredRoles = true;
+ getCachePerfStats().incReliableRegionsMissing(1);
+ if (getMembershipAttributes().getLossAction().isAllAccess())
+ getCachePerfStats().incReliableRegionsMissingFullAccess(1); // rahul
+ else if (getMembershipAttributes().getLossAction()
+ .isLimitedAccess())
+ getCachePerfStats().incReliableRegionsMissingLimitedAccess(1);
+ else if (getMembershipAttributes().getLossAction().isNoAccess())
+ getCachePerfStats().incReliableRegionsMissingNoAccess(1);
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("Initialization completed with missing required roles: {}", this.missingRequiredRoles);
+ }
+ isInitializingThread = true;
+ lostReliability(null, null);
+ }
+ }
+ }
+ }
+ }
+ catch (RegionDestroyedException ignore) {
+ // ignore to fix bug 34639 may be thrown by waitForRequiredRoles
+ }
+ catch (CancelException ignore) {
+ // ignore to fix bug 34639 may be thrown by waitForRequiredRoles
+ if (isInitializingThread) {
+ throw ignore;
+ }
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_UNEXPECTED_EXCEPTION), e);
+ }
+
+ }
+ // open latch which will allow any threads in lostReliability to proceed
+ this.initializationLatchAfterMemberTimeout.countDown();
+ }
+ private boolean isRecoveryNeeded() {
+ return getDataPolicy().withPersistence()
+ && getDiskRegion().isRecreated();
+ }
+
+ // called by InitialImageOperation to clean up destroyed tokens
+ // release afterGetInitialImageInitializationLatch before unlocking
+ // cleanUpLock
+ @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK")
+ private void cleanUpDestroyedTokensAndMarkGIIComplete(GIIStatus giiStatus)
+ {
+ //We need to clean up the disk before we release the after get initial image latch
+ DiskRegion dskRgn = getDiskRegion();
+ if (dskRgn != null && dskRgn.isBackup()) {
+ dskRgn.finishInitializeOwner(this, giiStatus);
+ }
+ ImageState is = getImageState();
+ is.lockGII();
+ // clear the version tag and left-members sets
+ is.getVersionTags();
+ is.getLeftMembers();
+ // remove DESTROYED tokens
+ RegionVersionVector rvv = is.getClearRegionVersionVector();
+ try {
+ Iterator/**/ keysIt = getImageState().getDestroyedEntries();
+ while (keysIt.hasNext()) {
+ this.entries.removeIfDestroyed(keysIt.next());
+ }
+ if (rvv != null) {
+ // clear any entries received in the GII that are older than the RVV versions.
+ // this can happen if entry chunks were received prior to the clear() being
+ // processed
+ clearEntries(rvv);
+ }
+ //need to do this before we release the afterGetInitialImageLatch
+ if(persistenceAdvisor != null) {
+ persistenceAdvisor.setOnline(GIIStatus.didGII(giiStatus), false, getPersistentID());
+ }
+ }
+ finally {
+ // release after gii lock first so basicDestroy will see isInitialized()
+ // be true
+ // when they get the cleanUp lock.
+ try {
+ releaseAfterGetInitialImageLatch();
+ } finally { // make sure unlockGII is done for bug 40001
+ is.unlockGII();
+ }
+ }
+
+ if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER){
+ CacheObserverHolder.getInstance().afterMarkingGIICompleted();
+ }
+
+ //"Initializing region {0}" which is not acompanied by a completed message. Users think thread is stuck in some operation. Hence adding this log
+ logger.info(LocalizedMessage.create(LocalizedStrings.DistributedRegion_INITIALIZING_REGION_COMPLETED_0, this.getName()));
+ }
+
+ /**
+ * @see LocalRegion#basicDestroy(EntryEventImpl, boolean, Object)
+ */
+ @Override
+ protected
+ void basicDestroy(EntryEventImpl event,
+ boolean cacheWrite,
+ Object expectedOldValue)
+ throws EntryNotFoundException, CacheWriterException, TimeoutException {
+ // disallow local destruction for mirrored keysvalues regions
+ boolean invokeWriter = cacheWrite;
+ boolean hasSeen = false;
+ if (hasSeenEvent(event)) {
+ hasSeen = true;
+ }
+ checkIfReplicatedAndLocalDestroy(event);
+
+ try {
+ if (this.requiresOneHopForMissingEntry(event)) {
+ // bug #45704: see if a one-hop must be done for this operation
+ RegionEntry re = getRegionEntry(event.getKey());
+ if (re == null /*|| re.isTombstone()*/ || !this.generateVersionTag) {
+ if (this.srp == null) {
+ // only assert for non-client regions.
+ Assert.assertTrue(!this.dataPolicy.withReplication() || !this.generateVersionTag);
+ }
+ if (!event.isBulkOpInProgress() || this.dataPolicy.withStorage()) {
+ // removeAll will send a single one-hop for empty regions. for other missing entries
+ // we need to get a valid version number before modifying the local cache
+ // TODO: deltaGII: verify that delegating to a peer when this region is also a client is acceptable
+ boolean didDistribute = RemoteDestroyMessage.distribute(event, expectedOldValue, !this.generateVersionTag);
+
+ if (!this.generateVersionTag && !didDistribute) {
+ throw new PersistentReplicatesOfflineException();
+ }
+
+ if (didDistribute) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("Event after remoteDestroy operation: {}", event);
+ }
+ invokeWriter = false; // remote cache invoked the writer
+ if (event.getVersionTag() == null) {
+ // if the event wasn't applied by the one-hop replicate it will not have a version tag
+ // and so should not be applied to this cache
+ return;
+ }
+ }
+ }
+ }
+ }
+
+ super.basicDestroy(event, invokeWriter, expectedOldValue);
+
+ // if this is a destroy coming in from remote source, free up lock resources
+ // if this is a local origin destroy, this will happen after lock is
+ // released
+ if (this.scope.isGlobal() && event.isOriginRemote()) {
+ try {
+ getLockService().freeResources(event.getKey());
+ }
+ catch (LockServiceDestroyedException ignore) {
+ }
+ }
+
+ return;
+ }
+ finally {
+ if (hasSeen) {
+ if (event.isBulkOpInProgress() && !event.isOriginRemote()) {
+ event.getRemoveAllOperation().addEntry(event, true);
+ }
+ if (!getConcurrencyChecksEnabled() || event.hasValidVersionTag()) {
+ distributeDestroy(event, expectedOldValue);
+ event.invokeCallbacks(this,true, false);
+ }
+ }
+ }
+ }
+
+ @Override
+ void basicDestroyPart3(RegionEntry re, EntryEventImpl event,
+ boolean inTokenMode, boolean duringRI, boolean invokeCallbacks, Object expectedOldValue) {
+
+ distributeDestroy(event, expectedOldValue);
+ super.basicDestroyPart3(re, event, inTokenMode, duringRI, invokeCallbacks, expectedOldValue);
+ }
+
+ void distributeDestroy(EntryEventImpl event, Object expectedOldValue) {
+ if (event.isDistributed() && !event.isOriginRemote() && !event.isBulkOpInProgress()) {
+ boolean distribute = !event.getInhibitDistribution();
+ if (distribute) {
+ DestroyOperation op = new DestroyOperation(event);
+ op.distribute();
+ }
+ }
+ }
+
+ @Override
+ boolean evictDestroy(LRUEntry entry) {
+ boolean evictDestroyWasDone = super.evictDestroy(entry);
+ if (evictDestroyWasDone) {
+ if (this.scope.isGlobal()) {
+ try {
+ getLockService().freeResources(entry.getKey());
+ }
+ catch (LockServiceDestroyedException ignore) {
+ }
+ }
+ }
+ return evictDestroyWasDone;
+ }
+
+
+ /**
+ * @see LocalRegion#basicInvalidateRegion(RegionEventImpl)
+ */
+ @Override
+ void basicInvalidateRegion(RegionEventImpl event)
+ {
+ // disallow local invalidation for replicated regions
+ if (!event.isDistributed() && getScope().isDistributed()
+ && getDataPolicy().withReplication()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_NOT_ALLOWED_TO_DO_A_LOCAL_INVALIDATION_ON_A_REPLICATED_REGION.toLocalizedString());
+ }
+ if (shouldDistributeInvalidateRegion(event)) {
+ distributeInvalidateRegion(event);
+ }
+ super.basicInvalidateRegion(event);
+ }
+
+ /**
+ * decide if InvalidateRegionOperation should be sent to peers. broken out so
+ * that BucketRegion can override
+ * @param event
+ * @return true if {@link InvalidateRegionOperation} should be distributed, false otherwise
+ */
+ protected boolean shouldDistributeInvalidateRegion(RegionEventImpl event) {
+ return event.isDistributed() && !event.isOriginRemote();
+ }
+
+ /**
+ * Distribute the invalidate of a region given its event.
+ * This implementation sends the invalidate to peers.
+ * @since 5.7
+ */
+ protected void distributeInvalidateRegion(RegionEventImpl event) {
+ new InvalidateRegionOperation(event).distribute();
+ }
+
+ /**
+ * @see LocalRegion#basicDestroyRegion(RegionEventImpl, boolean, boolean,
+ * boolean)
+ */
+ @Override
+ void basicDestroyRegion(RegionEventImpl event, boolean cacheWrite,
+ boolean lock, boolean callbackEvents) throws CacheWriterException,
+ TimeoutException
+ {
+ final String path = getFullPath();
+ //Keep track of regions that are being destroyed. This helps avoid a race
+ //when another member concurrently creates this region. See bug 42051.
+ boolean isClose = event.getOperation().isClose();
+ if(!isClose) {
+ cache.beginDestroy(path, this);
+ }
+ try {
+ super.basicDestroyRegion(event, cacheWrite, lock, callbackEvents);
+ // send destroy region operation even if this is a localDestroyRegion (or
+ // close)
+ if (!event.isOriginRemote()) {
+ distributeDestroyRegion(event, true);
+ } else {
+ if(!event.isReinitializing()) {
+ RegionEventImpl localEvent = new RegionEventImpl(this,
+ Operation.REGION_LOCAL_DESTROY, event.getCallbackArgument(), false, getMyId(),
+ generateEventID()/* generate EventID */);
+ distributeDestroyRegion(localEvent, false/*fixes bug 41111*/);
+ }
+ }
+ notifyBridgeClients(event);
+ }
+ catch (CancelException e) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("basicDestroyRegion short-circuited due to cancellation");
+ }
+ }
+ finally {
+ if(!isClose) {
+ cache.endDestroy(path, this);
+ }
+ RegionLogger.logDestroy(path, getMyId(), getPersistentID(), isClose);
+ }
+ }
+
+
+ @Override
+ protected void distributeDestroyRegion(RegionEventImpl event,
+ boolean notifyOfRegionDeparture) {
+ if(persistenceAdvisor != null) {
+ persistenceAdvisor.releaseTieLock();
+ }
+ new DestroyRegionOperation(event, notifyOfRegionDeparture).distribute();
+ }
+
+ /**
+ * Return true if invalidation occurred; false if it did not, for example if
+ * it was already invalidated
+ *
+ * @see LocalRegion#basicInvalidate(EntryEventImpl)
+ */
+ @Override
+ void basicInvalidate(EntryEventImpl event) throws EntryNotFoundException
+ {
+
+ boolean hasSeen = false;
+ if (hasSeenEvent(event)) {
+ hasSeen = true;
+ }
+ try {
+ // disallow local invalidation for replicated regions
+ if (event.isLocalInvalid() && !event.getOperation().isLocal() && getScope().isDistributed()
+ && getDataPolicy().withReplication()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_NOT_ALLOWED_TO_DO_A_LOCAL_INVALIDATION_ON_A_REPLICATED_REGION.toLocalizedString());
+ }
+ if (this.requiresOneHopForMissingEntry(event)) {
+ // bug #45704: see if a one-hop must be done for this operation
+ RegionEntry re = getRegionEntry(event.getKey());
+ if (re == null/* || re.isTombstone()*/ || !this.generateVersionTag) {
+ if (this.srp == null) {
+ // only assert for non-client regions.
+ Assert.assertTrue(!this.dataPolicy.withReplication() || !this.generateVersionTag);
+ }
+ // TODO: deltaGII: verify that delegating to a peer when this region is also a client is acceptable
+ boolean didDistribute = RemoteInvalidateMessage.distribute(event, !this.generateVersionTag);
+ if (!this.generateVersionTag && !didDistribute) {
+ throw new PersistentReplicatesOfflineException();
+ }
+ if (didDistribute) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Event after remoteInvalidate operation: {}", event);
+ }
+ if (event.getVersionTag() == null) {
+ // if the event wasn't applied by the one-hop replicate it will not have a version tag
+ // and so should not be applied to this cache
+ return;
+ }
+ }
+ }
+ }
+
+ super.basicInvalidate(event);
+
+ return;
+ } finally {
+ if (hasSeen) {
+ if (!getConcurrencyChecksEnabled() || event.hasValidVersionTag()) {
+ distributeInvalidate(event);
+ event.invokeCallbacks(this,true, false);
+ }
+ }
+ }
+ }
+
+ @Override
+ void basicInvalidatePart3(RegionEntry re, EntryEventImpl event,
+ boolean invokeCallbacks) {
+ distributeInvalidate(event);
+ super.basicInvalidatePart3(re, event, invokeCallbacks);
+ }
+
+ void distributeInvalidate(EntryEventImpl event) {
+ if (!this.regionInvalid && event.isDistributed() && !event.isOriginRemote()
+ && !isTX() /* only distribute if non-tx */) {
+ if (event.isDistributed() && !event.isOriginRemote()) {
+ boolean distribute = !event.getInhibitDistribution();
+ if (distribute) {
+ InvalidateOperation op = new InvalidateOperation(event);
+ op.distribute();
+ }
+ }
+ }
+ }
+
+
+ @Override
+ void basicUpdateEntryVersion(EntryEventImpl event)
+ throws EntryNotFoundException {
+
+ try {
+ if (!hasSeenEvent(event)) {
+ super.basicUpdateEntryVersion(event);
+ }
+ return;
+ } finally {
+ if (!getConcurrencyChecksEnabled() || event.hasValidVersionTag()) {
+ distributeUpdateEntryVersion(event);
+ }
+ }
+ }
+
+ private void distributeUpdateEntryVersion(EntryEventImpl event) {
+ if (!this.regionInvalid && event.isDistributed() && !event.isOriginRemote()
+ && !isTX() /* only distribute if non-tx */) {
+ if (event.isDistributed() && !event.isOriginRemote()) {
+ UpdateEntryVersionOperation op = new UpdateEntryVersionOperation(event);
+ op.distribute();
+ }
+ }
+ }
+
+ @Override
+ protected void basicClear(RegionEventImpl ev)
+ {
+ Lock dlock = this.getRegionDistributedLockIfGlobal();
+ try {
+ super.basicClear(ev);
+ }
+ finally {
+ if (dlock != null)
+ dlock.unlock();
+ }
+ }
+
+ @Override
+ void basicClear(RegionEventImpl regionEvent, boolean cacheWrite) {
+ if (this.concurrencyChecksEnabled && !this.dataPolicy.withReplication()) {
+ boolean retry = false;
+ do {
+ // non-replicate regions must defer to a replicate for clear/invalidate of region
+ Set repls = this.distAdvisor.adviseReplicates();
+ if (repls.size() > 0) {
+ InternalDistributedMember mbr = repls.iterator().next();
+ RemoteRegionOperation op = RemoteRegionOperation.clear(mbr, this);
+ try {
+ op.distribute();
+ return;
+ } catch (CancelException e) {
+ this.stopper.checkCancelInProgress(e);
+ retry = true;
+ } catch (RemoteOperationException e) {
+ this.stopper.checkCancelInProgress(e);
+ retry = true;
+ }
+ }
+ } while (retry);
+ }
+ // if no version vector or if no replicates are around, use the default mechanism
+ super.basicClear(regionEvent, cacheWrite);
+ }
+
+
+ @Override
+ void cmnClearRegion(RegionEventImpl regionEvent, boolean cacheWrite, boolean useRVV) {
+ boolean enableRVV = useRVV && this.dataPolicy.withReplication() && this.concurrencyChecksEnabled && !getDistributionManager().isLoner();
+
+ //Fix for 46338 - apparently multiple threads from the same VM are allowed
+ //to suspend locking, which is what distributedLockForClear() does. We don't
+ //want that to happen, so we'll synchronize to make sure only one thread on
+ //this member performs a clear.
+ synchronized(clearLock) {
+ if (enableRVV) {
+
+ distributedLockForClear();
+ try {
+ Set participants = getCacheDistributionAdvisor().adviseInvalidateRegion();
+ // pause all generation of versions and flush from the other members to this one
+ try {
+ obtainWriteLocksForClear(regionEvent, participants);
+ clearRegionLocally(regionEvent, cacheWrite, null);
+ if (!regionEvent.isOriginRemote() && regionEvent.isDistributed()) {
+ DistributedClearOperation.clear(regionEvent, null, participants);
+ }
+ } finally {
+ releaseWriteLocksForClear(regionEvent, participants);
+ }
+ }
+ finally {
+ distributedUnlockForClear();
+ }
+ } else {
+ Set participants = getCacheDistributionAdvisor().adviseInvalidateRegion();
+ clearRegionLocally(regionEvent, cacheWrite, null);
+ if (!regionEvent.isOriginRemote() && regionEvent.isDistributed()) {
+ DistributedClearOperation.clear(regionEvent, null, participants);
+ }
+ }
+ }
+
+ // since clients do not maintain RVVs except for tombstone GC
+ // we need to ensure that current ops reach the client queues
+ // before queuing a clear, but there is no infrastructure for doing so
+ notifyBridgeClients(regionEvent);
+ }
+
+ /**
+ * Obtain a distributed lock for the clear operation.
+ */
+ private void distributedLockForClear() {
+ if (!this.scope.isGlobal()) { // non-global regions must lock when using RVV
+ try {
+ getLockService().lock("_clearOperation", -1, -1);
+ } catch(IllegalStateException e) {
+ lockCheckReadiness();
+ throw e;
+ }
+ }
+ }
+
+ /**
+ * Release the distributed lock for the clear operation.
+ */
+ private void distributedUnlockForClear() {
+ if (!this.scope.isGlobal()) {
+ try {
+ getLockService().unlock("_clearOperation");
+ } catch(IllegalStateException e) {
+ lockCheckReadiness();
+ throw e;
+ }
+ }
+ }
+
+
+ /** obtain locks preventing generation of new versions in other members
+ * @param participants
+ **/
+ private void obtainWriteLocksForClear(RegionEventImpl regionEvent, Set participants) {
+ lockLocallyForClear(getDistributionManager(), getMyId());
+ DistributedClearOperation.lockAndFlushToOthers(regionEvent, participants);
+ }
+
+ /** pause local operations so that a clear() can be performed and flush comm channels to the given member
+ */
+ public void lockLocallyForClear(DM dm, InternalDistributedMember locker) {
+ RegionVersionVector rvv = getVersionVector();
+ if (rvv != null) {
+ // block new operations from being applied to the region map
+ rvv.lockForClear(getFullPath(), dm, locker);
+ //Check for region destroyed after we have locked, to make sure
+ //we don't continue a clear if the region has been destroyed.
+ checkReadiness();
+ // wait for current operations to
+ if (!locker.equals(dm.getDistributionManagerId())) {
+ Set mbrs = getDistributionAdvisor().adviseCacheOp();
+ StateFlushOperation.flushTo(mbrs, this);
+ }
+ }
+ }
+
+ /** releases the locks obtained in obtainWriteLocksForClear
+ * @param participants */
+ private void releaseWriteLocksForClear(RegionEventImpl regionEvent, Set participants) {
+ getVersionVector().unlockForClear(getMyId());
+ DistributedClearOperation.releaseLocks(regionEvent, participants);
+ }
+
+ /**
+ * Wait for in progress clears that were initiated by this member.
+ */
+ private void waitForInProgressClear() {
+
+ RegionVersionVector rvv = getVersionVector();
+ if (rvv != null) {
+ synchronized(clearLock) {
+ //do nothing;
+ //DAN - I'm a little scared that the compiler might optimize
+ //away this synchronization if we really do nothing. Hence
+ //my fine log message below. This might not be necessary.
+ if (logger.isDebugEnabled()) {
+ logger.debug("Done waiting for clear");
+ }
+ }
+ }
+
+ }
+
+ /**
+ * Distribute Tombstone garbage-collection information to all peers with storage
+ */
+ protected EventID distributeTombstoneGC(Set keysRemoved) {
+ this.getCachePerfStats().incTombstoneGCCount();
+ EventID eventId = new EventID(getSystem());
+ DistributedTombstoneOperation gc = DistributedTombstoneOperation.gc(this, eventId);
+ gc.distribute();
+ notifyClientsOfTombstoneGC(getVersionVector().getTombstoneGCVector(), keysRemoved, eventId, null);
+ return eventId;
+ }
+
+ // test hook for DistributedAckRegionCCEDUnitTest
+ public static boolean LOCALCLEAR_TESTHOOK;
+
+ @Override
+ void basicLocalClear(RegionEventImpl rEvent) {
+ if (getScope().isDistributed() && getDataPolicy().withReplication() && !LOCALCLEAR_TESTHOOK) {
+ throw new UnsupportedOperationException(LocalizedStrings.DistributedRegion_LOCALCLEAR_IS_NOT_SUPPORTED_ON_DISTRIBUTED_REPLICATED_REGIONS.toLocalizedString());
+ }
+ super.basicLocalClear(rEvent);
+ }
+
+ public final DistributionConfig getDistributionConfig() {
+ return getSystem().getDistributionManager().getConfig();
+ }
+
+ /**
+ * Sends a list of queued messages to members playing a specified role
+ *
+ * @param list
+ * List of QueuedOperation instances to send. Any messages sent will
+ * be removed from this list
+ * @param role
+ * the role that a recipient must be playing
+ * @return true if at least one message made it to at least one guy playing
+ * the role
+ */
+ boolean sendQueue(List list, Role role) {
+ SendQueueOperation op = new SendQueueOperation(getDistributionManager(),
+ this, list, role);
+ return op.distribute();
+ }
+
+ /*
+ * @see SearchLoadAndWriteProcessor#initialize(LocalRegion, Object, Object)
+ */
+ public final CacheDistributionAdvisor getDistributionAdvisor()
+ {
+ return this.distAdvisor;
+ }
+
+ public final CacheDistributionAdvisor getCacheDistributionAdvisor()
+ {
+ return this.distAdvisor;
+ }
+
+ public final PersistenceAdvisor getPersistenceAdvisor() {
+ return this.persistenceAdvisor;
+ }
+
+ public final PersistentMemberID getPersistentID() {
+ return this.persistentId;
+ }
+
+ /** Returns the distribution profile; lazily creates one if needed */
+ public Profile getProfile() {
+ return this.distAdvisor.createProfile();
+ }
+
+ public void fillInProfile(Profile p) {
+ assert p instanceof CacheProfile;
+ CacheProfile profile = (CacheProfile)p;
+ profile.dataPolicy = getDataPolicy();
+ profile.hasCacheLoader = basicGetLoader() != null;
+ profile.hasCacheWriter = basicGetWriter() != null;
+ profile.hasCacheListener = hasListener();
+ Assert.assertTrue(this.scope.isDistributed());
+ profile.scope = this.scope;
+ profile.inRecovery = getImageState().getInRecovery();
+ profile.isPersistent = getDataPolicy().withPersistence();
+ profile.setSubscriptionAttributes(getSubscriptionAttributes());
+ // Kishor : Below PDX check is added for rolling upgrade support. We are
+ // removing Old wan in this checkin. PDX region are always gatewayEnabled
+ // irrespective whether gatewayHub is configured or not.
+ // Old version Pdx region always has this attribute true so to avoid region
+ // attribute comparison across member we are setting it to true.
+ if (this.isPdxTypesRegion()) {
+ profile.isGatewayEnabled = true;
+ }
+ else {
+ profile.isGatewayEnabled = false;
+ }
+ profile.serialNumber = getSerialNumber();
+ profile.regionInitialized = this.isInitialized();
+ if (!this.isUsedForPartitionedRegionBucket()) {
+ profile.memberUnInitialized = getCache().isUnInitializedMember(
+ profile.getDistributedMember());
+ }
+ else {
+ profile.memberUnInitialized = false;
+ }
+ profile.persistentID = getPersistentID();
+ if(getPersistenceAdvisor() != null) {
+ profile.persistenceInitialized = getPersistenceAdvisor().isOnline();
+ }
+ profile.hasCacheServer = ((this.cache.getCacheServers().size() > 0)?true:false);
+ profile.requiresOldValueInEvents = this.dataPolicy.withReplication() &&
+ this.filterProfile != null && this.filterProfile.hasCQs();
+ profile.gatewaySenderIds = getGatewaySenderIds();
+ profile.asyncEventQueueIds = getAsyncEventQueueIds();
+ profile.isOffHeap = getOffHeap();
+ }
+
+ /**
+ * Return the DistributedLockService associated with this Region. This method
+ * will lazily create that service the first time it is invoked on this
+ * region.
+ */
+ public DistributedLockService getLockService()
+ {
+ synchronized (this.dlockMonitor) {
+// Assert.assertTrue(this.scope.isGlobal()); since 7.0 this is used for distributing clear() ops
+
+ String svcName = getFullPath();
+
+ if (this.dlockService == null) {
+ this.dlockService = DistributedLockService.getServiceNamed(svcName);
+ if (this.dlockService == null) {
+ this.dlockService = DLockService.create(
+ getFullPath(),
+ getSystem(),
+ true /*distributed*/,
+ false /*destroyOnDisconnect*/, // region destroy will destroy dls
+ false /*automateFreeResources*/); // manual freeResources only
+ }
+ // handle is-lock-grantor region attribute...
+ if (this.isLockGrantor) {
+ this.dlockService.becomeLockGrantor();
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("LockService for {} is using LockLease={}, LockTimeout=", svcName, getCache().getLockLease(), getCache().getLockTimeout());
+ }
+ }
+ return this.dlockService;
+ }
+ }
+
+ /**
+ * @see LocalRegion#isCurrentlyLockGrantor()
+ */
+ @Override
+ protected boolean isCurrentlyLockGrantor()
+ {
+ if (!this.scope.isGlobal())
+ return false;
+ return getLockService().isLockGrantor();
+ }
+
+ @Override
+ public boolean isLockGrantor()
+ {
+ if (!this.scope.isGlobal())
+ return false;
+ return this.isLockGrantor;
+ }
+
+ @Override
+ public void becomeLockGrantor()
+ {
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ if (!this.scope.isGlobal()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_DISTRIBUTION_LOCKS_ARE_ONLY_SUPPORTED_FOR_REGIONS_WITH_GLOBAL_SCOPE_NOT_0.toLocalizedString(this.scope));
+ }
+
+ DistributedLockService svc = getLockService();
+ try {
+ super.becomeLockGrantor();
+ if (!svc.isLockGrantor()) {
+ svc.becomeLockGrantor();
+ }
+ }
+ finally {
+ if (!svc.isLockGrantor()) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("isLockGrantor is false after becomeLockGrantor for {}", getFullPath());
+ }
+ }
+ }
+ }
+
+ /** @return the deserialized value */
+ @Override
+ @Retained
+ protected Object findObjectInSystem(KeyInfo keyInfo, boolean isCreate,
+ TXStateInterface txState, boolean generateCallbacks, Object localValue, boolean disableCopyOnRead,
+ boolean preferCD, ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones, boolean allowReadFromHDFS)
+ throws CacheLoaderException, TimeoutException
+ {
+ checkForLimitedOrNoAccess();
+
+ RegionEntry re = null;
+ final Object key = keyInfo.getKey();
+ final Object aCallbackArgument = keyInfo.getCallbackArg();
+ Operation op;
+ if (isCreate) {
+ op = Operation.CREATE;
+ }
+ else {
+ op = Operation.UPDATE;
+ }
+ long lastModified = 0L;
+ boolean fromServer = false;
+ EntryEventImpl event = null;
+ @Retained Object result = null;
+ boolean incrementUseCountForSqlf = false;
+ try {
+ {
+ if (this.srp != null) {
+ EntryEventImpl holder = EntryEventImpl.createVersionTagHolder();
+ try {
+ Object value = this.srp.get(key, aCallbackArgument, holder);
+ fromServer = value != null;
+ if (fromServer) {
+ event = EntryEventImpl.create(this, op, key, value,
+ aCallbackArgument, false,
+ getMyId(), generateCallbacks);
+ event.setVersionTag(holder.getVersionTag());
+ event.setFromServer(fromServer); // fix for bug 39358
+ if (clientEvent != null && clientEvent.getVersionTag() == null) {
+ clientEvent.setVersionTag(holder.getVersionTag());
+ }
+ }
+ } finally {
+ holder.release();
+ }
+ }
+ }
+
+ if (!fromServer) {
+ //Do not generate Event ID
+ event = EntryEventImpl.create(this, op, key, null /*newValue*/,
+ aCallbackArgument, false,
+ getMyId(), generateCallbacks);
+ if (requestingClient != null) {
+ event.setContext(requestingClient);
+ }
+ SearchLoadAndWriteProcessor processor =
+ SearchLoadAndWriteProcessor.getProcessor();
+ try {
+ processor.initialize(this, key, aCallbackArgument);
+ // processor fills in event
+ processor.doSearchAndLoad(event, txState, localValue);
+ if (clientEvent != null && clientEvent.getVersionTag() == null) {
+ clientEvent.setVersionTag(event.getVersionTag());
+ }
+ lastModified = processor.getLastModified();
+ }
+ finally {
+ processor.release();
+ }
+ }
+ if (event.hasNewValue() && !isMemoryThresholdReachedForLoad()) {
+ try {
+ // Set eventId. Required for interested clients.
+ event.setNewEventId(cache.getDistributedSystem());
+
+ long startPut = CachePerfStats.getStatTime();
+ validateKey(key);
+// if (event.getOperation().isLoad()) {
+// this.performedLoad(event, lastModified, txState);
+// }
+ // this next step also distributes the object to other processes, if necessary
+ try {
+ // set the tail key so that the event is passed to GatewaySender queues.
+ // if the tailKey is not set, the event gets filtered out in ParallelGatewaySenderQueue
+ if (this instanceof BucketRegion) {
+ if (((BucketRegion)this).getPartitionedRegion().isParallelWanEnabled())
+ ((BucketRegion)this).handleWANEvent(event);
+ }
+ re = basicPutEntry(event, lastModified);
+ incrementUseCountForSqlf = GemFireCacheImpl.sqlfSystem() ;
+ } catch (ConcurrentCacheModificationException e) {
+ // the cache was modified while we were searching for this entry and
+ // the netsearch result was elided. Return the current value from the cache
+ re = getRegionEntry(key);
+ if (re != null) {
+ event.setNewValue(re.getValue(this)); // OFFHEAP: need to incrc, copy to heap to setNewValue, decrc
+ }
+ }
+ if (!isTX()) {
+ getCachePerfStats().endPut(startPut, event.isOriginRemote());
+ }
+ }
+ catch (CacheWriterException cwe) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("findObjectInSystem: writer exception putting entry {} : {}", event, cwe);
+ }
+ }
+ }
+ if (isCreate) {
+ recordMiss(re, key);
+ }
+
+ if (preferCD) {
+ if (event.hasDelta()) {
+ result = event.getNewValue();
+ } else {
+ result = event.getRawNewValueAsHeapObject();
+ }
+ } else {
+ result = event.getNewValue();
+ }
+ //For SQLFire , we need to increment the use count so that returned
+ //object has use count 2
+ if( incrementUseCountForSqlf && result instanceof Chunk) {
+ ((Chunk)result).retain();
+ }
+ return result;
+ } finally {
+ if (event != null) {
+ event.release();
+ }
+ }
+ }
+
+ protected ConcurrentParallelGatewaySenderQueue getHDFSQueue() {
+ if (this.hdfsQueue == null) {
+ String asyncQId = this.getPartitionedRegion().getHDFSEventQueueName();
+ final AsyncEventQueueImpl asyncQ = (AsyncEventQueueImpl)this.getCache().getAsyncEventQueue(asyncQId);
+ final AbstractGatewaySender gatewaySender = (AbstractGatewaySender)asyncQ.getSender();
+ AbstractGatewaySenderEventProcessor ep = gatewaySender.getEventProcessor();
+ if (ep == null) return null;
+ hdfsQueue = (ConcurrentParallelGatewaySenderQueue)ep.getQueue();
+ }
+ return hdfsQueue;
+ }
+
+ /** hook for subclasses to note that a cache load was performed
+ * @see BucketRegion#performedLoad
+ */
+// void performedLoad(EntryEventImpl event, long lastModifiedTime, TXState txState)
+// throws CacheWriterException {
+// // no action in DistributedRegion
+// }
+
+ /**
+ * @see LocalRegion#cacheWriteBeforeDestroy(EntryEventImpl, Object)
+ * @return true if cacheWrite was performed
+ */
+ @Override
+ boolean cacheWriteBeforeDestroy(EntryEventImpl event, Object expectedOldValue)
+ throws CacheWriterException, EntryNotFoundException, TimeoutException
+ {
+
+ boolean result = false;
+ if (event.isDistributed()) {
+ CacheWriter localWriter = basicGetWriter();
+ Set netWriteRecipients = localWriter == null ? this.distAdvisor
+ .adviseNetWrite() : null;
+
+ if ((localWriter != null
+ || (netWriteRecipients != null && !netWriteRecipients.isEmpty())) &&
+ !event.inhibitAllNotifications()) {
+ final long start = getCachePerfStats().startCacheWriterCall();
+ try {
+ event.setOldValueFromRegion();
+ SearchLoadAndWriteProcessor processor =
+ SearchLoadAndWriteProcessor.getProcessor();
+ try {
+ processor.initialize(this, event.getKey(), null);
+ processor.doNetWrite(event, netWriteRecipients, localWriter,
+ SearchLoadAndWriteProcessor.BEFOREDESTROY);
+ result = true;
+ }
+ finally {
+ processor.release();
+ }
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ }
+ serverDestroy(event, expectedOldValue);
+ }
+ return result;
+ }
+
+ /**
+ * @see LocalRegion#cacheWriteBeforeRegionDestroy(RegionEventImpl)
+ */
+ @Override
+ boolean cacheWriteBeforeRegionDestroy(RegionEventImpl event)
+ throws CacheWriterException, TimeoutException
+ {
+ boolean result = false;
+ if (event.getOperation().isDistributed()) {
+ CacheWriter localWriter = basicGetWriter();
+ Set netWriteRecipients = localWriter == null ? this.distAdvisor
+ .adviseNetWrite() : null;
+
+ if (localWriter != null
+ || (netWriteRecipients != null && !netWriteRecipients.isEmpty())) {
+ final long start = getCachePerfStats().startCacheWriterCall();
+ try {
+ SearchLoadAndWriteProcessor processor =
+ SearchLoadAndWriteProcessor.getProcessor();
+ try {
+ processor.initialize(this, "preDestroyRegion", null);
+ processor.doNetWrite(event, netWriteRecipients, localWriter,
+ SearchLoadAndWriteProcessor.BEFOREREGIONDESTROY);
+ result = true;
+ }
+ finally {
+ processor.release();
+ }
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ }
+ serverRegionDestroy(event);
+ }
+ return result;
+ }
+
+ protected void distributedRegionCleanup(RegionEventImpl event)
+ {
+ if (event == null || event.getOperation() != Operation.REGION_REINITIALIZE) {
+ // only perform this if reinitialize is not due to resumption
+ // (REGION_REINITIALIZE)
+ // or if event is null then this was a failed initialize (create)
+ // wake up any threads in waitForRequiredRoles... they will checkReadiness
+ synchronized (this.missingRequiredRoles) {
+ this.missingRequiredRoles.notifyAll();
+ }
+ }
+
+ if(persistenceAdvisor != null) {
+ this.persistenceAdvisor.close(); // fix for bug 41094
+ }
+ this.distAdvisor.close();
+ DLockService dls = null;
+
+ //Fix for bug 46338. Wait for in progress clears before destroying the
+ //lock service, because destroying the service immediately releases the dlock
+ waitForInProgressClear();
+
+ synchronized (this.dlockMonitor) {
+ if (this.dlockService != null) {
+ dls = (DLockService)this.dlockService;
+ }
+ }
+ if (dls != null) {
+ try {
+ dls.destroyAndRemove();
+ }
+ catch (CancelException e) {
+ // bug 37118
+ if (logger.isDebugEnabled()) {
+ logger.debug("DLS destroy abridged due to shutdown", e);
+ }
+ }
+ catch (Exception ex) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributedRegion_DLS_DESTROY_MAY_HAVE_FAILED_FOR_0, this.getFullPath()), ex);
+ }
+ }
+ if (this.rmq != null) {
+ this.rmq.close();
+ }
+
+ //Fix for #48066 - make sure that region operations are completely
+ //distributed to peers before destroying the region.
+ long timeout = 1000L * getCache().getDistributedSystem().getConfig().getAckWaitThreshold();
+ Boolean flushOnClose = !Boolean.getBoolean("gemfire.no-flush-on-close"); // test hook
+ if (!this.cache.forcedDisconnect() &&
+ flushOnClose && this.getDistributionManager().getMembershipManager() != null
+ && this.getDistributionManager().getMembershipManager().isConnected()) {
+ getDistributionAdvisor().forceNewMembershipVersion();
+ try {
+ getDistributionAdvisor().waitForCurrentOperations(timeout);
+ } catch (Exception e) {
+ // log this but try to close the region so that listeners are invoked
+ logger.warn(LocalizedMessage.create(LocalizedStrings.GemFireCache_0_ERROR_CLOSING_REGION_1,
+ new Object[] { this, getFullPath() }), e);
+ }
+ }
+ }
+
+ /**
+ * In addition to inherited code this method also invokes
+ * RegionMembershipListeners
+ */
+ @Override
+ protected void postCreateRegion()
+ {
+ super.postCreateRegion();
+ // should we sync on this.distAdvisor first to prevent bug 44369?
+ synchronized (this.advisorListener) {
+ Set others = this.advisorListener.getInitialMembers();
+ CacheListener[] listeners = fetchCacheListenersField();
+ if (listeners != null) {
+ for (int i = 0; i < listeners.length; i++) {
+ if (listeners[i] instanceof RegionMembershipListener) {
+ RegionMembershipListener rml = (RegionMembershipListener)listeners[i];
+ try {
+ DistributedMember[] otherDms = new DistributedMember[others
+ .size()];
+ others.toArray(otherDms);
+ rml.initialMembers(this, otherDms);
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.error(LocalizedMessage.create(LocalizedStrings.DistributedRegion_EXCEPTION_OCCURRED_IN_REGIONMEMBERSHIPLISTENER), t);
+ }
+ }
+ }
+ }
+ Set allGatewaySenderIds = getAllGatewaySenderIds();
+ if (!allGatewaySenderIds.isEmpty()) {
+ for (GatewaySender sender : cache.getAllGatewaySenders()) {
+ if (sender.isParallel()
+ && allGatewaySenderIds.contains(sender.getId())) {
+ //Fix for Bug#51491. Once decided to support this configuration we have call addShadowPartitionedRegionForUserRR
+ if (sender.getId().contains(
+ AsyncEventQueueImpl.ASYNC_EVENT_QUEUE_PREFIX)) {
+ throw new AsyncEventQueueConfigurationException(
+ LocalizedStrings.ParallelAsyncEventQueue_0_CAN_NOT_BE_USED_WITH_REPLICATED_REGION_1.toLocalizedString(new Object[] {
+ AsyncEventQueueImpl
+ .getAsyncEventQueueIdFromSenderId(sender.getId()),
+ this.getFullPath() }));
+ }
+ throw new GatewaySenderConfigurationException(
+ LocalizedStrings.ParallelGatewaySender_0_CAN_NOT_BE_USED_WITH_REPLICATED_REGION_1
+ .toLocalizedString(new Object[] { sender.getId(),
+ this.getFullPath() }));
+
+ // if (sender.isRunning()) {
+ // ConcurrentParallelGatewaySenderQueue parallelQueue =
+ // (ConcurrentParallelGatewaySenderQueue)((ParallelGatewaySenderImpl)sender)
+ // .getQueues().toArray(new RegionQueue[1])[0];
+ // parallelQueue.addShadowPartitionedRegionForUserRR(this);
+ // }
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Free resources held by this region. This method is invoked after
+ * isDestroyed has been set to true.
+ *
+ * @see LocalRegion#postDestroyRegion(boolean, RegionEventImpl)
+ */
+ @Override
+ protected void postDestroyRegion(boolean destroyDiskRegion,
+ RegionEventImpl event)
+ {
+ distributedRegionCleanup(event);
+
+ try {
+ super.postDestroyRegion(destroyDiskRegion, event);
+ }
+ catch (CancelException e) {
+ // I don't think this should ever happens: bulletproofing for bug 39454
+ logger.warn("postDestroyRegion: encountered cancellation", e);
+ }
+
+ if (this.rmq != null && destroyDiskRegion) {
+ this.rmq.destroy();
+ }
+ }
+
+ @Override
+ void cleanupFailedInitialization()
+ {
+ super.cleanupFailedInitialization();
+ try {
+ RegionEventImpl ev = new RegionEventImpl(this, Operation.REGION_CLOSE, null, false, getMyId(),
+ generateEventID());
+ distributeDestroyRegion(ev, true);
+ distributedRegionCleanup(null);
+ } catch(RegionDestroyedException e) {
+ //someone else must have concurrently destroyed the region (maybe a distributed destroy)
+ } catch(CancelException e) {
+ //cache or DS is closed, ignore
+ } catch(VirtualMachineError e) {
+ SystemFailure.initiateFailure(e);
+ throw e;
+ } catch(Throwable t) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.DistributedRegion_ERROR_CLEANING_UP_FAILED_INITIALIZATION, this), t);
+ }
+ }
+
+ /**
+ * @see LocalRegion#handleCacheClose(Operation)
+ */
+ @Override
+ void handleCacheClose(Operation op)
+ {
+ try {
+ super.handleCacheClose(op);
+ }
+ finally {
+ distributedRegionCleanup(null);
+ }
+ }
+
+ /**
+ * invoke a cache writer before a put is performed elsewhere
+ *
+ * @see LocalRegion#cacheWriteBeforePut(EntryEventImpl, Set, CacheWriter, boolean, Object)
+ */
+ @Override
+ protected void cacheWriteBeforePut(EntryEventImpl event, Set netWriteRecipients,
+ CacheWriter localWriter,
+ boolean requireOldValue,
+ Object expectedOldValue)
+ throws CacheWriterException, TimeoutException
+ {
+ if ((localWriter != null
+ || (netWriteRecipients != null && !netWriteRecipients.isEmpty())) &&
+ !event.inhibitAllNotifications()) {
+ final boolean isNewKey = event.getOperation().isCreate();
+ final long start = getCachePerfStats().startCacheWriterCall();
+ try {
+ SearchLoadAndWriteProcessor processor =
+ SearchLoadAndWriteProcessor.getProcessor();
+ processor.initialize(this, "preUpdate", null);
+ try {
+ if (!isNewKey) {
+ processor.doNetWrite(event, netWriteRecipients, localWriter,
+ SearchLoadAndWriteProcessor.BEFOREUPDATE);
+ }
+ else {
+ processor.doNetWrite(event, netWriteRecipients, localWriter,
+ SearchLoadAndWriteProcessor.BEFORECREATE);
+ }
+ }
+ finally {
+ processor.release();
+ }
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ }
+
+ serverPut(event, requireOldValue, expectedOldValue);
+ }
+
+ @Override
+ protected void cacheListenersChanged(boolean nowHasListener)
+ {
+ if (nowHasListener) {
+ this.advisorListener.initRMLWrappers();
+ }
+ new UpdateAttributesProcessor(this).distribute();
+ }
+
+ @Override
+ protected void cacheWriterChanged(CacheWriter oldWriter)
+ {
+ super.cacheWriterChanged(oldWriter);
+ if (oldWriter == null ^ basicGetWriter() == null) {
+ new UpdateAttributesProcessor(this).distribute();
+ }
+ }
+
+ @Override
+ protected void cacheLoaderChanged(CacheLoader oldLoader)
+ {
+ super.cacheLoaderChanged(oldLoader);
+ if (oldLoader == null ^ basicGetLoader() == null) {
+ new UpdateAttributesProcessor(this).distribute();
+ }
+ }
+
+ public void addGatewaySenderId(String gatewaySenderId) {
+ super.addGatewaySenderId(gatewaySenderId);
+ new UpdateAttributesProcessor(this).distribute();
+ }
+
+ public void removeGatewaySenderId(String gatewaySenderId) {
+ super.removeGatewaySenderId(gatewaySenderId);
+ new UpdateAttributesProcessor(this).distribute();
+ }
+
+ public void addAsyncEventQueueId(String asyncEventQueueId) {
+ super.addAsyncEventQueueId(asyncEventQueueId);
+ new UpdateAttributesProcessor(this).distribute();
+ }
+
+ public void removeAsyncEventQueueId(String asyncEventQueueId) {
+ super.removeAsyncEventQueueId(asyncEventQueueId);
+ new UpdateAttributesProcessor(this).distribute();
+ }
+
+ public void checkSameSenderIdsAvailableOnAllNodes() {
+ List senderIds = this.getCacheDistributionAdvisor()
+ .adviseSameGatewaySenderIds(getGatewaySenderIds());
+ if (!senderIds.isEmpty()) {
+ throw new GatewaySenderConfigurationException(
+ LocalizedStrings.Region_REGION_0_HAS_1_GATEWAY_SENDER_IDS_ANOTHER_CACHE_HAS_THE_SAME_REGION_WITH_2_GATEWAY_SENDER_IDS_FOR_REGION_ACROSS_ALL_MEMBERS_IN_DS_GATEWAY_SENDER_IDS_SHOULD_BE_SAME
+ .toLocalizedString(new Object[] { this.getName(),
+ senderIds.get(0), senderIds.get(1) }));
+ }
+
+ List asycnQueueIds = this.getCacheDistributionAdvisor()
+ .adviseSameAsyncEventQueueIds(getAsyncEventQueueIds());
+ if (!asycnQueueIds.isEmpty()) {
+ throw new GatewaySenderConfigurationException(
+ LocalizedStrings.Region_REGION_0_HAS_1_ASYNC_EVENT_QUEUE_IDS_ANOTHER_CACHE_HAS_THE_SAME_REGION_WITH_2_ASYNC_EVENT_QUEUE_IDS_FOR_REGION_ACROSS_ALL_MEMBERS_IN_DS_ASYNC_EVENT_QUEUE_IDS_SHOULD_BE_SAME
+ .toLocalizedString(new Object[] { this.getName(),
+ asycnQueueIds.get(0), asycnQueueIds.get(1) }));
+ }
+ }
+ /**
+ * Wraps call to dlock service in order to throw RegionDestroyedException if
+ * dlock service throws IllegalStateException and isDestroyed is true.
+ */
+ private boolean isLockingSuspendedByCurrentThread()
+ {
+ try {
+ return getLockService().isLockingSuspendedByCurrentThread();
+ }
+ catch (IllegalStateException e) {
+ lockCheckReadiness();
+ throw e;
+ }
+ }
+
+ /**
+ * If this region's scope is GLOBAL, get a distributed lock on the given key,
+ * and return the Lock. The sender is responsible for unlocking.
+ *
+ * @return the acquired Lock if the region is GLOBAL, otherwise null.
+ *
+ * @throws NullPointerException
+ * if key is null
+ */
+ private Lock getDistributedLockIfGlobal(Object key) throws TimeoutException
+ {
+ if (getScope().isGlobal()) {
+ if (isLockingSuspendedByCurrentThread())
+ return null;
+ long start = System.currentTimeMillis();
+ long timeLeft = getCache().getLockTimeout();
+ long lockTimeout = timeLeft;
+ StringId msg = null;
+ Object[] msgArgs = null;
+ while (timeLeft > 0 || lockTimeout == -1) {
+ this.cache.getCancelCriterion().checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ Lock dlock = getDistributedLock(key);
+ if (!dlock.tryLock(timeLeft, TimeUnit.SECONDS)) {
+ msg = LocalizedStrings.DistributedRegion_ATTEMPT_TO_ACQUIRE_DISTRIBUTED_LOCK_FOR_0_FAILED_AFTER_WAITING_1_SECONDS;
+ msgArgs = new Object[] {key, Long.valueOf((System.currentTimeMillis() - start) / 1000L)};
+ break;
+ }
+
+ return dlock;
+ }
+ catch (InterruptedException ex) {
+ interrupted = true;
+ this.cache.getCancelCriterion().checkCancelInProgress(ex);
+ // FIXME Why is it OK to keep going?
+ if (lockTimeout > -1) {
+ timeLeft = getCache().getLockTimeout()
+ - ((System.currentTimeMillis() - start) / 1000L);
+ }
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } // while
+ if (msg == null) {
+ msg = LocalizedStrings.DistributedRegion_TIMED_OUT_AFTER_WAITING_0_SECONDS_FOR_THE_DISTRIBUTED_LOCK_FOR_1;
+ msgArgs = new Object[] {Integer.valueOf(getCache().getLockTimeout()), key};
+ }
+ throw new TimeoutException(msg.toLocalizedString(msgArgs));
+ }
+ else {
+ return null;
+ }
+ }
+
+ /**
+ * Checks if the entry is a valid entry
+ *
+ * @return true if entry not null or entry is not removed
+ *
+ */
+ protected boolean checkEntryNotValid(RegionEntry mapEntry)
+ {
+ return (mapEntry == null || (mapEntry.isRemoved() && !mapEntry.isTombstone()));
+ }
+
+ /**
+ * Get the best iterator for iterating over the contents of this
+ * region. This method will either an iterator that uses hash
+ * ordering from the entry map, or, in the case of an overflow
+ * region, an iterator that iterates over the entries in disk order.
+ */
+ public Iterator getBestIterator(boolean includeValues) {
+ DiskRegion dr = this.getDiskRegion();
+
+ if (DiskPage.DISK_PAGE_SIZE > 0 && includeValues && dr != null) {
+ //Wait for the disk region to recover values first.
+ dr.waitForAsyncRecovery();
+ if(dr.getNumOverflowOnDisk() > 0) {
+ return new DiskSavyIterator();
+ }
+ }
+ return this.entries.regionEntries().iterator();
+ }
+
+// /**
+// * The maximum number of entries that can be put into the diskMap before
+// * some of them are read from disk and returned by this iterator.
+// * The larger this number the more memory this iterator is allowed to consume
+// * and the better it will do in optimally reading the pending entries.
+// */
+// static final long MAX_PENDING_ENTRIES = Long.getLong("gemfire.MAX_PENDING_ENTRIES", 1000000).longValue();
+ /**
+ * Should only be used if this region has entries on disk that are not in memory.
+ * This currently happens for overflow and for recovery when values are not recovered.
+ * The first iteration does a normal iteration of the regionEntries.
+ * But if it finds an entry that is currently only on disk
+ * it saves it in a list sorted by the location on disk.
+ * Once the regionEntries iterator has nothing more to iterate
+ * it starts iterating over, in disk order, the entries on disk.
+ */
+ private class DiskSavyIterator implements Iterator {
+ private boolean usingIt = true;
+ private Iterator> it = entries.regionEntries().iterator();
+ // iterator for nested ArrayLists
+ private Iterator subIt = null;
+ //private final ArrayList diskList = new ArrayList(/*@todo presize based on number of entries only on disk*/);
+ // value will be either RegionEntry or an ArrayList
+// private long pendingCount = 0;
+ private final java.util.TreeMap diskMap = new java.util.TreeMap();
+
+// /**
+// * used to iterate over the fullest pages at the time we have
+// * added MAX_PENDING_ENTRIES to diskMap;
+// */
+// private Iterator> sortedDiskIt;
+
+ public DiskSavyIterator() {
+ }
+
+ public boolean hasNext() {
+ boolean result;
+ if (this.subIt != null) {
+ result = this.subIt.hasNext();
+ if (!result) {
+ this.subIt = null;
+ } else {
+ return result;
+ }
+ }
+// if (this.sortedDiskIt != null) {
+// result = this.sortedDiskIt.hasNext();
+// if (!result) {
+// this.sortedDiskIt = null;
+// } else {
+// return result;
+// }
+// }
+ result = this.it.hasNext();
+ if (this.usingIt && !result) {
+ this.usingIt = false;
+// long start = System.currentTimeMillis();
+// Collections.sort(this.diskList);
+// long end = System.currentTimeMillis();
+ this.it = this.diskMap.values().iterator();
+ result = this.it.hasNext();
+ }
+ return result;
+ }
+
+ public RegionEntry next() {
+ for (;;) {
+ if (this.subIt != null) {
+ return this.subIt.next();
+// } else if (this.sortedDiskIt != null) {
+// Map.Entry me = this.sortedDiskIt.next();
+// // remove the page from the diskMap.
+// this.diskMap.remove(me.getKey());
+// Object v = me.getValue();
+// int size = 1;
+// if (v instanceof ArrayList) {
+// ArrayList al = (ArrayList)v;
+// size = al.size();
+// // set up the iterator to start returning the entries on that page
+// this.subIt = al.iterator();
+// v = this.subIt.next();
+// }
+
+// // decrement pendingCount by the number of entries on the page
+// this.pendingCount -= size;
+// // return the first region entry on this page
+// return v;
+ }
+ if (this.usingIt) {
+ RegionEntry re = (RegionEntry)this.it.next();
+ DiskPosition dp = new DiskPosition();
+ if (re.isOverflowedToDisk(DistributedRegion.this, dp)) {
+ // add dp to sorted list
+ DiskPage dPage = new DiskPage(dp);
+ Object v = this.diskMap.get(dPage);
+ if (v == null) {
+ this.diskMap.put(dPage, re);
+ } else if (v instanceof ArrayList) {
+ ArrayList al = (ArrayList)v;
+ al.add(re);
+ } else {
+ ArrayList al = new ArrayList();
+ al.add(v);
+ al.add(re);
+ this.diskMap.put(dPage, al);
+ }
+ if (!hasNext()) {
+ assert false; // must be true
+ }
+// this.pendingCount++;
+// if (this.usingIt && this.pendingCount >= MAX_PENDING_ENTRIES) {
+// // find the pages that have the most entries
+// int largestPage = 1;
+// ArrayList> largestPages
+// = new ArrayList>();
+// for (Map.Entry me: this.diskMap.entrySet()) {
+// int meSize = 1;
+// if (me.getValue() instanceof ArrayList) {
+// meSize = ((ArrayList)me.getValue()).size();
+// }
+// if (meSize > largestPage) {
+// largestPage = meSize;
+// largestPages.clear(); // throw away smaller pages
+// largestPages.add(me);
+// } else if (meSize == largestPage) {
+// largestPages.add(me);
+// } else {
+// // ignore this page
+// }
+// }
+// Collections.sort(largestPages, new Comparator
+// >() {
+// /**
+// * Note: this comparator imposes orderings that are inconsistent
+// * with equals.
+// */
+// public int compare(Map.Entry o1, Map.Entry o2) {
+// return o1.getKey().compareTo(o2.getKey());
+// }
+// });
+// this.sortedDiskIt = largestPages.iterator();
+// // loop around and fetch first value from sortedDiskIt
+// }
+ } else {
+ return re;
+ }
+ } else {
+ Object v = this.it.next();
+ if (v instanceof ArrayList) {
+ ArrayList al = (ArrayList)v;
+ this.subIt = al.iterator();
+ return this.subIt.next();
+ } else {
+ return (RegionEntry) v;
+ }
+ }
+ }
+ }
+
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ public static class DiskPosition implements Comparable {
+ private long oplogId;
+ private long offset;
+
+ DiskPosition() {
+ }
+ void setPosition(long oplogId, long offset) {
+ this.oplogId = oplogId;
+ this.offset = offset;
+ }
+
+ @Override
+ public int hashCode() {
+ return Long.valueOf(this.oplogId ^ this.offset).hashCode();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o instanceof DiskPosition) {
+ DiskPosition other = (DiskPosition)o;
+ return this.oplogId == other.oplogId && this.offset == other.offset;
+ } else {
+ return false;
+ }
+ }
+ public int compareTo(DiskPosition o) {
+ int result = Long.signum(this.oplogId - o.oplogId);
+ if (result == 0) {
+ result = Long.signum(this.offset - o.offset);
+ }
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ StringBuffer sb = new StringBuffer();
+ sb.append("<").append(this.oplogId).append(":").append(this.offset).append(">");
+ return sb.toString();
+ }
+ }
+ static class DiskPage extends DiskPosition {
+
+ static final long DISK_PAGE_SIZE = Long.getLong("gemfire.DISK_PAGE_SIZE", 8 * 1024L).longValue();
+
+ DiskPage(DiskPosition dp) {
+ this.setPosition(dp.oplogId, dp.offset / DISK_PAGE_SIZE);
+ }
+ }
+
+ /**
+ * Returns the lock lease value to use for DistributedLock and
+ * RegionDistributedLock. -1 is supported as non-expiring lock.
+ */
+ protected long getLockLeaseForLock()
+ {
+ if (getCache().getLockLease() == -1) {
+ return -1;
+ }
+ return getCache().getLockLease() * 1000;
+ }
+
+ /**
+ * Returns the lock timeout value to use for DistributedLock and
+ * RegionDistributedLock. -1 is supported as a lock that never times out.
+ */
+ protected long getLockTimeoutForLock(long time, TimeUnit unit)
+ {
+ if (time == -1) {
+ return -1;
+ }
+ return TimeUnit.MILLISECONDS.convert(time, unit);
+ }
+
+
+
+ /** ******************* DistributedLock ************************************* */
+
+ private class DistributedLock implements Lock
+ {
+ private final Object key;
+
+ public DistributedLock(Object key) {
+ this.key = key;
+ }
+
+ public void lock()
+ {
+ try {
+ boolean locked = basicTryLock(-1, TimeUnit.MILLISECONDS, false);
+ if (!locked) {
+ lockCheckReadiness();
+ }
+ Assert.assertTrue(locked, "Failed to acquire DistributedLock");
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ lockCheckReadiness();
+ Assert.assertTrue(false, "Failed to acquire DistributedLock");
+ }
+ }
+
+ public void lockInterruptibly() throws InterruptedException
+ {
+ try {
+ boolean locked = basicTryLock(-1, TimeUnit.MILLISECONDS, true);
+ if (!locked) {
+ lockCheckReadiness();
+ }
+ Assert.assertTrue(locked, "Failed to acquire DistributedLock");
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ }
+
+ public boolean tryLock()
+ {
+ try {
+ ReplyProcessor21.forceSevereAlertProcessing();
+ return getLockService().lock(this.key, 0, getLockLeaseForLock());
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ finally {
+ ReplyProcessor21.unforceSevereAlertProcessing();
+ }
+ }
+
+ public boolean tryLock(long time, TimeUnit unit)
+ throws InterruptedException {
+ return basicTryLock(time, unit, true);
+ }
+
+
+ private boolean basicTryLock(long time, TimeUnit unit, boolean interruptible)
+ throws InterruptedException {
+// if (Thread.interrupted()) throw new InterruptedException(); not necessary lockInterruptibly does this
+ final DM dm = getDistributionManager();
+
+ long start = System.currentTimeMillis();
+ long timeoutMS = getLockTimeoutForLock(time, unit);
+ long end;
+ if (timeoutMS < 0) {
+ timeoutMS = Long.MAX_VALUE;
+ end = Long.MAX_VALUE;
+ }
+ else {
+ end = start + timeoutMS;
+ }
+
+ long ackSAThreshold = getSystem().getConfig().getAckSevereAlertThreshold() * 1000;
+ boolean suspected = false;
+ boolean severeAlertIssued = false;
+ DistributedMember lockHolder = null;
+
+ long waitInterval;
+ long ackWaitThreshold;
+
+ if (ackSAThreshold > 0) {
+ ackWaitThreshold = getSystem().getConfig().getAckWaitThreshold() * 1000;
+ waitInterval = ackWaitThreshold;
+ }
+ else {
+ waitInterval = timeoutMS;
+ ackWaitThreshold = 0;
+ }
+
+ do {
+ try {
+ waitInterval = Math.min(end-System.currentTimeMillis(), waitInterval);
+ ReplyProcessor21.forceSevereAlertProcessing();
+ final boolean gotLock;
+ if (interruptible) {
+ gotLock = getLockService().lockInterruptibly(this.key,
+ waitInterval, getLockLeaseForLock());
+ }
+ else {
+ gotLock = getLockService().lock(this.key,
+ waitInterval, getLockLeaseForLock());
+ }
+ if (gotLock) {
+ return true;
+ }
+ if (ackSAThreshold > 0) {
+ long elapsed = System.currentTimeMillis() - start;
+ if (elapsed > ackWaitThreshold) {
+ if (!suspected) {
+ // start suspect processing on the holder of the lock
+ suspected = true;
+ severeAlertIssued = false; // in case this is a new lock holder
+ waitInterval = ackSAThreshold;
+ DLockRemoteToken remoteToken =
+ ((DLockService)getLockService()).queryLock(key);
+ lockHolder = remoteToken.getLessee();
+ if (lockHolder != null) {
+ dm.getMembershipManager()
+ .suspectMember(lockHolder,
+ "Has not released a global region entry lock in over "
+ + ackWaitThreshold / 1000 + " seconds");
+ }
+ }
+ else if (elapsed > ackSAThreshold) {
+ DLockRemoteToken remoteToken =
+ ((DLockService)getLockService()).queryLock(key);
+ if (lockHolder != null && remoteToken.getLessee() != null
+ && lockHolder.equals(remoteToken.getLessee())) {
+ if (!severeAlertIssued) {
+ severeAlertIssued = true;
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.DistributedRegion_0_SECONDS_HAVE_ELAPSED_WAITING_FOR_GLOBAL_REGION_ENTRY_LOCK_HELD_BY_1,
+ new Object[] {Long.valueOf(ackWaitThreshold+ackSAThreshold), lockHolder}));
+ }
+ }
+ else {
+ // the lock holder has changed
+ suspected = false;
+ waitInterval = ackWaitThreshold;
+ lockHolder = null;
+ }
+ }
+ }
+ } // ackSAThreshold processing
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ finally {
+ ReplyProcessor21.unforceSevereAlertProcessing();
+ }
+ } while (System.currentTimeMillis() < end);
+
+ return false;
+ }
+
+ public void unlock()
+ {
+ try {
+ ReplyProcessor21.forceSevereAlertProcessing();
+ getLockService().unlock(this.key);
+ if (!DistributedRegion.this.entries.containsKey(key)) {
+ getLockService().freeResources(key);
+ }
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ finally {
+ ReplyProcessor21.unforceSevereAlertProcessing();
+ }
+ }
+ public Condition newCondition() {
+ throw new UnsupportedOperationException(LocalizedStrings.DistributedRegion_NEWCONDITION_UNSUPPORTED.toLocalizedString());
+ }
+ }
+
+ /////////////////// RegionDistributedLock //////////////////
+
+ private class RegionDistributedLock implements Lock
+ {
+
+ public RegionDistributedLock() {
+ }
+
+ public void lock()
+ {
+ try {
+ boolean locked = getLockService().suspendLocking(-1);
+ Assert.assertTrue(locked, "Failed to acquire RegionDistributedLock");
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ }
+
+ public void lockInterruptibly() throws InterruptedException
+ {
+// if (Thread.interrupted()) throw new InterruptedException(); not necessary suspendLockingInterruptibly does this
+ try {
+ boolean locked = getLockService().suspendLockingInterruptibly(-1);
+ Assert.assertTrue(locked, "Failed to acquire RegionDistributedLock");
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ }
+
+ public boolean tryLock()
+ {
+ try {
+ return getLockService().suspendLocking(0);
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ }
+
+ public boolean tryLock(long time, TimeUnit unit)
+ throws InterruptedException
+ {
+// if (Thread.interrupted()) throw new InterruptedException(); not necessary suspendLockingINterruptibly does this
+ try {
+ return getLockService().suspendLockingInterruptibly(
+ getLockTimeoutForLock(time, unit));
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ }
+
+ public void unlock()
+ {
+ try {
+ getLockService().resumeLocking();
+ }
+ catch (IllegalStateException ex) {
+ lockCheckReadiness();
+ throw ex;
+ }
+ }
+ public Condition newCondition() {
+ throw new UnsupportedOperationException(LocalizedStrings.DistributedRegion_NEWCONDITION_UNSUPPORTED.toLocalizedString());
+ }
+ }
+
+ // - add in region locking for destroy and invalidate...
+
+ /**
+ * If this region's scope is GLOBAL, get the region distributed lock. The
+ * sender is responsible for unlocking.
+ *
+ * @return the acquired Lock if the region is GLOBAL and not already suspend,
+ * otherwise null.
+ */
+ Lock getRegionDistributedLockIfGlobal() throws TimeoutException
+ {
+ if (getScope().isGlobal()) {
+ if (isLockingSuspendedByCurrentThread())
+ return null;
+ Lock dlock = getRegionDistributedLock();
+ dlock.lock();
+ return dlock;
+ }
+ return null;
+ }
+
+ /*
+ * void localDestroyRegion(Object aCallbackArgument) { try { Lock dlock =
+ * this.getRegionDistributedLockIfGlobal(); try {
+ * super.localDestroyRegion(aCallbackArgument); } finally { if (dlock != null) {
+ * dlock.unlock(); } } } catch (TimeoutException e) { throw new
+ * GemFireCacheException("localDestroyRegion timed out", e); } }
+ *
+ * void destroyRegion(Object aCallbackArgument) throws CacheWriterException,
+ * TimeoutException { Lock dlock = this.getRegionDistributedLockIfGlobal();
+ * try { super.destroyRegion(aCallbackArgument); } finally { if (dlock !=
+ * null) { dlock.unlock(); } } }
+ *
+ * void invalidateRegion(Object aCallbackArgument) throws TimeoutException {
+ * Lock dlock = this.getRegionDistributedLockIfGlobal(); try {
+ * super.invalidateRegion(aCallbackArgument); } finally { if (dlock != null) {
+ * dlock.unlock(); } } }
+ */
+
+
+ /**
+ * Distribute the PutAllOp.
+ * This implementation distributes it to peers.
+ * @since 5.7
+ */
+ @Override
+ public void postPutAllSend(DistributedPutAllOperation putAllOp, VersionedObjectList successfulPuts) {
+ if (putAllOp.putAllDataSize > 0) {
+ putAllOp.distribute();
+ } else {
+ if (logger.isDebugEnabled()) {
+ logger.debug("DR.postPutAll: no data to distribute");
+ }
+ }
+ }
+ @Override
+ public void postRemoveAllSend(DistributedRemoveAllOperation op, VersionedObjectList successfulOps) {
+ if (op.removeAllDataSize > 0) {
+ op.distribute();
+ } else {
+ getCache().getLoggerI18n().fine("DR.postRemoveAll: no data to distribute");
+ }
+ }
+
+ @Override
+ public VersionedObjectList basicPutAll(final Map, ?> map,
+ final DistributedPutAllOperation putAllOp, final Map retryVersions) {
+ Lock dlock = this.getRegionDistributedLockIfGlobal();
+ try {
+ return super.basicPutAll(map, putAllOp, retryVersions);
+ } finally {
+ if (dlock != null) {
+ dlock.unlock();
+ }
+ }
+ }
+
+ @Override
+ public VersionedObjectList basicRemoveAll(final Collection keys,
+ final DistributedRemoveAllOperation removeAllOp, final ArrayList retryVersions) {
+ Lock dlock = this.getRegionDistributedLockIfGlobal();
+ try {
+ return super.basicRemoveAll(keys, removeAllOp, retryVersions);
+ } finally {
+ if (dlock != null) {
+ dlock.unlock();
+ }
+ }
+ }
+
+
+ /** Returns true if any required roles are currently missing */
+ boolean isMissingRequiredRoles()
+ {
+ return this.isMissingRequiredRoles;
+ }
+
+ /**
+ * Returns the missing required roles after waiting up to the timeout
+ *
+ * @throws IllegalStateException
+ * if region is not configured with required roles
+ * @throws InterruptedException TODO-javadocs
+ */
+ public Set waitForRequiredRoles(long timeout) throws InterruptedException
+ {
+ if (Thread.interrupted()) throw new InterruptedException();
+ checkReadiness();
+ if (!getMembershipAttributes().hasRequiredRoles()) {
+ throw new IllegalStateException(LocalizedStrings.DistributedRegion_REGION_HAS_NOT_BEEN_CONFIGURED_WITH_REQUIRED_ROLES.toLocalizedString());
+ }
+ if (!this.isMissingRequiredRoles) { // should we delete this check?
+ if (logger.isDebugEnabled()) {
+ logger.debug("No missing required roles to wait for.");
+ }
+ return Collections.EMPTY_SET; // early-out: no missing required roles
+ }
+ if (timeout != 0) { // if timeout is zero then fall through past waits
+ if (timeout == -1) { // infinite timeout
+ while (this.isMissingRequiredRoles) {
+ checkReadiness();
+ this.cache.getCancelCriterion().checkCancelInProgress(null); // bail if distribution has stopped
+ synchronized (this.missingRequiredRoles) {
+ // one more check while synced
+ if (this.isMissingRequiredRoles) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("About to wait for missing required roles.");
+ }
+ // TODO an infinite wait here might be a problem...
+ this.missingRequiredRoles.wait(); // spurious wakeup ok
+ }
+ }
+ }
+ }
+ else { // use the timeout
+ long endTime = System.currentTimeMillis() + timeout;
+ while (this.isMissingRequiredRoles) {
+ checkReadiness();
+ this.cache.getCancelCriterion().checkCancelInProgress(null); // bail if distribution has stopped
+ synchronized (this.missingRequiredRoles) {
+ // one more check while synced
+ if (this.isMissingRequiredRoles) {
+ long timeToWait = endTime - System.currentTimeMillis();
+ if (timeToWait > 0) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("About to wait up to {} milliseconds for missing required roles.", timeToWait);
+ }
+ this.missingRequiredRoles.wait(timeToWait); // spurious wakeup ok
+ }
+ else {
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
+ // check readiness again: thread may have been notified at destroy time
+ checkReadiness();
+ if (this.isMissingRequiredRoles) {
+ // sync on missingRequiredRoles to prevent mods to required role status...
+ synchronized (this.missingRequiredRoles) {
+ return Collections.unmodifiableSet(new HashSet(
+ this.missingRequiredRoles));
+ }
+ }
+ else {
+ return Collections.EMPTY_SET;
+ }
+ }
+
+ /** Returns true if the role is currently present this region's membership. */
+ public boolean isRoleInRegionMembership(Role role)
+ {
+ checkReadiness();
+ return basicIsRoleInRegionMembership(role);
+ }
+
+ protected boolean basicIsRoleInRegionMembership(Role role)
+ {
+ if (getSystem().getDistributedMember().getRoles().contains(role)) {
+ // since we are playing the role
+ return true;
+ }
+ Set members = this.distAdvisor.adviseGeneric();
+ for (Iterator iter = members.iterator(); iter.hasNext();) {
+ DistributedMember member = (DistributedMember)iter.next();
+ Set roles = member.getRoles();
+ if (roles.contains(role)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public void remoteRegionInitialized(CacheProfile profile) {
+ synchronized(this.advisorListener) {
+ if (this.advisorListener.members == null && hasListener()) {
+ Object callback = TEST_HOOK_ADD_PROFILE? profile : null;
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.REGION_CREATE, callback, true, profile.peerMemberId);
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REMOTE_REGION_CREATE,
+ event);
+ }
+ }
+ }
+
+ @Override
+ protected void removeSenderFromAdvisor(InternalDistributedMember sender, int serial, boolean regionDestroyed)
+ {
+ getDistributionAdvisor().removeIdWithSerial(sender, serial, regionDestroyed);
+ }
+
+ /** doesn't throw RegionDestroyedException, used by CacheDistributionAdvisor */
+ public DistributionAdvisee getParentAdvisee() {
+ return (DistributionAdvisee) basicGetParentRegion();
+ }
+
+ /**
+ * Used to get membership events from our advisor to implement
+ * RegionMembershipListener invocations.
+ *
+ * @since 5.0
+ */
+ protected class AdvisorListener implements MembershipListener
+ {
+ private Set members = new HashSet();
+
+ protected boolean destroyed = false;
+
+ protected synchronized void addMembers(Set newMembers)
+ {
+ this.members.addAll(newMembers);
+ }
+
+ protected synchronized Set getInitialMembers()
+ {
+ Set initMembers = this.members;
+ this.members = null;
+ return initMembers;
+ }
+
+ public void quorumLost(Set failures, List remaining) {
+ }
+
+ public void memberSuspect(InternalDistributedMember id,
+ InternalDistributedMember whoSuspected, String reason) {
+ }
+
+ /** called when membership listeners are added after region creation */
+ protected synchronized void initRMLWrappers() {
+ Set membersWithThisRegion = DistributedRegion.this.distAdvisor.adviseGeneric();
+ initPostCreateRegionMembershipListeners(membersWithThisRegion);
+ }
+
+ public synchronized void memberJoined(InternalDistributedMember id)
+ {
+ if (this.destroyed) {
+ return;
+ }
+ if (this.members != null) {
+ this.members.add(id);
+ }
+ // bug #44684 - do not notify listener of create until remote member is initialized
+// if (this.members == null && hasListener()) {
+// RegionEventImpl event = new RegionEventImpl(DistributedRegion.this,
+// Operation.REGION_CREATE, null, true, id);
+// dispatchListenerEvent(EnumListenerEvent.AFTER_REMOTE_REGION_CREATE,
+// event);
+// }
+ if (getMembershipAttributes().hasRequiredRoles()) {
+ // newlyAcquiredRoles is used for intersection and RoleEvent
+ Set newlyAcquiredRoles = Collections.EMPTY_SET;
+ synchronized (missingRequiredRoles) {
+ if (isMissingRequiredRoles) {
+ Set roles = id.getRoles();
+ newlyAcquiredRoles = new HashSet(missingRequiredRoles);
+ newlyAcquiredRoles.retainAll(roles); // find the intersection
+ if (!newlyAcquiredRoles.isEmpty()) {
+ if (DistributedRegion.this.rmq != null) {
+ Iterator it = newlyAcquiredRoles.iterator();
+ final DM dm = getDistributionManager();
+ while (it.hasNext()) {
+ getCache().getCancelCriterion().checkCancelInProgress(null);
+ final Role role = (Role)it.next();
+ try {
+ // do this in the waiting pool to make it async
+ // @todo darrel/klund: add a single serial executor for
+ // queue flush
+ dm.getWaitingThreadPool().execute(new Runnable() {
+ public void run()
+ {
+ DistributedRegion.this.rmq.roleReady(role);
+ }
+ });
+ break;
+ }
+ catch (RejectedExecutionException ex) {
+ throw ex;
+ }
+ } // while
+ }
+ missingRequiredRoles.removeAll(newlyAcquiredRoles);
+ if (this.members == null && missingRequiredRoles.isEmpty()) {
+ isMissingRequiredRoles = false;
+ getCachePerfStats().incReliableRegionsMissing(-1);
+ if (getMembershipAttributes().getLossAction().isAllAccess())
+ getCachePerfStats().incReliableRegionsMissingFullAccess(-1); // rahul
+ else if (getMembershipAttributes().getLossAction()
+ .isLimitedAccess())
+ getCachePerfStats()
+ .incReliableRegionsMissingLimitedAccess(-1);
+ else if (getMembershipAttributes().getLossAction().isNoAccess())
+ getCachePerfStats().incReliableRegionsMissingNoAccess(-1);
+
+ boolean async = resumeReliability(id, newlyAcquiredRoles);
+ if (async) {
+ this.destroyed = true;
+ }
+ }
+ }
+ }
+ if (!this.destroyed) {
+ // any number of threads may be waiting on missingRequiredRoles
+ missingRequiredRoles.notifyAll();
+ }
+ }
+ if (!this.destroyed && this.members == null && hasListener()) {
+ if (!newlyAcquiredRoles.isEmpty()) {
+ // fire afterRoleGain event
+ RoleEventImpl relEvent = new RoleEventImpl(DistributedRegion.this,
+ Operation.REGION_CREATE, null, true, id, newlyAcquiredRoles);
+ dispatchListenerEvent(EnumListenerEvent.AFTER_ROLE_GAIN, relEvent);
+ }
+ }
+ }
+ }
+
+ public synchronized void memberDeparted(InternalDistributedMember id,
+ boolean crashed)
+ {
+ if (this.destroyed) {
+ return;
+ }
+ if (this.members != null) {
+ this.members.remove(id);
+ }
+ if (this.members == null && hasListener()) {
+ RegionEventImpl event = new RegionEventImpl(DistributedRegion.this,
+ Operation.REGION_CLOSE, null, true, id);
+ if (crashed) {
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REMOTE_REGION_CRASH,
+ event);
+ }
+ else {
+ // @todo darrel: it would be nice to know if what actual op was done
+ // could be close, local destroy, or destroy (or load snap?)
+ if (DestroyRegionOperation.isRegionDepartureNotificationOk()) {
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REMOTE_REGION_DEPARTURE, event);
+ }
+ }
+ }
+ if (getMembershipAttributes().hasRequiredRoles()) {
+ Set newlyMissingRoles = Collections.EMPTY_SET;
+ synchronized (missingRequiredRoles) {
+ Set roles = id.getRoles();
+ for (Iterator iter = roles.iterator(); iter.hasNext();) {
+ Role role = (Role)iter.next();
+ if (getMembershipAttributes().getRequiredRoles().contains(role)
+ && !basicIsRoleInRegionMembership(role)) {
+ if (newlyMissingRoles == Collections.EMPTY_SET) {
+ newlyMissingRoles = new HashSet();
+ }
+ newlyMissingRoles.add(role);
+ if (this.members == null && !isMissingRequiredRoles) {
+ isMissingRequiredRoles = true;
+ getCachePerfStats().incReliableRegionsMissing(1);
+ if (getMembershipAttributes().getLossAction().isAllAccess())
+ getCachePerfStats().incReliableRegionsMissingFullAccess(1); // rahul
+ else if (getMembershipAttributes().getLossAction()
+ .isLimitedAccess())
+ getCachePerfStats().incReliableRegionsMissingLimitedAccess(1);
+ else if (getMembershipAttributes().getLossAction().isNoAccess())
+ getCachePerfStats().incReliableRegionsMissingNoAccess(1);
+
+ boolean async = lostReliability(id, newlyMissingRoles);
+ if (async) {
+ this.destroyed = true;
+ }
+ }
+ }
+ }
+ if (!this.destroyed) {
+ missingRequiredRoles.addAll(newlyMissingRoles);
+ // any number of threads may be waiting on missingRequiredRoles...
+ missingRequiredRoles.notifyAll();
+ }
+ }
+ if (!this.destroyed && this.members == null && hasListener()) {
+ if (!newlyMissingRoles.isEmpty()) {
+ // fire afterRoleLoss event
+ RoleEventImpl relEvent = new RoleEventImpl(DistributedRegion.this,
+ Operation.REGION_CLOSE, null, true, id, newlyMissingRoles);
+ dispatchListenerEvent(EnumListenerEvent.AFTER_ROLE_LOSS, relEvent);
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Used to bootstrap txState.
+ * @param key
+ * @return distributedRegions,
+ * member with parimary bucket for partitionedRegions
+ */
+ @Override
+ public DistributedMember getOwnerForKey(KeyInfo key) {
+ //Asif: fix for sqlfabric bug 42266
+ assert !this.isInternalRegion() || this.isMetaRegionWithTransactions();
+ if (!this.getAttributes().getDataPolicy().withStorage()
+ || (this.concurrencyChecksEnabled && this.getAttributes()
+ .getDataPolicy() == DataPolicy.NORMAL)) {
+ // execute on random replicate
+ return getRandomReplicate();
+ }
+ // if we are non-persistent, forward transactions to
+ // a persistent member
+ if (this.concurrencyChecksEnabled && !generateVersionTag) {
+ return getRandomPersistentReplicate();
+ }
+ return super.getOwnerForKey(key);
+ }
+
+ /**
+ * Execute the provided named function in all locations that contain the given
+ * keys. So function can be executed on just one fabric node, executed in
+ * parallel on a subset of nodes in parallel across all the nodes.
+ *
+ * @param function
+ * @param args
+ * @since 5.8
+ */
+ @Override
+ public ResultCollector executeFunction(
+ final DistributedRegionFunctionExecutor execution,
+ final Function function, final Object args,
+ final ResultCollector rc, final Set filter,
+ final ServerToClientFunctionResultSender sender) {
+ DistributedMember target = getTransactionalNode();
+ if (target != null) {
+ if (target.equals(getMyId())) {
+ return executeLocally(execution, function, args, 0, rc, filter, sender);
+ }
+ return executeOnReplicate(execution, function, args, rc, filter, target);
+ } else if (this.getAttributes().getDataPolicy().withReplication()
+ || this.getAttributes().getDataPolicy().withPreloaded()) {
+ // execute locally
+ final Set singleMember = Collections
+ .singleton(getMyId());
+ execution.validateExecution(function, singleMember);
+ execution.setExecutionNodes(singleMember);
+ return executeLocally(execution, function, args, 0, rc, filter, sender);
+ } else {
+ // select a random replicate
+ target = getRandomReplicate();
+ if (target == null) {
+ throw new FunctionException(LocalizedStrings
+ .DistributedRegion_NO_REPLICATED_REGION_FOUND_FOR_EXECUTING_FUNCTION_0
+ .toLocalizedString(function.getId()));
+ }
+ }
+ final LocalResultCollector, ?> localRC = execution
+ .getLocalResultCollector(function, rc);
+ return executeOnReplicate(execution, function, args, localRC, filter, target);
+ }
+
+ private ResultCollector executeOnReplicate(
+ final DistributedRegionFunctionExecutor execution,
+ final Function function, final Object args, ResultCollector rc,
+ final Set filter, final DistributedMember target) {
+ final Set singleMember = Collections.singleton(target);
+ execution.validateExecution(function, singleMember);
+ execution.setExecutionNodes(singleMember);
+
+ HashMap memberArgs = new HashMap();
+ memberArgs.put((InternalDistributedMember)target, execution.getArgumentsForMember(target.getId()));
+
+ ResultSender resultSender = new DistributedRegionFunctionResultSender(null, rc,
+ function, execution.getServerResultSender());
+
+ DistributedRegionFunctionResultWaiter waiter = new DistributedRegionFunctionResultWaiter(
+ this.getSystem(), this.getFullPath(), rc, function, filter,
+ Collections.singleton(target), memberArgs, resultSender);
+
+ rc = waiter.getFunctionResultFrom(Collections.singleton(target),
+ function, execution);
+ return rc;
+ }
+
+ /**
+ * @return the node which a transaction is already is progress, null otherwise
+ */
+ private DistributedMember getTransactionalNode() {
+ if (cache.getTxManager().getTXState() != null) {
+ return cache.getTxManager().getTXState().getTarget();
+ }
+ return null;
+ }
+
+ /**
+ * Implementation of {@link ProfileVisitor} that selects a random replicated
+ * member from the available ones for this region.
+ */
+ static final class GetRandomReplicate implements
+ ProfileVisitor {
+
+ private boolean onlyPersistent = false;
+
+ InternalDistributedMember member = null;
+
+ private int randIndex = -1;
+
+ public GetRandomReplicate() {
+ }
+
+ public GetRandomReplicate(boolean onlyPersistent) {
+ this.onlyPersistent = onlyPersistent;
+ }
+
+ public boolean visit(DistributionAdvisor advisor, Profile profile,
+ int profileIndex, int numProfiles, DistributedMember member) {
+ final CacheProfile cp = (CacheProfile)profile;
+ if (this.randIndex < 0) {
+ this.randIndex = PartitionedRegion.rand.nextInt(numProfiles);
+ }
+ if (cp.dataPolicy.withReplication() && cp.regionInitialized
+ && !cp.memberUnInitialized) {
+ if (onlyPersistent && !cp.dataPolicy.withPersistence()) {
+ return true;
+ }
+ // store the last replicated member in any case since in the worst case
+ // there may be no replicated node after "randIndex" in which case the
+ // last visited member will be used
+ this.member = cp.getDistributedMember();
+ if (profileIndex >= this.randIndex) {
+ return false;
+ }
+ }
+ return true;
+ }
+ }
+
+ /**
+ * @return a random replicate, null if there are none
+ */
+ public InternalDistributedMember getRandomReplicate() {
+ /* [sumedh] The old code causes creation of a unnecessary HashSet
+ * and population with all replicates (which may be large), then
+ * copy into an array and then selection of a random one from that.
+ * The new approach uses a much more efficient visitor instead.
+ Set replicates = this.getCacheDistributionAdvisor().adviseReplicates();
+ if (replicates.isEmpty()) {
+ return null;
+ }
+ return (InternalDistributedMember)(replicates
+ .toArray()[new Random().nextInt(replicates.size())]);
+ */
+ final GetRandomReplicate getReplicate = new GetRandomReplicate();
+ this.getCacheDistributionAdvisor().accept(getReplicate, null);
+ return getReplicate.member;
+ }
+
+ /**
+ * @return a random persistent replicate, null if there is none
+ */
+ public InternalDistributedMember getRandomPersistentReplicate() {
+ final GetRandomReplicate getPersistentReplicate = new GetRandomReplicate(true);
+ this.getCacheDistributionAdvisor().accept(getPersistentReplicate, null);
+ return getPersistentReplicate.member;
+ }
+
+ void executeOnRegion(DistributedRegionFunctionStreamingMessage msg,
+ final Function function, final Object args, int prid,
+ final Set filter, boolean isReExecute) throws IOException {
+ final DM dm = getDistributionManager();
+ ResultSender resultSender = new DistributedRegionFunctionResultSender(dm, msg, function);
+ final RegionFunctionContextImpl context = new RegionFunctionContextImpl(
+ function.getId(), this, args, filter, null, null, resultSender,
+ isReExecute);
+ FunctionStats stats = FunctionStats.getFunctionStats(function.getId(), dm.getSystem());
+ try {
+ long start = stats.startTime();
+ stats.startFunctionExecution(function.hasResult());
+ function.execute(context);
+ stats.endFunctionExecution(start,function.hasResult());
+ }
+ catch (FunctionException functionException) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("FunctionException occured on remote node while executing Function: {}", function.getId(), functionException);
+ }
+ stats.endFunctionExecutionWithException(function.hasResult());
+ throw functionException;
+ }
+ catch (CacheClosedException cacheClosedexception) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("CacheClosedException occured on remote node while executing Function: {}", function.getId(), cacheClosedexception);
+ }
+ throw cacheClosedexception;
+ }
+ catch (Exception exception) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Exception occured on remote node while executing Function: {}", function.getId(), exception);
+ }
+ stats.endFunctionExecutionWithException(function.hasResult());
+ throw new FunctionException(exception);
+ }
+ }
+
+ ResultCollector executeLocally(
+ final DistributedRegionFunctionExecutor execution,
+ final Function function, final Object args, int prid,
+ final ResultCollector rc, final Set filter,
+ final ServerToClientFunctionResultSender sender) {
+ final LocalResultCollector, ?> localRC = execution
+ .getLocalResultCollector(function, rc);
+ final DM dm = getDistributionManager();
+ final DistributedRegionFunctionResultSender resultSender = new DistributedRegionFunctionResultSender(
+ dm, localRC, function, sender);
+ final RegionFunctionContextImpl context = new RegionFunctionContextImpl(
+ function.getId(), DistributedRegion.this, args, filter, null, null,
+ resultSender, execution.isReExecute());
+ execution.executeFunctionOnLocalNode(function, context, resultSender, dm, isTX());
+ return localRC;
+ }
+
+ @Override
+ protected void setMemoryThresholdFlag(MemoryEvent event) {
+ Set others = getCacheDistributionAdvisor().adviseGeneric();
+
+ if (event.isLocal() || others.contains(event.getMember())) {
+ if (event.getState().isCritical()
+ && !event.getPreviousState().isCritical()
+ && (event.getType() == ResourceType.HEAP_MEMORY || (event.getType() == ResourceType.OFFHEAP_MEMORY && getOffHeap()))) {
+ setMemoryThresholdReachedCounterTrue(event.getMember());
+ } else if (!event.getState().isCritical()
+ && event.getPreviousState().isCritical()
+ && (event.getType() == ResourceType.HEAP_MEMORY || (event.getType() == ResourceType.OFFHEAP_MEMORY && getOffHeap()))) {
+ removeMemberFromCriticalList(event.getMember());
+ }
+ }
+ }
+
+ @Override
+ public void removeMemberFromCriticalList(DistributedMember member) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("DR: removing member {} from critical member list", member);
+ }
+ synchronized(this.memoryThresholdReachedMembers) {
+ this.memoryThresholdReachedMembers.remove(member);
+ if (this.memoryThresholdReachedMembers.size() == 0) {
+ memoryThresholdReached.set(false);
+ }
+ }
+ }
+
+ @Override
+ public Set getMemoryThresholdReachedMembers() {
+ synchronized (this.memoryThresholdReachedMembers) {
+ return Collections.unmodifiableSet(this.memoryThresholdReachedMembers);
+ }
+ }
+
+ @Override
+ public void initialCriticalMembers(boolean localMemoryIsCritical,
+ Set critialMembers) {
+ Set others = getCacheDistributionAdvisor().adviseGeneric();
+ for (InternalDistributedMember idm: critialMembers) {
+ if (others.contains(idm)) {
+ setMemoryThresholdReachedCounterTrue(idm);
+ }
+ }
+ }
+
+ /**
+ * @param idm member whose threshold has been exceeded
+ */
+ private void setMemoryThresholdReachedCounterTrue(final DistributedMember idm) {
+ synchronized(this.memoryThresholdReachedMembers) {
+ this.memoryThresholdReachedMembers.add(idm);
+ if (this.memoryThresholdReachedMembers.size() > 0) {
+ memoryThresholdReached.set(true);
+ }
+ }
+ }
+
+ /**
+ * Fetch Version for the given key from a remote replicate member.
+ * @param key
+ * @throws EntryNotFoundException if the entry is not found on replicate member
+ * @return VersionTag for the key
+ */
+ protected VersionTag fetchRemoteVersionTag(Object key) {
+ VersionTag tag = null;
+ assert this.dataPolicy != DataPolicy.REPLICATE;
+ TransactionId txId = cache.getCacheTransactionManager().suspend();
+ try {
+ boolean retry = true;
+ InternalDistributedMember member = getRandomReplicate();
+ while (retry) {
+ try {
+ if (member == null) {
+ break;
+ }
+ FetchVersionResponse response = RemoteFetchVersionMessage.send(member, this, key);
+ tag = response.waitForResponse();
+ retry = false;
+ } catch (RemoteOperationException e) {
+ member = getRandomReplicate();
+ if (member != null) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Retrying RemoteFetchVersionMessage on member:{}", member);
+ }
+ }
+ }
+ }
+ } finally {
+ if (txId != null) {
+ cache.getCacheTransactionManager().resume(txId);
+ }
+ }
+ return tag;
+ }
+
+ /**
+ * Test hook for bug 48578. Returns true if it sees a net loader.
+ * Returns false if it does not have one.
+ */
+ public boolean hasNetLoader() {
+ return this.hasNetLoader(getCacheDistributionAdvisor());
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
new file mode 100644
index 000000000000..41c7b88acaee
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -0,0 +1,3140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CopyHelper;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.DeltaSerializationException;
+import com.gemstone.gemfire.GemFireIOException;
+import com.gemstone.gemfire.InvalidDeltaException;
+import com.gemstone.gemfire.SerializationException;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.cache.EntryEvent;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.EntryOperation;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.SerializedCacheValue;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
+import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
+import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
+import com.gemstone.gemfire.cache.query.internal.index.IndexUtils;
+import com.gemstone.gemfire.cache.util.TimestampedEntryEvent;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionMessage;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.ByteArrayDataInput;
+import com.gemstone.gemfire.internal.DSFIDFactory;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.Sendable;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
+import com.gemstone.gemfire.internal.cache.delta.Delta;
+import com.gemstone.gemfire.internal.cache.lru.Sizeable;
+import com.gemstone.gemfire.internal.cache.partitioned.PartitionMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.PutMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tx.DistTxKeyInfo;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventCallbackArgument;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
+import com.gemstone.gemfire.internal.offheap.Releasable;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.internal.offheap.annotations.Released;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_OLD_VALUE;
+
+import com.gemstone.gemfire.internal.util.ArrayUtils;
+import com.gemstone.gemfire.internal.util.BlobHelper;
+import com.gemstone.gemfire.pdx.internal.PeerTypeRegistration;
+
+/**
+ * Implementation of an entry event
+ */
+// must be public for DataSerializableFixedID
+public class EntryEventImpl
+ implements EntryEvent, InternalCacheEvent, DataSerializableFixedID, EntryOperation
+ , Releasable
+{
+ private static final Logger logger = LogService.getLogger();
+
+ // PACKAGE FIELDS //
+ public transient LocalRegion region;
+ private transient RegionEntry re;
+
+ protected KeyInfo keyInfo;
+
+ //private long eventId;
+ /** the event's id. Scoped by distributedMember. */
+ protected EventID eventID;
+
+ private Object newValue = null;
+ /**
+ * If we ever serialize the new value then it should be
+ * stored in this field in case we need the serialized form
+ * again later. This was added to fix bug 43781.
+ * Note that we also have the "newValueBytes" field.
+ * But it is only non-null if setSerializedNewValue was called.
+ */
+ private byte[] cachedSerializedNewValue = null;
+ @Retained(ENTRY_EVENT_OLD_VALUE)
+ private Object oldValue = null;
+ protected Delta delta = null;
+
+ protected short eventFlags = 0x0000;
+
+ protected TXId txId = null;
+
+ protected Operation op;
+
+ /* To store the operation/modification type */
+ private transient EnumListenerEvent eventType;
+
+ /**
+ * This field will be null unless this event is used for a putAll operation.
+ *
+ * @since 5.0
+ */
+ protected transient DistributedPutAllOperation putAllOp;
+
+ /**
+ * This field will be null unless this event is used for a removeAll operation.
+ *
+ * @since 8.1
+ */
+ protected transient DistributedRemoveAllOperation removeAllOp;
+
+ /**
+ * The member that originated this event
+ *
+ * @since 5.0
+ */
+ protected DistributedMember distributedMember;
+
+
+ /**
+ * transient storage for the message that caused the event
+ */
+ transient DistributionMessage causedByMessage;
+
+
+ //private static long eventID = 0;
+
+ /**
+ * The originating membershipId of this event.
+ *
+ * @since 5.1
+ */
+ protected ClientProxyMembershipID context = null;
+
+ /**
+ * A custom context object that can be used for any other contextual
+ * information. Currently used by SQL Fabric to pass around evaluated rows
+ * from raw byte arrays and routing object.
+ */
+ private transient Object contextObj = null;
+
+ /**
+ * this holds the bytes representing the change in value effected by this
+ * event. It is used when the value implements the Delta interface.
+ */
+ private byte[] deltaBytes = null;
+
+
+ /** routing information for cache clients for this event */
+ private FilterInfo filterInfo;
+
+ /**new value stored in serialized form*/
+ protected byte[] newValueBytes;
+ /**old value stored in serialized form*/
+ private byte[] oldValueBytes;
+
+ /** version tag for concurrency checks */
+ protected VersionTag versionTag;
+
+ /** boolean to indicate that this operation should be optimized by not fetching from HDFS*/
+ private transient boolean fetchFromHDFS = true;
+
+ private transient boolean isPutDML = false;
+
+ /** boolean to indicate that the RegionEntry for this event was loaded from HDFS*/
+ private transient boolean loadedFromHDFS= false;
+
+ private transient boolean isCustomEviction = false;
+
+ /** boolean to indicate that the RegionEntry for this event has been evicted*/
+ private transient boolean isEvicted = false;
+
+ private transient boolean isPendingSecondaryExpireDestroy = false;
+
+ public final static Object SUSPECT_TOKEN = new Object();
+
+ public EntryEventImpl() {
+ }
+
+ /**
+ * create a new entry event that will be used for conveying version information
+ * and anything else of use while processing another event
+ * @return the empty event object
+ */
+ @Retained
+ public static EntryEventImpl createVersionTagHolder() {
+ return createVersionTagHolder(null);
+ }
+
+ /**
+ * create a new entry event that will be used for conveying version information
+ * and anything else of use while processing another event
+ * @return the empty event object
+ */
+ @Retained
+ public static EntryEventImpl createVersionTagHolder(VersionTag tag) {
+ EntryEventImpl result = new EntryEventImpl();
+ result.setVersionTag(tag);
+ result.disallowOffHeapValues();
+ return result;
+ }
+
+ /**
+ * Reads the contents of this message from the given input.
+ */
+ public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+ this.eventID = (EventID)DataSerializer.readObject(in);
+ Object key = DataSerializer.readObject(in);
+ Object value = DataSerializer.readObject(in);
+ this.keyInfo = new KeyInfo(key, value, null);
+ this.op = Operation.fromOrdinal(in.readByte());
+ this.eventFlags = in.readShort();
+ this.keyInfo.setCallbackArg(DataSerializer.readObject(in));
+ this.txId = (TXId)DataSerializer.readObject(in);
+
+ if (in.readBoolean()) { // isDelta
+ this.delta = (Delta)DataSerializer.readObject(in);
+ }
+ else {
+ // OFFHEAP Currently values are never deserialized to off heap memory. If that changes then this code needs to change.
+ if (in.readBoolean()) { // newValueSerialized
+ this.newValueBytes = DataSerializer.readByteArray(in);
+ this.cachedSerializedNewValue = this.newValueBytes;
+ this.newValue = CachedDeserializableFactory.create(this.newValueBytes);
+ }
+ else {
+ this.newValue = DataSerializer.readObject(in);
+ }
+ }
+
+ // OFFHEAP Currently values are never deserialized to off heap memory. If that changes then this code needs to change.
+ if (in.readBoolean()) { // oldValueSerialized
+ this.oldValueBytes = DataSerializer.readByteArray(in);
+ this.oldValue = CachedDeserializableFactory.create(this.oldValueBytes);
+ }
+ else {
+ this.oldValue = DataSerializer.readObject(in);
+ }
+ this.distributedMember = DSFIDFactory.readInternalDistributedMember(in);
+ this.context = ClientProxyMembershipID.readCanonicalized(in);
+ this.tailKey = DataSerializer.readLong(in);
+ }
+
+ @Retained
+ protected EntryEventImpl(LocalRegion region, Operation op, Object key,
+ boolean originRemote, DistributedMember distributedMember,
+ boolean generateCallbacks, boolean fromRILocalDestroy) {
+ this.region = region;
+ this.op = op;
+ this.keyInfo = this.region.getKeyInfo(key);
+ setOriginRemote(originRemote);
+ setGenerateCallbacks(generateCallbacks);
+ this.distributedMember = distributedMember;
+ setFromRILocalDestroy(fromRILocalDestroy);
+ }
+
+ /**
+ * Doesn't specify oldValue as this will be filled in later as part of an
+ * operation on the region, or lets it default to null.
+ */
+ @Retained
+ protected EntryEventImpl(
+ final LocalRegion region,
+ Operation op, Object key, @Retained(ENTRY_EVENT_NEW_VALUE) Object newVal,
+ Object callbackArgument,
+ boolean originRemote, DistributedMember distributedMember,
+ boolean generateCallbacks, boolean initializeId) {
+
+ this.region = region;
+ this.op = op;
+ this.keyInfo = this.region.getKeyInfo(key, newVal, callbackArgument);
+
+ if (newVal instanceof Delta) {
+ this.delta = (Delta)newVal;
+ }
+ else if (!Token.isInvalid(newVal)) {
+ basicSetNewValue(newVal);
+ }
+
+ this.txId = this.region.getTXId();
+ /**
+ * this might set txId for events done from a thread that has a tx even
+ * though the op is non-tx. For example region ops.
+ */
+ if (newVal == Token.LOCAL_INVALID) {
+ setLocalInvalid(true);
+ }
+ setOriginRemote(originRemote);
+ setGenerateCallbacks(generateCallbacks);
+ this.distributedMember = distributedMember;
+ }
+
+ /**
+ * Called by BridgeEntryEventImpl to use existing EventID
+ */
+ @Retained
+ protected EntryEventImpl(LocalRegion region, Operation op, Object key,
+ @Retained(ENTRY_EVENT_NEW_VALUE) Object newValue, Object callbackArgument, boolean originRemote,
+ DistributedMember distributedMember, boolean generateCallbacks,
+ EventID eventID) {
+ this(region, op, key, newValue,
+ callbackArgument, originRemote, distributedMember, generateCallbacks,
+ true /* initializeId */);
+ Assert.assertTrue(eventID != null || !(region instanceof PartitionedRegion));
+ this.setEventId(eventID);
+ }
+
+ /**
+ * create an entry event from another entry event
+ */
+ @Retained
+ public EntryEventImpl(@Retained({ENTRY_EVENT_NEW_VALUE, ENTRY_EVENT_OLD_VALUE}) EntryEventImpl other) {
+ this(other, true);
+ }
+
+ @Retained
+ public EntryEventImpl(@Retained({ENTRY_EVENT_NEW_VALUE, ENTRY_EVENT_OLD_VALUE}) EntryEventImpl other, boolean setOldValue) {
+ region = other.region;
+
+ this.eventID = other.eventID;
+ basicSetNewValue(other.basicGetNewValue());
+ this.newValueBytes = other.newValueBytes;
+ this.cachedSerializedNewValue = other.cachedSerializedNewValue;
+ this.re = other.re;
+ this.delta = other.delta;
+ if (setOldValue) {
+ retainAndSetOldValue(other.basicGetOldValue());
+ this.oldValueBytes = other.oldValueBytes;
+ }
+ this.eventFlags = other.eventFlags;
+ setEventFlag(EventFlags.FLAG_CALLBACKS_INVOKED, false);
+ txId = other.txId;
+ op = other.op;
+ distributedMember = other.distributedMember;
+ this.filterInfo = other.filterInfo;
+ this.keyInfo = other.keyInfo.isDistKeyInfo() ? new DistTxKeyInfo(
+ (DistTxKeyInfo) other.keyInfo) : new KeyInfo(other.keyInfo);
+ if (other.getRawCallbackArgument() instanceof GatewaySenderEventCallbackArgument) {
+ this.keyInfo
+ .setCallbackArg((new GatewaySenderEventCallbackArgument(
+ (GatewaySenderEventCallbackArgument) other
+ .getRawCallbackArgument())));
+ }
+ this.context = other.context;
+ this.deltaBytes = other.deltaBytes;
+ this.tailKey = other.tailKey;
+ this.versionTag = other.versionTag;
+ //set possible duplicate
+ this.setPossibleDuplicate(other.isPossibleDuplicate());
+ }
+
+ @Retained
+ public EntryEventImpl(Object key2) {
+ this.keyInfo = new KeyInfo(key2, null, null);
+ }
+
+ /**
+ * This constructor is used to create a bridge event in server-side
+ * command classes. Events created with this are not intended to be
+ * used in cache operations.
+ * @param id the identity of the client's event
+ */
+ @Retained
+ public EntryEventImpl(EventID id) {
+ this.eventID = id;
+ this.offHeapOk = false;
+ }
+
+ /**
+ * Creates and returns an EntryEventImpl. Generates and assigns a bucket id to the
+ * EntryEventImpl if the region parameter is a PartitionedRegion.
+ */
+ @Retained
+ public static EntryEventImpl create(LocalRegion region,
+ Operation op,
+ Object key, @Retained(ENTRY_EVENT_NEW_VALUE) Object newValue, Object callbackArgument,
+ boolean originRemote, DistributedMember distributedMember) {
+ return create(region,op,key,newValue,callbackArgument,originRemote,distributedMember,true,true);
+ }
+
+ /**
+ * Creates and returns an EntryEventImpl. Generates and assigns a bucket id to the
+ * EntryEventImpl if the region parameter is a PartitionedRegion.
+ */
+ @Retained
+ public static EntryEventImpl create(LocalRegion region,
+ Operation op,
+ Object key,
+ @Retained(ENTRY_EVENT_NEW_VALUE) Object newValue,
+ Object callbackArgument,
+ boolean originRemote,
+ DistributedMember distributedMember,
+ boolean generateCallbacks) {
+ return create(region, op, key, newValue, callbackArgument, originRemote,
+ distributedMember, generateCallbacks,true);
+ }
+
+ /**
+ * Creates and returns an EntryEventImpl. Generates and assigns a bucket id to the
+ * EntryEventImpl if the region parameter is a PartitionedRegion.
+ *
+ * Called by BridgeEntryEventImpl to use existing EventID
+ *
+ * {@link EntryEventImpl#EntryEventImpl(LocalRegion, Operation, Object, Object, Object, boolean, DistributedMember, boolean, EventID)}
+ */
+ @Retained
+ public static EntryEventImpl create(LocalRegion region, Operation op, Object key,
+ @Retained(ENTRY_EVENT_NEW_VALUE) Object newValue, Object callbackArgument, boolean originRemote,
+ DistributedMember distributedMember, boolean generateCallbacks,
+ EventID eventID) {
+ EntryEventImpl entryEvent = new EntryEventImpl(region,op,key,newValue,callbackArgument,originRemote,distributedMember,generateCallbacks,eventID);
+ return entryEvent;
+ }
+
+ /**
+ * Creates and returns an EntryEventImpl. Generates and assigns a bucket id to the
+ * EntryEventImpl if the region parameter is a PartitionedRegion.
+ *
+ * {@link EntryEventImpl#EntryEventImpl(LocalRegion, Operation, Object, boolean, DistributedMember, boolean, boolean)}
+ */
+ @Retained
+ public static EntryEventImpl create(LocalRegion region, Operation op, Object key,
+ boolean originRemote, DistributedMember distributedMember,
+ boolean generateCallbacks, boolean fromRILocalDestroy) {
+ EntryEventImpl entryEvent = new EntryEventImpl(region,op,key,originRemote,distributedMember,generateCallbacks,fromRILocalDestroy);
+ return entryEvent;
+ }
+
+ /**
+ * Creates and returns an EntryEventImpl. Generates and assigns a bucket id to the
+ * EntryEventImpl if the region parameter is a PartitionedRegion.
+ *
+ * This creator does not specify the oldValue as this will be filled in later as part of an
+ * operation on the region, or lets it default to null.
+ *
+ * {@link EntryEventImpl#EntryEventImpl(LocalRegion, Operation, Object, Object, Object, boolean, DistributedMember, boolean, boolean)}
+ */
+ @Retained
+ public static EntryEventImpl create(final LocalRegion region,
+ Operation op, Object key, @Retained(ENTRY_EVENT_NEW_VALUE) Object newVal,
+ Object callbackArgument,
+ boolean originRemote, DistributedMember distributedMember,
+ boolean generateCallbacks, boolean initializeId) {
+ EntryEventImpl entryEvent = new EntryEventImpl(region,op,key,newVal,callbackArgument,
+ originRemote,distributedMember,generateCallbacks,initializeId);
+ return entryEvent;
+ }
+
+ /**
+ * Creates a PutAllEvent given the distributed operation, the region, and the
+ * entry data.
+ *
+ * @since 5.0
+ */
+ @Retained
+ static EntryEventImpl createPutAllEvent(
+ DistributedPutAllOperation putAllOp, LocalRegion region,
+ Operation entryOp, Object entryKey, @Retained(ENTRY_EVENT_NEW_VALUE) Object entryNewValue)
+ {
+ EntryEventImpl e;
+ if (putAllOp != null) {
+ EntryEventImpl event = putAllOp.getBaseEvent();
+ if (event.isBridgeEvent()) {
+ e = EntryEventImpl.create(region, entryOp, entryKey, entryNewValue,
+ event.getRawCallbackArgument(), false, event.distributedMember,
+ event.isGenerateCallbacks());
+ e.setContext(event.getContext());
+ } else {
+ e = EntryEventImpl.create(region, entryOp, entryKey, entryNewValue, event.getCallbackArgument(),
+ false, region.getMyId(), event.isGenerateCallbacks());
+ }
+
+ } else {
+ e = EntryEventImpl.create(region, entryOp, entryKey, entryNewValue, null,
+ false, region.getMyId(), true);
+ }
+
+ e.putAllOp = putAllOp;
+ return e;
+ }
+
+ protected static EntryEventImpl createRemoveAllEvent(
+ DistributedRemoveAllOperation op,
+ LocalRegion region,
+ Object entryKey) {
+ EntryEventImpl e;
+ final Operation entryOp = Operation.REMOVEALL_DESTROY;
+ if (op != null) {
+ EntryEventImpl event = op.getBaseEvent();
+ if (event.isBridgeEvent()) {
+ e = EntryEventImpl.create(region, entryOp, entryKey, null,
+ event.getRawCallbackArgument(), false, event.distributedMember,
+ event.isGenerateCallbacks());
+ e.setContext(event.getContext());
+ } else {
+ e = EntryEventImpl.create(region, entryOp, entryKey, null, event.getCallbackArgument(),
+ false, region.getMyId(), event.isGenerateCallbacks());
+ }
+
+ } else {
+ e = EntryEventImpl.create(region, entryOp, entryKey, null, null,
+ false, region.getMyId(), true);
+ }
+
+ e.removeAllOp = op;
+ return e;
+ }
+ public boolean isBulkOpInProgress() {
+ return getPutAllOperation() != null || getRemoveAllOperation() != null;
+ }
+
+ /** return the putAll operation for this event, if any */
+ public DistributedPutAllOperation getPutAllOperation() {
+ return this.putAllOp;
+ }
+ public DistributedPutAllOperation setPutAllOperation(DistributedPutAllOperation nv) {
+ DistributedPutAllOperation result = this.putAllOp;
+ if (nv != null && nv.getBaseEvent() != null) {
+ setCallbackArgument(nv.getBaseEvent().getCallbackArgument());
+ }
+ this.putAllOp = nv;
+ return result;
+ }
+ public DistributedRemoveAllOperation getRemoveAllOperation() {
+ return this.removeAllOp;
+ }
+ public DistributedRemoveAllOperation setRemoveAllOperation(DistributedRemoveAllOperation nv) {
+ DistributedRemoveAllOperation result = this.removeAllOp;
+ if (nv != null && nv.getBaseEvent() != null) {
+ setCallbackArgument(nv.getBaseEvent().getCallbackArgument());
+ }
+ this.removeAllOp = nv;
+ return result;
+ }
+
+ private final boolean testEventFlag(short mask)
+ {
+ return EventFlags.isSet(this.eventFlags, mask);
+ }
+
+ private final void setEventFlag(short mask, boolean on)
+ {
+ this.eventFlags = EventFlags.set(this.eventFlags, mask, on);
+ }
+
+ public DistributedMember getDistributedMember()
+ {
+ return this.distributedMember;
+ }
+
+ /////////////////////// INTERNAL BOOLEAN SETTERS
+ public void setOriginRemote(boolean b)
+ {
+ setEventFlag(EventFlags.FLAG_ORIGIN_REMOTE, b);
+ }
+
+ public void setLocalInvalid(boolean b)
+ {
+ setEventFlag(EventFlags.FLAG_LOCAL_INVALID, b);
+ }
+
+ void setGenerateCallbacks(boolean b)
+ {
+ setEventFlag(EventFlags.FLAG_GENERATE_CALLBACKS, b);
+ }
+
+ /** set the the flag telling whether callbacks should be invoked for a partitioned region */
+ public void setInvokePRCallbacks(boolean b) {
+ setEventFlag(EventFlags.FLAG_INVOKE_PR_CALLBACKS, b);
+ }
+
+ /** get the flag telling whether callbacks should be invoked for a partitioned region */
+ public boolean getInvokePRCallbacks() {
+ return testEventFlag(EventFlags.FLAG_INVOKE_PR_CALLBACKS);
+ }
+
+ public boolean getInhibitDistribution() {
+ return testEventFlag(EventFlags.FLAG_INHIBIT_DISTRIBUTION);
+ }
+
+ public void setInhibitDistribution(boolean b) {
+ setEventFlag(EventFlags.FLAG_INHIBIT_DISTRIBUTION, b);
+ }
+
+ /** was the entry destroyed or missing and allowed to be destroyed again? */
+ public boolean getIsRedestroyedEntry() {
+ return testEventFlag(EventFlags.FLAG_REDESTROYED_TOMBSTONE);
+ }
+
+ public void setIsRedestroyedEntry(boolean b) {
+ setEventFlag(EventFlags.FLAG_REDESTROYED_TOMBSTONE, b);
+ }
+
+ public void isConcurrencyConflict(boolean b) {
+ setEventFlag(EventFlags.FLAG_CONCURRENCY_CONFLICT, b);
+ }
+
+ public boolean isConcurrencyConflict() {
+ return testEventFlag(EventFlags.FLAG_CONCURRENCY_CONFLICT);
+ }
+
+ /** set the DistributionMessage that caused this event */
+ public void setCausedByMessage(DistributionMessage msg) {
+ this.causedByMessage = msg;
+ }
+
+ /**
+ * get the PartitionMessage that caused this event, or null if
+ * the event was not caused by a PartitionMessage
+ */
+ public PartitionMessage getPartitionMessage() {
+ if (this.causedByMessage != null && this.causedByMessage instanceof PartitionMessage) {
+ return (PartitionMessage)this.causedByMessage;
+ }
+ return null;
+ }
+
+ /**
+ * get the RemoteOperationMessage that caused this event, or null if
+ * the event was not caused by a RemoteOperationMessage
+ */
+ public RemoteOperationMessage getRemoteOperationMessage() {
+ if (this.causedByMessage != null && this.causedByMessage instanceof RemoteOperationMessage) {
+ return (RemoteOperationMessage)this.causedByMessage;
+ }
+ return null;
+ }
+
+ /////////////// BOOLEAN GETTERS
+ public boolean isLocalLoad()
+ {
+ return this.op.isLocalLoad();
+ }
+
+ public boolean isNetSearch()
+ {
+ return this.op.isNetSearch();
+ }
+
+ public boolean isNetLoad()
+ {
+ return this.op.isNetLoad();
+ }
+
+ public boolean isDistributed()
+ {
+ return this.op.isDistributed();
+ }
+
+ public boolean isExpiration()
+ {
+ return this.op.isExpiration();
+ }
+
+ public boolean isEviction() {
+ return this.op.isEviction();
+ }
+
+ public final boolean isCustomEviction() {
+ return this.isCustomEviction;
+ }
+
+ public final void setCustomEviction(boolean customEvict) {
+ this.isCustomEviction = customEvict;
+ }
+
+ public final void setEvicted() {
+ this.isEvicted = true;
+ }
+
+ public final boolean isEvicted() {
+ return this.isEvicted;
+ }
+
+ public final boolean isPendingSecondaryExpireDestroy() {
+ return this.isPendingSecondaryExpireDestroy;
+ }
+
+ public final void setPendingSecondaryExpireDestroy (boolean value) {
+ this.isPendingSecondaryExpireDestroy = value;
+ }
+ // Note that isOriginRemote is sometimes set to false even though the event
+ // was received from a peer. This is done to force distribution of the
+ // message to peers and to cause concurrency version stamping to be performed.
+ // This is done by all one-hop operations, like RemoteInvalidateMessage.
+ public boolean isOriginRemote()
+ {
+ return testEventFlag(EventFlags.FLAG_ORIGIN_REMOTE);
+ }
+
+ /* return whether this event originated from a WAN gateway and carries a WAN version tag */
+ public boolean isFromWANAndVersioned() {
+ return (this.versionTag != null && this.versionTag.isGatewayTag());
+ }
+
+ /* return whether this event originated in a client and carries a version tag */
+ public boolean isFromBridgeAndVersioned() {
+ return (this.context != null) && (this.versionTag != null);
+ }
+
+ public boolean isGenerateCallbacks()
+ {
+ return testEventFlag(EventFlags.FLAG_GENERATE_CALLBACKS);
+ }
+
+ public void setNewEventId(DistributedSystem sys) {
+ Assert.assertTrue(this.eventID == null, "Double setting event id");
+ EventID newID = new EventID(sys);
+ if (this.eventID != null) {
+ if (logger.isTraceEnabled(LogMarker.BRIDGE_SERVER)) {
+ logger.trace(LogMarker.BRIDGE_SERVER, "Replacing event ID with {} in event {}", newID, this);
+ }
+ }
+ this.eventID = newID;
+ }
+
+ public void reserveNewEventId(DistributedSystem sys, int count) {
+ Assert.assertTrue(this.eventID == null, "Double setting event id");
+ this.eventID = new EventID(sys);
+ if (count > 1) {
+ this.eventID.reserveSequenceId(count-1);
+ }
+ }
+
+ public void setEventId(EventID id)
+ {
+ this.eventID = id;
+ }
+
+ /**
+ * Return the event id, if any
+ * @return null if no event id has been set
+ */
+ public final EventID getEventId() {
+ return this.eventID;
+ }
+
+ public boolean isBridgeEvent() {
+ return hasClientOrigin();
+ }
+ public boolean hasClientOrigin() {
+ return getContext() != null;
+ }
+
+ /**
+ * sets the ID of the client that initiated this event
+ */
+ public void setContext(ClientProxyMembershipID contx) {
+ Assert.assertTrue(contx != null);
+ this.context = contx;
+ }
+
+ /**
+ * gets the ID of the client that initiated this event. Null if a server-initiated event
+ */
+ public ClientProxyMembershipID getContext()
+ {
+ return this.context;
+ }
+
+ // INTERNAL
+ boolean isLocalInvalid()
+ {
+ return testEventFlag(EventFlags.FLAG_LOCAL_INVALID);
+ }
+
+ /////////////////////////////////////////////////
+
+ /**
+ * Returns the key.
+ *
+ * @return the key.
+ */
+ public Object getKey()
+ {
+ return keyInfo.getKey();
+ }
+
+ /**
+ * Returns the value in the cache prior to this event. When passed to an event
+ * handler after an event occurs, this value reflects the value that was in
+ * the cache in this VM, not necessarily the value that was in the cache VM
+ * that initiated the operation.
+ *
+ * @return the value in the cache prior to this event.
+ */
+ public final Object getOldValue() {
+ try {
+ if (isOriginRemote() && this.region.isProxy()) {
+ return null;
+ }
+ @Unretained Object ov = basicGetOldValue();
+ if (ov == null) {
+ return null;
+ } else if (ov == Token.NOT_AVAILABLE) {
+ return AbstractRegion.handleNotAvailable(ov);
+ }
+ boolean doCopyOnRead = getRegion().isCopyOnRead();
+ if (ov != null) {
+ if (ov instanceof StoredObject) {
+ // TODO OFFHEAP: returns off-heap PdxInstance
+ return ((StoredObject) ov).getValueAsDeserializedHeapObject();
+ } else
+ if (ov instanceof CachedDeserializable) {
+ CachedDeserializable cd = (CachedDeserializable)ov;
+ if (doCopyOnRead) {
+ return cd.getDeserializedWritableCopy(this.region, this.re);
+ } else {
+ return cd.getDeserializedValue(this.region, this.re);
+ }
+ }
+ else {
+ if (doCopyOnRead) {
+ return CopyHelper.copy(ov);
+ } else {
+ return ov;
+ }
+ }
+ }
+ return null;
+ } catch(IllegalArgumentException i) {
+ IllegalArgumentException iae = new IllegalArgumentException(LocalizedStrings.DONT_RELEASE.toLocalizedString("Error while deserializing value for key="+getKey()));
+ iae.initCause(i);
+ throw iae;
+ }
+ }
+
+ /**
+ * Like getRawNewValue except that if the result is an off-heap reference then copy it to the heap.
+ * ALERT: If there is a Delta, returns that, not the (applied) new value.
+ * TODO OFFHEAP: to prevent the heap copy use getRawNewValue instead
+ */
+ public final Object getRawNewValueAsHeapObject() {
+ if (this.delta != null) {
+ return this.delta;
+ }
+ return OffHeapHelper.getHeapForm(OffHeapHelper.copyIfNeeded(basicGetNewValue()));
+ }
+
+ /**
+ * If new value is a Delta return it.
+ * Else if new value is off-heap return the StoredObject form (unretained OFF_HEAP_REFERENCE).
+ * Its refcount is not inced by this call and the returned object can only be safely used for the lifetime of the EntryEventImpl instance that returned the value.
+ * Else return the raw form.
+ */
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ public final Object getRawNewValue() {
+ if (this.delta != null) return this.delta;
+ return basicGetNewValue();
+ }
+
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ public Object getValue() {
+ return basicGetNewValue();
+ }
+
+ /**
+ * Returns the delta that represents the new value; null if no delta.
+ * @return the delta that represents the new value; null if no delta.
+ */
+ public final Delta getDeltaNewValue() {
+ return this.delta;
+ }
+
+ /**
+ * Applies the delta
+ */
+ private Object applyDeltaWithCopyOnRead(boolean doCopyOnRead) {
+ //try {
+ if (applyDelta(true)) {
+ Object applied = basicGetNewValue();
+ // if applyDelta returns true then newValue should not be off-heap
+ assert !(applied instanceof StoredObject);
+ if (applied == this.oldValue && doCopyOnRead) {
+ applied = CopyHelper.copy(applied);
+ }
+ return applied;
+ }
+ //} catch (EntryNotFoundException ex) {
+ // only (broken) product code has the opportunity to call this before
+ // this.oldValue is set. If oldValue is not set yet, then
+ // we most likely haven't synchronized on the region entry yet.
+ // (If we have, then make sure oldValue is set before
+ // calling this method).
+ //throw new AssertionError("too early to call getNewValue");
+ //}
+ return null;
+ }
+
+ @Released(ENTRY_EVENT_NEW_VALUE)
+ protected void basicSetNewValue(@Retained(ENTRY_EVENT_NEW_VALUE) Object v) {
+ if (v == this.newValue) return;
+ if (this.offHeapOk) {
+ OffHeapHelper.releaseAndTrackOwner(this.newValue, this);
+ }
+ if (v instanceof Chunk) {
+ ReferenceCountHelper.setReferenceCountOwner(this);
+ if (!((Chunk) v).retain()) {
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ this.newValue = null;
+ return;
+ }
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ }
+ this.newValue = v;
+ this.cachedSerializedNewValue = null;
+ }
+ /**
+ * Returns true if this event has a reference to an off-heap new or old value.
+ */
+ public boolean hasOffHeapValue() {
+ return (this.newValue instanceof Chunk) || (this.oldValue instanceof Chunk);
+ }
+
+ @Unretained
+ protected final Object basicGetNewValue() {
+ Object result = this.newValue;
+ if (!this.offHeapOk && result instanceof Chunk) {
+ //this.region.getCache().getLogger().info("DEBUG new value already freed " + System.identityHashCode(result));
+ throw new IllegalStateException("Attempt to access off heap value after the EntryEvent was released.");
+ }
+ return result;
+ }
+
+ private class OldValueOwner {
+ private EntryEventImpl getEvent() {
+ return EntryEventImpl.this;
+ }
+ @Override
+ public int hashCode() {
+ return getEvent().hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof OldValueOwner) {
+ return getEvent().equals(((OldValueOwner) obj).getEvent());
+ } else {
+ return false;
+ }
+ }
+ @Override
+ public String toString() {
+ return "OldValueOwner " + getEvent().toString();
+ }
+ }
+
+ /**
+ * Note if v might be an off-heap reference that you did not retain for this EntryEventImpl
+ * then call retainsAndSetOldValue instead of this method.
+ * @param v the caller should have already retained this off-heap reference.
+ */
+ @Released(ENTRY_EVENT_OLD_VALUE)
+ private void basicSetOldValue(@Unretained(ENTRY_EVENT_OLD_VALUE) Object v) {
+ @Released final Object curOldValue = this.oldValue;
+ if (v == curOldValue) return;
+ if (this.offHeapOk) {
+ if (curOldValue instanceof Chunk) {
+ if (ReferenceCountHelper.trackReferenceCounts()) {
+ OffHeapHelper.releaseAndTrackOwner(curOldValue, new OldValueOwner());
+ } else {
+ OffHeapHelper.release(curOldValue);
+ }
+ }
+ }
+
+ this.oldValue = v;
+ }
+
+ @Released(ENTRY_EVENT_OLD_VALUE)
+ private void retainAndSetOldValue(@Retained(ENTRY_EVENT_OLD_VALUE) Object v) {
+ if (v == this.oldValue) return;
+
+ if (v instanceof Chunk) {
+ if (ReferenceCountHelper.trackReferenceCounts()) {
+ ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
+ boolean couldNotRetain = (!((Chunk) v).retain());
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ if (couldNotRetain) {
+ this.oldValue = null;
+ return;
+ }
+ } else {
+ if (!((Chunk) v).retain()) {
+ this.oldValue = null;
+ return;
+ }
+ }
+ }
+ basicSetOldValue(v);
+ }
+
+ @Unretained(ENTRY_EVENT_OLD_VALUE)
+ private Object basicGetOldValue() {
+ @Unretained(ENTRY_EVENT_OLD_VALUE)
+ Object result = this.oldValue;
+ if (!this.offHeapOk && result instanceof Chunk) {
+ //this.region.getCache().getLogger().info("DEBUG old value already freed " + System.identityHashCode(result));
+ throw new IllegalStateException("Attempt to access off heap value after the EntryEvent was released.");
+ }
+ return result;
+ }
+
+ /**
+ * Like getRawOldValue except that if the result is an off-heap reference then copy it to the heap.
+ * To avoid the heap copy use getRawOldValue instead.
+ */
+ public final Object getRawOldValueAsHeapObject() {
+ return OffHeapHelper.getHeapForm(OffHeapHelper.copyIfNeeded(basicGetOldValue()));
+ }
+ /*
+ * If the old value is off-heap return the StoredObject form (unretained OFF_HEAP_REFERENCE).
+ * Its refcount is not inced by this call and the returned object can only be safely used for the lifetime of the EntryEventImpl instance that returned the value.
+ * Else return the raw form.
+ */
+ @Unretained
+ public final Object getRawOldValue() {
+ return basicGetOldValue();
+ }
+ /**
+ * Just like getRawOldValue except if the raw old value is off-heap deserialize it.
+ * Note that in some cases sqlf ignores the request to deserialize.
+ */
+ @Unretained(ENTRY_EVENT_OLD_VALUE)
+ public final Object getOldValueAsOffHeapDeserializedOrRaw() {
+ Object result = basicGetOldValue();
+ if (result instanceof StoredObject) {
+ result = ((StoredObject) result).getDeserializedForReading();
+ }
+ return AbstractRegion.handleNotAvailable(result); // fixes 49499
+ }
+
+ /**
+ * Added this function to expose isCopyOnRead function to the
+ * child classes of EntryEventImpl
+ *
+ */
+ protected boolean isRegionCopyOnRead() {
+ return getRegion().isCopyOnRead();
+ }
+
+ /**
+ * Returns the value in the cache after this event.
+ *
+ * @return the value in the cache after this event.
+ */
+ public final Object getNewValue() {
+
+ boolean doCopyOnRead = getRegion().isCopyOnRead();
+ try {
+ if (applyDelta(true)) {
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ Object applied = basicGetNewValue();
+ if (applied == this.oldValue && doCopyOnRead) {
+ applied = CopyHelper.copy(applied);
+ }
+ return applied;
+ }
+ } catch (EntryNotFoundException ex) {
+ // only (broken) product code has the opportunity to call this before
+ // this.oldValue is set. If oldValue is not set yet, then
+ // we most likely haven't synchronized on the region entry yet.
+ // (If we have, then make sure oldValue is set before
+ // calling this method).
+ throw new AssertionError("too early to call getNewValue");
+ }
+ Object nv = basicGetNewValue();
+ if (nv != null) {
+ if (nv == Token.NOT_AVAILABLE) {
+ // I'm not sure this can even happen
+ return AbstractRegion.handleNotAvailable(nv);
+ }
+ if (nv instanceof StoredObject) {
+ // TODO OFFHEAP currently we copy offheap new value to the heap here. Check callers of this method to see if they can be optimized to use offheap values.
+ // TODO OFFHEAP: returns off-heap PdxInstance
+ return ((StoredObject) nv).getValueAsDeserializedHeapObject();
+ } else
+ if (nv instanceof CachedDeserializable) {
+ CachedDeserializable cd = (CachedDeserializable)nv;
+ Object v = null;
+ if (doCopyOnRead) {
+ v = cd.getDeserializedWritableCopy(this.region, this.re);
+ } else {
+ v = cd.getDeserializedValue(this.region, this.re);
+ }
+ assert !(v instanceof CachedDeserializable) : "for key "+this.getKey()+" found nested CachedDeserializable";
+ return v;
+ }
+ else {
+ if (doCopyOnRead) {
+ return CopyHelper.copy(nv);
+ } else {
+ return nv;
+ }
+ }
+ }
+ return null;
+ }
+
+ public final String getNewValueStringForm() {
+ return StringUtils.forceToString(basicGetNewValue());
+ }
+ public final String getOldValueStringForm() {
+ return StringUtils.forceToString(basicGetOldValue());
+ }
+
+ protected boolean applyDelta(boolean throwOnNullOldValue)
+ throws EntryNotFoundException {
+ if (this.newValue != null || this.delta == null) {
+ return false;
+ }
+ if (this.oldValue == null) {
+ if (throwOnNullOldValue) {
+ // !!!:ezoerner:20080611 It would be nice if the client got this
+ // exception
+ throw new EntryNotFoundException(
+ "Cannot apply a delta without an existing value");
+ }
+ return false;
+ }
+ // swizzle BucketRegion in event for Delta.
+ // !!!:ezoerner:20090602 this is way ugly; this whole class severely
+ // needs refactoring
+ LocalRegion originalRegion = this.region;
+ try {
+ if (originalRegion instanceof BucketRegion) {
+ this.region = ((BucketRegion)this.region).getPartitionedRegion();
+ }
+ basicSetNewValue(this.delta.apply(this));
+ } finally {
+ this.region = originalRegion;
+ }
+ return true;
+ }
+
+ /** Set a deserialized value */
+ public final void setNewValue(@Retained(ENTRY_EVENT_NEW_VALUE) Object obj) {
+ if (obj instanceof Delta) {
+ this.delta = (Delta)obj;
+ basicSetNewValue(null);
+ }
+ else {
+ basicSetNewValue(obj);
+ }
+ }
+
+ public TransactionId getTransactionId()
+ {
+ return this.txId;
+ }
+
+ public void setTransactionId(TransactionId txId)
+ {
+ this.txId = (TXId)txId;
+ }
+
+ /**
+ * Answer true if this event resulted from a loader.
+ *
+ * @return true if isLocalLoad or isNetLoad
+ */
+ public boolean isLoad()
+ {
+ return this.op.isLoad();
+ }
+
+ public void setRegion(LocalRegion r)
+ {
+ this.region = r;
+ }
+
+ /**
+ * @see com.gemstone.gemfire.cache.CacheEvent#getRegion()
+ */
+ public final LocalRegion getRegion() {
+ return region;
+ }
+
+ public Operation getOperation()
+ {
+ return this.op;
+ }
+
+ public void setOperation(Operation op)
+ {
+ this.op = op;
+ PartitionMessage prm = getPartitionMessage();
+ if (prm != null) {
+ prm.setOperation(this.op);
+ }
+ }
+
+ /**
+ * @see com.gemstone.gemfire.cache.CacheEvent#getCallbackArgument()
+ */
+ public Object getCallbackArgument()
+ {
+ Object result = this.keyInfo.getCallbackArg();
+ while (result instanceof WrappedCallbackArgument) {
+ WrappedCallbackArgument wca = (WrappedCallbackArgument)result;
+ result = wca.getOriginalCallbackArg();
+ }
+ if (result == Token.NOT_AVAILABLE) {
+ result = AbstractRegion.handleNotAvailable(result);
+ }
+ return result;
+ }
+ public boolean isCallbackArgumentAvailable() {
+ return this.getRawCallbackArgument() != Token.NOT_AVAILABLE;
+ }
+
+ /**
+ * Returns the value of the EntryEventImpl field.
+ * This is for internal use only. Customers should always call
+ * {@link #getCallbackArgument}
+ * @since 5.5
+ */
+ public Object getRawCallbackArgument() {
+ return this.keyInfo.getCallbackArg();
+ }
+
+ /**
+ * Sets the value of raw callback argument field.
+ */
+ public void setRawCallbackArgument(Object newCallbackArgument) {
+ this.keyInfo.setCallbackArg(newCallbackArgument);
+ }
+
+ public void setCallbackArgument(Object newCallbackArgument) {
+ if (this.keyInfo.getCallbackArg() instanceof WrappedCallbackArgument) {
+ ((WrappedCallbackArgument)this.keyInfo.getCallbackArg())
+ .setOriginalCallbackArgument(newCallbackArgument);
+ }
+ else {
+ this.keyInfo.setCallbackArg(newCallbackArgument);
+ }
+ }
+
+ /**
+ * @return null if new value is not serialized; otherwise returns a SerializedCacheValueImpl containing the new value.
+ */
+ public SerializedCacheValue> getSerializedNewValue() {
+ // In the case where there is a delta that has not been applied yet,
+ // do not apply it here since it would not produce a serialized new
+ // value (return null instead to indicate the new value is not
+ // in serialized form).
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ final Object tmp = basicGetNewValue();
+ if (tmp instanceof CachedDeserializable) {
+ if (tmp instanceof StoredObject) {
+ if (!((StoredObject) tmp).isSerialized()) {
+ // TODO OFFHEAP can we handle offheap byte[] better?
+ return null;
+ }
+ }
+ byte[] bytes = this.newValueBytes;
+ if (bytes == null) {
+ bytes = this.cachedSerializedNewValue;
+ }
+ return new SerializedCacheValueImpl(this, getRegion(), this.re,
+ (CachedDeserializable)tmp, bytes);
+ } else {
+ // Note we return null even if cachedSerializedNewValue is not null.
+ // This is because some callers of this method use it to indicate
+ // that a CacheDeserializable should be created during deserialization.
+ return null;
+ }
+ }
+
+ /**
+ * Implement this interface if you want to call {@link #exportNewValue}.
+ *
+ *
+ */
+ public interface NewValueImporter {
+ /**
+ * @return true if the importer prefers the value to be in serialized form.
+ */
+ boolean prefersNewSerialized();
+
+ /**
+ * Only return true if the importer can use the value before the event that exported it is released.
+ * If false is returned then off-heap values will be copied to the heap for the importer.
+ * @return true if the importer can deal with the value being an unretained OFF_HEAP_REFERENCE.
+ */
+ boolean isUnretainedNewReferenceOk();
+
+ /**
+ * Import a new value that is currently in object form.
+ * @param nv the new value to import; unretained if isUnretainedNewReferenceOk returns true
+ * @param isSerialized true if the imported new value represents data that needs to be serialized; false if the imported new value is a simple sequence of bytes.
+ */
+ void importNewObject(@Unretained(ENTRY_EVENT_NEW_VALUE) Object nv, boolean isSerialized);
+
+ /**
+ * Import a new value that is currently in byte array form.
+ * @param nv the new value to import
+ * @param isSerialized true if the imported new value represents data that needs to be serialized; false if the imported new value is a simple sequence of bytes.
+ */
+ void importNewBytes(byte[] nv, boolean isSerialized);
+ }
+
+ /**
+ * Export the event's new value to the given importer.
+ */
+ public final void exportNewValue(NewValueImporter importer) {
+ final boolean prefersSerialized = importer.prefersNewSerialized();
+ if (prefersSerialized) {
+ if (getCachedSerializedNewValue() != null) {
+ importer.importNewBytes(getCachedSerializedNewValue(), true);
+ return;
+ } else {
+ if (this.newValueBytes != null && this.newValue instanceof CachedDeserializable) {
+ importer.importNewBytes(this.newValueBytes, true);
+ return;
+ }
+ }
+ }
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ final Object nv = getRawNewValue();
+ if (nv instanceof StoredObject) {
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ final StoredObject so = (StoredObject) nv;
+ final boolean isSerialized = so.isSerialized();
+ if (nv instanceof Chunk) {
+ if (importer.isUnretainedNewReferenceOk()) {
+ importer.importNewObject(nv, isSerialized);
+ } else {
+ if (!isSerialized || prefersSerialized) {
+ byte[] bytes = so.getValueAsHeapByteArray();
+ importer.importNewBytes(bytes, isSerialized);
+ if (isSerialized) {
+ setCachedSerializedNewValue(bytes);
+ }
+ } else {
+ // TODO OFFHEAP: returns off-heap PdxInstance which is not ok since isUnretainedNewReferenceOk returned false
+ importer.importNewObject(so.getValueAsDeserializedHeapObject(), true);
+ }
+ }
+ } else {
+ importer.importNewObject(nv, isSerialized);
+ }
+ } else if (nv instanceof byte[]) {
+ importer.importNewBytes((byte[])nv, false);
+ } else if (nv instanceof CachedDeserializable) {
+ CachedDeserializable cd = (CachedDeserializable) nv;
+ Object cdV = cd.getValue();
+ if (cdV instanceof byte[]) {
+ importer.importNewBytes((byte[]) cdV, true);
+ setCachedSerializedNewValue((byte[]) cdV);
+ } else {
+ importer.importNewObject(cdV, true);
+ }
+ } else {
+ importer.importNewObject(nv, true);
+ }
+ }
+ /**
+ * Implement this interface if you want to call {@link #exportOldValue}.
+ *
+ *
+ */
+ public interface OldValueImporter {
+ /**
+ * @return true if the importer prefers the value to be in serialized form.
+ */
+ boolean prefersOldSerialized();
+
+ /**
+ * Only return true if the importer can use the value before the event that exported it is released.
+ * @return true if the importer can deal with the value being an unretained OFF_HEAP_REFERENCE.
+ */
+ boolean isUnretainedOldReferenceOk();
+
+ /**
+ * @return return true if you want the old value to possibly be an instanceof CachedDeserializable; false if you want the value contained in a CachedDeserializable.
+ */
+ boolean isCachedDeserializableValueOk();
+
+ /**
+ * Import an old value that is currently in object form.
+ * @param ov the old value to import; unretained if isUnretainedOldReferenceOk returns true
+ * @param isSerialized true if the imported old value represents data that needs to be serialized; false if the imported old value is a simple sequence of bytes.
+ */
+ void importOldObject(@Unretained(ENTRY_EVENT_OLD_VALUE) Object ov, boolean isSerialized);
+
+ /**
+ * Import an old value that is currently in byte array form.
+ * @param ov the old value to import
+ * @param isSerialized true if the imported old value represents data that needs to be serialized; false if the imported old value is a simple sequence of bytes.
+ */
+ void importOldBytes(byte[] ov, boolean isSerialized);
+ }
+
+ /**
+ * Export the event's old value to the given importer.
+ */
+ public final void exportOldValue(OldValueImporter importer) {
+ final boolean prefersSerialized = importer.prefersOldSerialized();
+ if (prefersSerialized) {
+ if (this.oldValueBytes != null && this.oldValue instanceof CachedDeserializable) {
+ importer.importOldBytes(this.oldValueBytes, true);
+ return;
+ }
+ }
+ @Unretained(ENTRY_EVENT_OLD_VALUE)
+ final Object ov = getRawOldValue();
+ if (ov instanceof StoredObject) {
+ final StoredObject so = (StoredObject) ov;
+ final boolean isSerialized = so.isSerialized();
+ if (ov instanceof Chunk) {
+ if (importer.isUnretainedOldReferenceOk()) {
+ importer.importOldObject(ov, isSerialized);
+ } else {
+ if (!isSerialized || prefersSerialized) {
+ importer.importOldBytes(so.getValueAsHeapByteArray(), isSerialized);
+ } else {
+ // TODO OFFHEAP: returns off-heap PdxInstance which is not ok since isUnretainedNewReferenceOk returned false
+ importer.importOldObject(so.getValueAsDeserializedHeapObject(), true);
+ }
+ }
+ } else {
+ importer.importOldObject(ov, isSerialized);
+ }
+ } else if (ov instanceof byte[]) {
+ importer.importOldBytes((byte[])ov, false);
+ } else if (!importer.isCachedDeserializableValueOk() && ov instanceof CachedDeserializable) {
+ CachedDeserializable cd = (CachedDeserializable) ov;
+ Object cdV = cd.getValue();
+ if (cdV instanceof byte[]) {
+ importer.importOldBytes((byte[]) cdV, true);
+ } else {
+ importer.importOldObject(cdV, true);
+ }
+ } else {
+ importer.importOldObject(ov, true);
+ }
+ }
+
+ /**
+ * If applyDelta is true then first attempt to apply a delta (if we have one) and return the value.
+ * Else if new value is a Delta return it.
+ * Else if new value is off-heap return the StoredObject form (unretained OFF_HEAP_REFERENCE).
+ * Its refcount is not inced by this call and the returned object can only be safely used for the lifetime of the EntryEventImpl instance that returned the value.
+ * Else return the raw form.
+ */
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ public final Object getRawNewValue(boolean applyDelta) {
+ if (applyDelta) {
+ boolean doCopyOnRead = getRegion().isCopyOnRead();
+ Object newValueWithDelta = applyDeltaWithCopyOnRead(doCopyOnRead);
+ if (newValueWithDelta != null) {
+ return newValueWithDelta;
+ }
+ // if applyDelta is true and we have already applied the delta then
+ // just return the applied value instead of the delta object.
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ Object newValue = basicGetNewValue();
+ if (newValue != null) return newValue;
+ }
+ return getRawNewValue();
+ }
+ /**
+ * Just like getRawNewValue(true) except if the raw new value is off-heap deserialize it.
+ * Note that in some cases sqlf ignores the request to deserialize.
+ */
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ public final Object getNewValueAsOffHeapDeserializedOrRaw() {
+ Object result = getRawNewValue(true);
+ if (result instanceof StoredObject) {
+ result = ((StoredObject) result).getDeserializedForReading();
+ }
+ return AbstractRegion.handleNotAvailable(result); // fixes 49499
+ }
+
+ /**
+ * If the new value is stored off-heap return a retained OFF_HEAP_REFERENCE (caller must release).
+ * @return a retained OFF_HEAP_REFERENCE if the new value is off-heap; otherwise returns null
+ */
+ @Retained(ENTRY_EVENT_NEW_VALUE)
+ public StoredObject getOffHeapNewValue() {
+ final Object tmp = basicGetNewValue();
+ if (tmp instanceof StoredObject) {
+ StoredObject result = (StoredObject) tmp;
+ if (!result.retain()) {
+ return null;
+ }
+ return result;
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * If the old value is stored off-heap return a retained OFF_HEAP_REFERENCE (caller must release).
+ * @return a retained OFF_HEAP_REFERENCE if the old value is off-heap; otherwise returns null
+ */
+ @Retained(ENTRY_EVENT_OLD_VALUE)
+ public StoredObject getOffHeapOldValue() {
+ final Object tmp = basicGetOldValue();
+ if (tmp instanceof StoredObject) {
+ StoredObject result = (StoredObject) tmp;
+ if (!result.retain()) {
+ return null;
+ }
+ return result;
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Result may be unretained because sqlf getDeserializedForReading returns unretained.
+ */
+ public final Object getDeserializedValue() {
+ if (this.delta == null) {
+ final Object val = basicGetNewValue();
+ if (val instanceof StoredObject) {
+ // TODO OFFHEAP: returns off-heap PdxInstance
+ return ((StoredObject) val).getValueAsDeserializedHeapObject();
+ } else
+ if (val instanceof CachedDeserializable) {
+ return ((CachedDeserializable)val).getDeserializedForReading();
+ }
+ else {
+ return val;
+ }
+ }
+ else {
+ return this.delta;
+ }
+ }
+
+ public final byte[] getSerializedValue() {
+ if (this.newValueBytes == null) {
+ final Object val;
+ if (this.delta == null) {
+ val = basicGetNewValue();
+ if (val instanceof byte[]) {
+ return (byte[])val;
+ }
+ else if (val instanceof CachedDeserializable) {
+ return ((CachedDeserializable)val).getSerializedValue();
+ }
+ }
+ else {
+ val = this.delta;
+ }
+ try {
+ return CacheServerHelper.serialize(val);
+ } catch (IOException ioe) {
+ throw new GemFireIOException("unexpected exception", ioe);
+ }
+ }
+ else {
+ return this.newValueBytes;
+ }
+ }
+
+ /**
+ * Forces this entry's new value to be in serialized form.
+ * @since 5.0.2
+ */
+ public void makeSerializedNewValue() {
+ makeSerializedNewValue(false);
+ }
+
+ /**
+ * @param isSynced true if RegionEntry currently under synchronization
+ */
+ private final void makeSerializedNewValue(boolean isSynced) {
+ Object obj = basicGetNewValue();
+
+ // ezoerner:20080611 In the case where there is an unapplied
+ // delta, do not apply the delta or serialize yet unless entry is
+ // under synchronization (isSynced is true)
+ if (isSynced) {
+ this.setSerializationDeferred(false);
+ }
+ else if (obj == null && this.delta != null) {
+ // defer serialization until setNewValueInRegion
+ this.setSerializationDeferred(true);
+ return;
+ }
+ basicSetNewValue(getCachedDeserializable(obj, this));
+ }
+
+ public static Object getCachedDeserializable(Object obj) {
+ return getCachedDeserializable(obj, null);
+ }
+
+ public static Object getCachedDeserializable(Object obj, EntryEventImpl ev) {
+ if (obj instanceof byte[]
+ || obj == null
+ || obj instanceof CachedDeserializable
+ || obj == Token.NOT_AVAILABLE
+ || Token.isInvalidOrRemoved(obj)
+ // don't serialize delta object already serialized
+ || obj instanceof com.gemstone.gemfire.Delta
+ || obj instanceof Delta) { // internal delta
+ return obj;
+ }
+ final CachedDeserializable cd;
+ // avoid unneeded serialization of byte[][] used by SQLFabric that
+ // will end up being deserialized in any case (serialization is cheap
+ // for byte[][] anyways)
+ if (obj instanceof byte[][]) {
+ int objSize = Sizeable.PER_OBJECT_OVERHEAD + 4;
+ for (byte[] bytes : (byte[][])obj) {
+ if (bytes != null) {
+ objSize += CachedDeserializableFactory.getByteSize(bytes);
+ }
+ else {
+ objSize += Sizeable.PER_OBJECT_OVERHEAD;
+ }
+ }
+ cd = CachedDeserializableFactory.create(obj, objSize);
+ }
+ else {
+ final byte[] b = serialize(obj);
+ cd = CachedDeserializableFactory.create(b);
+ if (ev != null) {
+ ev.newValueBytes = b;
+ ev.cachedSerializedNewValue = b;
+ }
+ }
+ return cd;
+ }
+ public void setCachedSerializedNewValue(byte[] v) {
+ this.cachedSerializedNewValue = v;
+ }
+ public byte[] getCachedSerializedNewValue() {
+ return this.cachedSerializedNewValue;
+ }
+
+ public final void setSerializedNewValue(byte[] serializedValue) {
+ Object newVal = null;
+ if (serializedValue != null) {
+ if (CachedDeserializableFactory.preferObject()) {
+ newVal = deserialize(serializedValue);
+ } else {
+ newVal = CachedDeserializableFactory.create(serializedValue);
+ }
+ if (newVal instanceof Delta) {
+ this.delta = (Delta)newVal;
+ newVal = null;
+ // We need the newValueBytes field and the newValue field to be in sync.
+ // In the case of non-null delta set both fields to null.
+ serializedValue = null;
+ }
+ }
+ this.newValueBytes = serializedValue;
+ basicSetNewValue(newVal);
+ this.cachedSerializedNewValue = serializedValue;
+ }
+
+ public void setSerializedOldValue(byte[] serializedOldValue){
+ this.oldValueBytes = serializedOldValue;
+ final Object ov;
+ if (CachedDeserializableFactory.preferObject()) {
+ ov = deserialize(serializedOldValue);
+ }
+ else if (serializedOldValue != null) {
+ ov = CachedDeserializableFactory.create(serializedOldValue);
+ }
+ else {
+ ov = null;
+ }
+ retainAndSetOldValue(ov);
+ }
+
+ /**
+ * If true (the default) then preserve old values in events.
+ * If false then mark non-null values as being NOT_AVAILABLE.
+ */
+ private static final boolean EVENT_OLD_VALUE = !Boolean.getBoolean("gemfire.disable-event-old-value");
+
+
+ void putExistingEntry(final LocalRegion owner, RegionEntry entry) throws RegionClearedException {
+ putExistingEntry(owner, entry, false, null);
+ }
+
+ /**
+ * Put a newValue into the given, write synced, existing, region entry.
+ * Sets oldValue in event if hasn't been set yet.
+ * @param oldValueForDelta Used by Delta Propagation feature
+ *
+ * @throws RegionClearedException
+ */
+ void putExistingEntry(final LocalRegion owner, final RegionEntry reentry,
+ boolean requireOldValue, Object oldValueForDelta) throws RegionClearedException {
+ makeUpdate();
+ // only set oldValue if it hasn't already been set to something
+ if (this.oldValue == null) {
+ if (!reentry.isInvalidOrRemoved()) {
+ if (requireOldValue ||
+ EVENT_OLD_VALUE
+ || this.region instanceof HARegion // fix for bug 37909
+ || GemFireCacheImpl.sqlfSystem()
+ ) {
+ @Retained Object ov;
+ if (ReferenceCountHelper.trackReferenceCounts()) {
+ ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
+ if (GemFireCacheImpl.sqlfSystem()) {
+ ov = reentry.getValueOffHeapOrDiskWithoutFaultIn(this.region);
+ } else {
+ ov = reentry._getValueRetain(owner, true);
+ }
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ } else {
+ if (GemFireCacheImpl.sqlfSystem()) {
+ ov = reentry.getValueOffHeapOrDiskWithoutFaultIn(this.region);
+ } else {
+ ov = reentry._getValueRetain(owner, true);
+ }
+ }
+ if (ov == null) ov = Token.NOT_AVAILABLE;
+ // ov has already been retained so call basicSetOldValue instead of retainAndSetOldValue
+ basicSetOldValue(ov);
+ } else {
+ basicSetOldValue(Token.NOT_AVAILABLE);
+ }
+ }
+ }
+ if (this.oldValue == Token.NOT_AVAILABLE) {
+ FilterProfile fp = this.region.getFilterProfile();
+ if (this.op.guaranteesOldValue() ||
+ (fp != null /* #41532 */&& fp.entryRequiresOldValue(this.getKey()))) {
+ setOldValueForQueryProcessing();
+ }
+ }
+
+ //setNewValueInRegion(null);
+ setNewValueInRegion(owner, reentry, oldValueForDelta);
+ }
+
+ /**
+ * If we are currently a create op then turn us into an update
+ *
+ * @since 5.0
+ */
+ void makeUpdate()
+ {
+ setOperation(this.op.getCorrespondingUpdateOp());
+ }
+
+ /**
+ * If we are currently an update op then turn us into a create
+ *
+ * @since 5.0
+ */
+ void makeCreate()
+ {
+ setOperation(this.op.getCorrespondingCreateOp());
+ }
+
+ /**
+ * Put a newValue into the given, write synced, new, region entry.
+ * @throws RegionClearedException
+ */
+ void putNewEntry(final LocalRegion owner, final RegionEntry reentry)
+ throws RegionClearedException {
+ if (!this.op.guaranteesOldValue()) { // preserves oldValue for CM ops in clients
+ basicSetOldValue(null);
+ }
+ makeCreate();
+ setNewValueInRegion(owner, reentry, null);
+ }
+
+ void setRegionEntry(RegionEntry re) {
+ this.re = re;
+ }
+
+ RegionEntry getRegionEntry() {
+ return this.re;
+ }
+
+ @Retained(ENTRY_EVENT_NEW_VALUE)
+ private void setNewValueInRegion(final LocalRegion owner,
+ final RegionEntry reentry, Object oldValueForDelta) throws RegionClearedException {
+
+ boolean wasTombstone = reentry.isTombstone();
+
+ // put in newValue
+
+ if (applyDelta(this.op.isCreate())) {
+ if (this.isSerializationDeferred()) {
+ makeSerializedNewValue(true);
+ }
+ }
+
+ // If event contains new value, then it may mean that the delta bytes should
+ // not be applied. This is possible if the event originated locally.
+ if (this.deltaBytes != null && this.newValue == null) {
+ processDeltaBytes(oldValueForDelta);
+ }
+
+ if (owner!=null) {
+ owner.generateAndSetVersionTag(this, reentry);
+ } else {
+ this.region.generateAndSetVersionTag(this, reentry);
+ }
+
+ Object v = this.newValue;
+ if (v == null) {
+ v = isLocalInvalid() ? Token.LOCAL_INVALID : Token.INVALID;
+ }
+ else {
+ this.region.regionInvalid = false;
+ }
+
+ reentry.setValueResultOfSearch(this.op.isNetSearch());
+
+ //dsmith:20090524
+ //This is a horrible hack, but we need to get the size of the object
+ //When we store an entry. This code is only used when we do a put
+ //in the primary.
+ if(v instanceof com.gemstone.gemfire.Delta && region.isUsedForPartitionedRegionBucket()) {
+ int vSize;
+ Object ov = basicGetOldValue();
+ if(ov instanceof CachedDeserializable && !GemFireCacheImpl.DELTAS_RECALCULATE_SIZE) {
+ vSize = ((CachedDeserializable) ov).getValueSizeInBytes();
+ } else {
+ vSize = CachedDeserializableFactory.calcMemSize(v, region.getObjectSizer(), false);
+ }
+ v = CachedDeserializableFactory.create(v, vSize);
+ basicSetNewValue(v);
+ }
+
+ Object preparedV = reentry.prepareValueForCache(this.region, v, this, this.hasDelta());
+ if (preparedV != v) {
+ v = preparedV;
+ if (v instanceof Chunk) {
+ if (!((Chunk) v).isCompressed()) { // fix bug 52109
+ // If we put it off heap and it is not compressed then remember that value.
+ // Otherwise we want to remember the decompressed value in the event.
+ basicSetNewValue(v);
+ }
+ }
+ }
+ boolean isTombstone = (v == Token.TOMBSTONE);
+ boolean success = false;
+ boolean calledSetValue = false;
+ try {
+ setNewValueBucketSize(owner, v);
+
+ // ezoerner:20081030
+ // last possible moment to do index maintenance with old value in
+ // RegionEntry before new value is set.
+ // As part of an update, this is a remove operation as prelude to an add that
+ // will come after the new value is set.
+ // If this is an "update" from INVALID state, treat this as a create instead
+ // for the purpose of index maintenance since invalid entries are not
+ // indexed.
+
+ if ((this.op.isUpdate() && !reentry.isInvalid()) || this.op.isInvalidate()) {
+ IndexManager idxManager = IndexUtils.getIndexManager(this.region, false);
+ if (idxManager != null) {
+ try {
+ idxManager.updateIndexes(reentry,
+ IndexManager.REMOVE_ENTRY,
+ this.op.isUpdate() ?
+ IndexProtocol.BEFORE_UPDATE_OP :
+ IndexProtocol.OTHER_OP);
+ }
+ catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }
+ final IndexUpdater indexUpdater = this.region.getIndexUpdater();
+ if (indexUpdater != null) {
+ final LocalRegion indexRegion;
+ if (owner != null) {
+ indexRegion = owner;
+ }
+ else {
+ indexRegion = this.region;
+ }
+ try {
+ indexUpdater.onEvent(indexRegion, this, reentry);
+ calledSetValue = true;
+ reentry.setValueWithTombstoneCheck(v, this); // already called prepareValueForCache
+ success = true;
+ } finally {
+ indexUpdater.postEvent(indexRegion, this, reentry, success);
+ }
+ }
+ else {
+ calledSetValue = true;
+ reentry.setValueWithTombstoneCheck(v, this); // already called prepareValueForCache
+ success = true;
+ }
+ } finally {
+ if (!success && reentry instanceof OffHeapRegionEntry && v instanceof Chunk) {
+ OffHeapRegionEntryHelper.releaseEntry((OffHeapRegionEntry)reentry, (Chunk)v);
+ }
+ }
+ if (logger.isTraceEnabled()) {
+ if (v instanceof CachedDeserializable) {
+ logger.trace("EntryEventImpl.setNewValueInRegion: put CachedDeserializable({},{})",
+ this.getKey(), ((CachedDeserializable)v).getStringForm());
+ }
+ else {
+ logger.trace("EntryEventImpl.setNewValueInRegion: put({},{})",
+ this.getKey(), StringUtils.forceToString(v));
+ }
+ }
+
+ if (!isTombstone && wasTombstone) {
+ owner.unscheduleTombstone(reentry);
+ }
+ }
+
+ /**
+ * The size the new value contributes to a pr bucket.
+ * Note if this event is not on a pr then this value will be 0.
+ */
+ private transient int newValueBucketSize;
+ public int getNewValueBucketSize() {
+ return this.newValueBucketSize;
+ }
+ private void setNewValueBucketSize(LocalRegion lr, Object v) {
+ if (lr == null) {
+ lr = this.region;
+ }
+ this.newValueBucketSize = lr.calculateValueSize(v);
+ }
+
+ private void processDeltaBytes(Object oldValueInVM) {
+ if (!this.region.hasSeenEvent(this)) {
+ if (oldValueInVM == null || Token.isInvalidOrRemoved(oldValueInVM)) {
+ this.region.getCachePerfStats().incDeltaFailedUpdates();
+ throw new InvalidDeltaException("Old value not found for key "
+ + this.keyInfo.getKey());
+ }
+ FilterProfile fp = this.region.getFilterProfile();
+ // If compression is enabled then we've already gotten a new copy due to the
+ // serializaion and deserialization that occurs.
+ boolean copy = this.region.getCompressor() == null &&
+ (this.region.isCopyOnRead()
+ || this.region.getCloningEnabled()
+ || (fp != null && fp.getCqCount() > 0));
+ Object value = oldValueInVM;
+ boolean wasCD = false;
+ if (value instanceof CachedDeserializable) {
+ wasCD = true;
+ if (copy) {
+ value = ((CachedDeserializable)value).getDeserializedWritableCopy(this.region, re);
+ } else {
+ value = ((CachedDeserializable)value).getDeserializedValue(
+ this.region, re);
+ }
+ } else {
+ if (copy) {
+ value = CopyHelper.copy(value);
+ }
+ }
+ boolean deltaBytesApplied = false;
+ try {
+ long start = CachePerfStats.getStatTime();
+ ((com.gemstone.gemfire.Delta)value).fromDelta(new DataInputStream(
+ new ByteArrayInputStream(getDeltaBytes())));
+ this.region.getCachePerfStats().endDeltaUpdate(start);
+ deltaBytesApplied = true;
+ } catch (RuntimeException rte) {
+ throw rte;
+ } catch (VirtualMachineError e) {
+ SystemFailure.initiateFailure(e);
+ throw e;
+ } catch (Throwable t) {
+ SystemFailure.checkFailure();
+ throw new DeltaSerializationException(
+ "Exception while deserializing delta bytes.", t);
+ } finally {
+ if (!deltaBytesApplied) {
+ this.region.getCachePerfStats().incDeltaFailedUpdates();
+ }
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("Delta has been applied for key {}", getKey());
+ }
+ // assert event.getNewValue() == null;
+ if (wasCD) {
+ CachedDeserializable old = (CachedDeserializable)oldValueInVM;
+ int valueSize;
+ if (GemFireCacheImpl.DELTAS_RECALCULATE_SIZE) {
+ valueSize = CachedDeserializableFactory.calcMemSize(value, region
+ .getObjectSizer(), false);
+ } else {
+ valueSize = old.getValueSizeInBytes();
+ }
+ value = CachedDeserializableFactory.create(value, valueSize);
+ }
+ setNewValue(value);
+ if (this.causedByMessage != null
+ && this.causedByMessage instanceof PutMessage) {
+ ((PutMessage)this.causedByMessage).setDeltaValObj(value);
+ }
+ } else {
+ this.region.getCachePerfStats().incDeltaFailedUpdates();
+ throw new InvalidDeltaException(
+ "Cache encountered replay of event containing delta bytes for key "
+ + this.keyInfo.getKey());
+ }
+ }
+
+ void setTXEntryOldValue(Object oldVal, boolean mustBeAvailable) {
+ if (Token.isInvalidOrRemoved(oldVal)) {
+ oldVal = null;
+ }
+ else {
+ if (mustBeAvailable || oldVal == null || EVENT_OLD_VALUE) {
+ // set oldValue to oldVal
+ }
+ else {
+ oldVal = Token.NOT_AVAILABLE;
+ }
+ }
+ retainAndSetOldValue(oldVal);
+ }
+
+ void putValueTXEntry(final TXEntryState tx) {
+ Object v = basicGetNewValue();
+ if (v == null) {
+ if (deltaBytes != null) {
+ // since newValue is null, and we have deltaBytes
+ // there must be a nearSidePendingValue
+ processDeltaBytes(tx.getNearSidePendingValue());
+ v = basicGetNewValue();
+ } else if (this.delta != null) {
+ v = this.delta;
+ } else {
+ v = isLocalInvalid() ? Token.LOCAL_INVALID : Token.INVALID;
+ }
+ }
+
+ if (this.op != Operation.LOCAL_INVALIDATE
+ && this.op != Operation.LOCAL_DESTROY) {
+ // fix for bug 34387
+ tx.setPendingValue(OffHeapHelper.copyIfNeeded(v)); // TODO OFFHEAP optimize
+ }
+ tx.setCallbackArgument(getCallbackArgument());
+ }
+
+ /** @return false if entry doesn't exist */
+ public boolean setOldValueFromRegion()
+ {
+ try {
+ RegionEntry re = this.region.getRegionEntry(getKey());
+ if (re == null) return false;
+ ReferenceCountHelper.skipRefCountTracking();
+ Object v = re._getValueRetain(this.region, true);
+ ReferenceCountHelper.unskipRefCountTracking();
+ try {
+ return setOldValue(v);
+ } finally {
+ OffHeapHelper.releaseWithNoTracking(v);
+ }
+ }
+ catch (EntryNotFoundException ex) {
+ return false;
+ }
+ }
+
+ /** Return true if old value is the DESTROYED token */
+ boolean oldValueIsDestroyedToken()
+ {
+ return this.oldValue == Token.DESTROYED || this.oldValue == Token.TOMBSTONE;
+ }
+
+ void setOldValueDestroyedToken()
+ {
+ basicSetOldValue(Token.DESTROYED);
+ }
+
+ /**
+ * @return false if value 'v' indicates that entry does not exist
+ */
+ public boolean setOldValue(Object v) {
+ return setOldValue(v, false);
+ }
+
+
+ /**
+ * @param force true if the old value should be forcibly set, used
+ * for HARegions, methods like putIfAbsent, etc.,
+ * where the old value must be available.
+ * @return false if value 'v' indicates that entry does not exist
+ */
+ public boolean setOldValue(Object v, boolean force) {
+ if (v == null || Token.isRemoved(v)) {
+ return false;
+ }
+ else {
+ if (Token.isInvalid(v)) {
+ v = null;
+ }
+ else {
+ if (force ||
+ (this.region instanceof HARegion) // fix for bug 37909
+ ) {
+ // set oldValue to "v".
+ } else if (EVENT_OLD_VALUE) {
+ // TODO Rusty add compression support here
+ // set oldValue to "v".
+ } else {
+ v = Token.NOT_AVAILABLE;
+ }
+ }
+ retainAndSetOldValue(v);
+ return true;
+ }
+ }
+
+ /**
+ * sets the old value for concurrent map operation results received
+ * from a server.
+ */
+ public void setConcurrentMapOldValue(Object v) {
+ if (Token.isRemoved(v)) {
+ return;
+ } else {
+ if (Token.isInvalid(v)) {
+ v = null;
+ }
+ retainAndSetOldValue(v);
+ }
+ }
+
+ /** Return true if new value available */
+ public boolean hasNewValue() {
+ Object tmp = this.newValue;
+ if (tmp == null && hasDelta()) {
+ // ???:ezoerner:20080611 what if applying the delta would produce
+ // null or (strangely) NOT_AVAILABLE.. do we need to apply it here to
+ // find out?
+ return true;
+ }
+ return tmp != null && tmp != Token.NOT_AVAILABLE;
+ }
+
+ public final boolean hasOldValue() {
+ return this.oldValue != null && this.oldValue != Token.NOT_AVAILABLE;
+ }
+ public final boolean isOldValueAToken() {
+ return this.oldValue instanceof Token;
+ }
+
+ /**
+ * This should only be used in case of internal delta and not for Delta of
+ * Delta Propagation feature .
+ *
+ * @return boolean
+ */
+ public boolean hasDelta() {
+ return (this.delta != null);
+ }
+
+ public boolean isOldValueAvailable() {
+ if (isOriginRemote() && this.region.isProxy()) {
+ return false;
+ } else {
+ return basicGetOldValue() != Token.NOT_AVAILABLE;
+ }
+ }
+
+ public void oldValueNotAvailable() {
+ basicSetOldValue(Token.NOT_AVAILABLE);
+ }
+
+ public static Object deserialize(byte[] bytes) {
+ return deserialize(bytes, null, null);
+ }
+
+ public static Object deserialize(byte[] bytes, Version version,
+ ByteArrayDataInput in) {
+ if (bytes == null)
+ return null;
+ try {
+ return BlobHelper.deserializeBlob(bytes, version, in);
+ }
+ catch (IOException e) {
+ throw new SerializationException(LocalizedStrings.EntryEventImpl_AN_IOEXCEPTION_WAS_THROWN_WHILE_DESERIALIZING.toLocalizedString(), e);
+ }
+ catch (ClassNotFoundException e) {
+ // fix for bug 43602
+ throw new SerializationException(LocalizedStrings.EntryEventImpl_A_CLASSNOTFOUNDEXCEPTION_WAS_THROWN_WHILE_TRYING_TO_DESERIALIZE_CACHED_VALUE.toLocalizedString(), e);
+ }
+ }
+
+ /**
+ * If a PdxInstance is returned then it will have an unretained reference
+ * to Chunk's off-heap address.
+ */
+ public static @Unretained Object deserializeChunk(Chunk bytes) {
+ if (bytes == null)
+ return null;
+ try {
+ return BlobHelper.deserializeOffHeapBlob(bytes);
+ }
+ catch (IOException e) {
+ throw new SerializationException(LocalizedStrings.EntryEventImpl_AN_IOEXCEPTION_WAS_THROWN_WHILE_DESERIALIZING.toLocalizedString(), e);
+ }
+ catch (ClassNotFoundException e) {
+ // fix for bug 43602
+ throw new SerializationException(LocalizedStrings.EntryEventImpl_A_CLASSNOTFOUNDEXCEPTION_WAS_THROWN_WHILE_TRYING_TO_DESERIALIZE_CACHED_VALUE.toLocalizedString(), e);
+ }
+ }
+
+ /**
+ * Serialize an object into a byte[]
+ *
+ * @throws IllegalArgumentException
+ * If obj
should not be serialized
+ */
+ public static byte[] serialize(Object obj) {
+ return serialize(obj, null);
+ }
+
+ /**
+ * Serialize an object into a byte[]
+ *
+ * @throws IllegalArgumentException
+ * If obj
should not be serialized
+ */
+ public static byte[] serialize(Object obj, Version version)
+ {
+ if (obj == null || obj == Token.NOT_AVAILABLE
+ || Token.isInvalidOrRemoved(obj))
+ throw new IllegalArgumentException(LocalizedStrings.EntryEventImpl_MUST_NOT_SERIALIZE_0_IN_THIS_CONTEXT.toLocalizedString(obj));
+ try {
+ return BlobHelper.serializeToBlob(obj, version);
+ }
+ catch (IOException e) {
+ throw new SerializationException(LocalizedStrings.EntryEventImpl_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING.toLocalizedString(), e);
+ }
+ }
+
+
+ /**
+ * Serialize an object into a byte[]
. If the byte array
+ * provided by the wrapper is sufficient to hold the data, it is used
+ * otherwise a new byte array gets created & its reference is stored in the
+ * wrapper. The User Bit is also appropriately set as Serialized
+ *
+ * @param wrapper
+ * Object of type BytesAndBitsForCompactor which is used to fetch
+ * the serialized data. The byte array of the wrapper is used
+ * if possible else a the new byte array containing the data is
+ * set in the wrapper.
+ * @throws IllegalArgumentException
+ * If obj
should not be serialized
+ */
+ public static void fillSerializedValue(BytesAndBitsForCompactor wrapper,
+ Object obj, byte userBits) {
+ if (obj == null || obj == Token.NOT_AVAILABLE
+ || Token.isInvalidOrRemoved(obj))
+ throw new IllegalArgumentException(
+ LocalizedStrings.EntryEvents_MUST_NOT_SERIALIZE_0_IN_THIS_CONTEXT.toLocalizedString(obj));
+ try {
+ HeapDataOutputStream hdos = null;
+ if (wrapper.getBytes().length < 32) {
+ hdos = new HeapDataOutputStream(Version.CURRENT);
+ }
+ else {
+ hdos = new HeapDataOutputStream(wrapper.getBytes());
+ }
+ DataSerializer.writeObject(obj, hdos);
+ // return hdos.toByteArray();
+ hdos.sendTo(wrapper, userBits);
+ }
+ catch (IOException e) {
+ RuntimeException e2 = new IllegalArgumentException(
+ LocalizedStrings.EntryEventImpl_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING.toLocalizedString());
+ e2.initCause(e);
+ throw e2;
+ }
+ }
+
+ protected String getShortClassName() {
+ String cname = getClass().getName();
+ return cname.substring(getClass().getPackage().getName().length()+1);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder buf = new StringBuilder();
+ buf.append(getShortClassName());
+ buf.append("[");
+
+ buf.append("op=");
+ buf.append(getOperation());
+ buf.append(";key=");
+ buf.append(this.getKey());
+ buf.append(";oldValue=");
+ try {
+ ArrayUtils.objectStringNonRecursive(basicGetOldValue(), buf);
+ } catch (IllegalStateException ex) {
+ buf.append("OFFHEAP_VALUE_FREED");
+ }
+ buf.append(";newValue=");
+ try {
+ ArrayUtils.objectStringNonRecursive(basicGetNewValue(), buf);
+ } catch (IllegalStateException ex) {
+ buf.append("OFFHEAP_VALUE_FREED");
+ }
+ buf.append(";callbackArg=");
+ buf.append(this.getRawCallbackArgument());
+ buf.append(";originRemote=");
+ buf.append(isOriginRemote());
+ buf.append(";originMember=");
+ buf.append(getDistributedMember());
+// if (this.partitionMessage != null) {
+// buf.append("; partitionMessage=");
+// buf.append(this.partitionMessage);
+// }
+ if (this.isPossibleDuplicate()) {
+ buf.append(";posDup");
+ }
+ if (callbacksInvoked()) {
+ buf.append(";callbacksInvoked");
+ }
+ if (this.versionTag != null) {
+ buf.append(";version=").append(this.versionTag);
+ }
+ if (getContext() != null) {
+ buf.append(";context=");
+ buf.append(getContext());
+ }
+ if (this.eventID != null) {
+ buf.append(";id=");
+ buf.append(this.eventID);
+ }
+ if (this.deltaBytes != null) {
+ buf.append(";[" + this.deltaBytes.length + " deltaBytes]");
+ }
+// else {
+// buf.append(";[no deltaBytes]");
+// }
+ if (this.filterInfo != null) {
+ buf.append(";routing=");
+ buf.append(this.filterInfo);
+ }
+ if (this.isFromServer()) {
+ buf.append(";isFromServer");
+ }
+ if (this.isConcurrencyConflict()) {
+ buf.append(";isInConflict");
+ }
+ if (this.getInhibitDistribution()) {
+ buf.append(";inhibitDistribution");
+ }
+ buf.append("]");
+ return buf.toString();
+ }
+
+ public int getDSFID() {
+ return ENTRY_EVENT;
+ }
+
+ public void toData(DataOutput out) throws IOException
+ {
+ DataSerializer.writeObject(this.eventID, out);
+ DataSerializer.writeObject(this.getKey(), out);
+ DataSerializer.writeObject(this.keyInfo.getValue(), out);
+ out.writeByte(this.op.ordinal);
+ out.writeShort(this.eventFlags & EventFlags.FLAG_TRANSIENT_MASK);
+ DataSerializer.writeObject(this.getRawCallbackArgument(), out);
+ DataSerializer.writeObject(this.txId, out);
+
+ {
+ boolean isDelta = this.delta != null;
+ out.writeBoolean(isDelta);
+ if (isDelta) {
+ DataSerializer.writeObject(this.delta, out);
+ }
+ else {
+ Object nv = basicGetNewValue();
+ boolean newValueSerialized = nv instanceof CachedDeserializable;
+ if (newValueSerialized) {
+ if (nv instanceof StoredObject) {
+ newValueSerialized = ((StoredObject) nv).isSerialized();
+ }
+ }
+ out.writeBoolean(newValueSerialized);
+ if (newValueSerialized) {
+ if (this.newValueBytes != null) {
+ DataSerializer.writeByteArray(this.newValueBytes, out);
+ } else if (this.cachedSerializedNewValue != null) {
+ DataSerializer.writeByteArray(this.cachedSerializedNewValue, out);
+ } else {
+ CachedDeserializable cd = (CachedDeserializable)nv;
+ DataSerializer.writeObjectAsByteArray(cd.getValue(), out);
+ }
+ }
+ else {
+ DataSerializer.writeObject(nv, out);
+ }
+ }
+ }
+
+ {
+ Object ov = basicGetOldValue();
+ boolean oldValueSerialized = ov instanceof CachedDeserializable;
+ if (oldValueSerialized) {
+ if (ov instanceof StoredObject) {
+ oldValueSerialized = ((StoredObject) ov).isSerialized();
+ }
+ }
+ out.writeBoolean(oldValueSerialized);
+ if (oldValueSerialized) {
+ if (this.oldValueBytes != null) {
+ DataSerializer.writeByteArray(this.oldValueBytes, out);
+ }
+ else {
+ CachedDeserializable cd = (CachedDeserializable)ov;
+ DataSerializer.writeObjectAsByteArray(cd.getValue(), out);
+ }
+ }
+ else {
+ DataSerializer.writeObject(ov, out);
+ }
+ }
+ InternalDataSerializer.invokeToData((InternalDistributedMember)this.distributedMember, out);
+ DataSerializer.writeObject(getContext(), out);
+ DataSerializer.writeLong(tailKey, out);
+ }
+
+ private static abstract class EventFlags
+ {
+ private static final short FLAG_ORIGIN_REMOTE = 0x01;
+ // localInvalid: true if a null new value should be treated as a local
+ // invalid.
+ private static final short FLAG_LOCAL_INVALID = 0x02;
+ private static final short FLAG_GENERATE_CALLBACKS = 0x04;
+ private static final short FLAG_POSSIBLE_DUPLICATE = 0x08;
+ private static final short FLAG_INVOKE_PR_CALLBACKS = 0x10;
+ private static final short FLAG_CONCURRENCY_CONFLICT = 0x20;
+ private static final short FLAG_INHIBIT_LISTENER_NOTIFICATION = 0x40;
+ private static final short FLAG_CALLBACKS_INVOKED = 0x80;
+ private static final short FLAG_ISCREATE = 0x100;
+ private static final short FLAG_SERIALIZATION_DEFERRED = 0x200;
+ private static final short FLAG_FROM_SERVER = 0x400;
+ private static final short FLAG_FROM_RI_LOCAL_DESTROY = 0x800;
+ private static final short FLAG_INHIBIT_DISTRIBUTION = 0x1000;
+ private static final short FLAG_REDESTROYED_TOMBSTONE = 0x2000;
+ private static final short FLAG_INHIBIT_ALL_NOTIFICATIONS= 0x4000;
+
+ /** mask for clearing transient flags when serializing */
+ private static final short FLAG_TRANSIENT_MASK =
+ ~(FLAG_CALLBACKS_INVOKED
+ | FLAG_ISCREATE
+ | FLAG_INHIBIT_LISTENER_NOTIFICATION
+ | FLAG_SERIALIZATION_DEFERRED
+ | FLAG_FROM_SERVER
+ | FLAG_FROM_RI_LOCAL_DESTROY
+ | FLAG_INHIBIT_DISTRIBUTION
+ | FLAG_REDESTROYED_TOMBSTONE
+ );
+
+ protected static final boolean isSet(short flags, short mask)
+ {
+ return (flags & mask) != 0;
+ }
+
+ /** WARNING: Does not set the bit in place, returns new short with bit set */
+ protected static final short set(short flags, short mask, boolean on)
+ {
+ return (short)(on ? (flags | mask) : (flags & ~mask));
+ }
+ }
+
+ /**
+ * @return null if old value is not serialized; otherwise returns a SerializedCacheValueImpl containing the old value.
+ */
+ public final SerializedCacheValue> getSerializedOldValue() {
+ @Unretained(ENTRY_EVENT_OLD_VALUE)
+ final Object tmp = basicGetOldValue();
+ if (tmp instanceof CachedDeserializable) {
+ if (tmp instanceof StoredObject) {
+ if (!((StoredObject) tmp).isSerialized()) {
+ // TODO OFFHEAP can we handle offheap byte[] better?
+ return null;
+ }
+ }
+ return new SerializedCacheValueImpl(this, this.region, this.re,
+ (CachedDeserializable)tmp, this.oldValueBytes);
+ }
+ else {
+ return null;
+ }
+ }
+
+ /**
+ * Compute an estimate of the size of the new value
+ * for a PR. Since PR's always store values in a cached deserializable
+ * we need to compute its size as a blob.
+ *
+ * @return the size of serialized bytes for the new value
+ */
+ public int getNewValSizeForPR()
+ {
+ int newSize = 0;
+ applyDelta(false);
+ Object v = basicGetNewValue();
+ if (v != null) {
+ try {
+ newSize = CachedDeserializableFactory.calcSerializedSize(v)
+ + CachedDeserializableFactory.overhead();
+ } catch (IllegalArgumentException iae) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.EntryEventImpl_DATASTORE_FAILED_TO_CALCULATE_SIZE_OF_NEW_VALUE), iae);
+ newSize = 0;
+ }
+ }
+ return newSize;
+ }
+
+ /**
+ * Compute an estimate of the size of the old value
+ *
+ * @return the size of serialized bytes for the old value
+ */
+ public int getOldValSize()
+ {
+ int oldSize = 0;
+ if (hasOldValue()) {
+ try {
+ oldSize = CachedDeserializableFactory.calcMemSize(basicGetOldValue());
+ } catch (IllegalArgumentException iae) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.EntryEventImpl_DATASTORE_FAILED_TO_CALCULATE_SIZE_OF_OLD_VALUE), iae);
+ oldSize = 0;
+ }
+ }
+ return oldSize;
+ }
+
+ public EnumListenerEvent getEventType() {
+ return this.eventType;
+ }
+
+ /**
+ * Sets the operation type.
+ * @param eventType
+ */
+ public void setEventType(EnumListenerEvent eventType) {
+ this.eventType = eventType;
+ }
+
+ /**
+ * set this to true after dispatching the event to a cache listener
+ */
+ public void callbacksInvoked(boolean dispatched) {
+ setEventFlag(EventFlags.FLAG_CALLBACKS_INVOKED, dispatched);
+ }
+
+ /**
+ * has this event been dispatched to a cache listener?
+ */
+ public boolean callbacksInvoked() {
+ return testEventFlag(EventFlags.FLAG_CALLBACKS_INVOKED);
+ }
+
+ /**
+ * set this to true to inhibit application cache listener notification
+ * during event dispatching
+ */
+ public void inhibitCacheListenerNotification(boolean inhibit) {
+ setEventFlag(EventFlags.FLAG_INHIBIT_LISTENER_NOTIFICATION, inhibit);
+ }
+
+ /**
+ * are events being inhibited from dispatch to application cache listeners
+ * for this event?
+ */
+ public boolean inhibitCacheListenerNotification() {
+ return testEventFlag(EventFlags.FLAG_INHIBIT_LISTENER_NOTIFICATION);
+ }
+
+
+ /**
+ * dispatch listener events for this event
+ * @param notifyGateways pass the event on to WAN queues
+ */
+ void invokeCallbacks(LocalRegion rgn,boolean skipListeners, boolean notifyGateways) {
+ if (!callbacksInvoked()) {
+ callbacksInvoked(true);
+ if (this.op.isUpdate()) {
+ rgn.invokePutCallbacks(EnumListenerEvent.AFTER_UPDATE, this,
+ !skipListeners, notifyGateways); // gateways are notified in part2 processing
+ }
+ else if (this.op.isCreate()) {
+ rgn.invokePutCallbacks(EnumListenerEvent.AFTER_CREATE, this,
+ !skipListeners, notifyGateways);
+ }
+ else if (this.op.isDestroy()) {
+ rgn.invokeDestroyCallbacks(EnumListenerEvent.AFTER_DESTROY,
+ this, !skipListeners, notifyGateways);
+ }
+ else if (this.op.isInvalidate()) {
+ rgn.invokeInvalidateCallbacks(EnumListenerEvent.AFTER_INVALIDATE,
+ this, !skipListeners);
+ }
+ }
+ }
+
+ private void setFromRILocalDestroy(boolean on) {
+ setEventFlag(EventFlags.FLAG_FROM_RI_LOCAL_DESTROY, on);
+ }
+
+ public boolean isFromRILocalDestroy(){
+ return testEventFlag(EventFlags.FLAG_FROM_RI_LOCAL_DESTROY);
+ }
+
+ protected Long tailKey = -1L;
+
+ /**
+ * Used to store next region version generated for a change on this entry
+ * by phase-1 commit on the primary.
+ *
+ * Not to be used in fromData and toData
+ */
+ protected transient long nextRegionVersion = -1L;
+
+ public void setNextRegionVersion(long regionVersion) {
+ this.nextRegionVersion = regionVersion;
+ }
+
+ public long getNextRegionVersion() {
+ return this.nextRegionVersion;
+ }
+
+ /**
+ * Return true if this event came from a server by the client doing a get.
+ * @since 5.7
+ */
+ public boolean isFromServer() {
+ return testEventFlag(EventFlags.FLAG_FROM_SERVER);
+ }
+ /**
+ * Sets the fromServer flag to v. This must be set to true if an event
+ * comes from a server while the affected region entry is not locked. Among
+ * other things it causes version conflict checks to be performed to protect
+ * against overwriting a newer version of the entry.
+ * @since 5.7
+ */
+ public void setFromServer(boolean v) {
+ setEventFlag(EventFlags.FLAG_FROM_SERVER, v);
+ }
+
+ /**
+ * If true, the region associated with this event had already
+ * applied the operation it encapsulates when an attempt was
+ * made to apply the event.
+ * @return the possibleDuplicate
+ */
+ public boolean isPossibleDuplicate() {
+ return testEventFlag(EventFlags.FLAG_POSSIBLE_DUPLICATE);
+ }
+
+ /**
+ * If the operation encapsulated by this event has already been
+ * seen by the region to which it pertains, this flag should be
+ * set to true.
+ * @param possibleDuplicate the possibleDuplicate to set
+ */
+ public void setPossibleDuplicate(boolean possibleDuplicate) {
+ setEventFlag(EventFlags.FLAG_POSSIBLE_DUPLICATE, possibleDuplicate);
+ }
+
+
+ /**
+ * are events being inhibited from dispatch to to gateway/async queues,
+ * client queues, cache listener and cache write. If set, sending
+ * notifications for the data that is read from a persistent store (HDFS) and
+ * is being reinserted in the cache is skipped.
+ */
+ public boolean inhibitAllNotifications() {
+ return testEventFlag(EventFlags.FLAG_INHIBIT_ALL_NOTIFICATIONS);
+
+ }
+
+ /**
+ * set this to true to inhibit notifications that are sent to gateway/async queues,
+ * client queues, cache listener and cache write. This is used to skip sending
+ * notifications for the data that is read from a persistent store (HDFS) and
+ * is being reinserted in the cache
+ */
+ public void setInhibitAllNotifications(boolean inhibit) {
+ setEventFlag(EventFlags.FLAG_INHIBIT_ALL_NOTIFICATIONS, inhibit);
+ }
+
+ /**
+ * sets the routing information for cache clients
+ */
+ public void setLocalFilterInfo(FilterInfo info) {
+ this.filterInfo = info;
+ }
+
+ /**
+ * retrieves the routing information for cache clients in this VM
+ */
+ public FilterInfo getLocalFilterInfo() {
+ return this.filterInfo;
+ }
+
+
+ public LocalRegion getLocalRegion() {
+ return this.region;
+ }
+
+ /**
+ * This method returns the delta bytes used in Delta Propagation feature.
+ * For internal delta, see getRawNewValue().
+ *
+ * @return delta bytes
+ */
+ public byte[] getDeltaBytes() {
+ return deltaBytes;
+ }
+
+ /**
+ * This method sets the delta bytes used in Delta Propagation feature. For
+ * internal delta, see setNewValue().
+ *
+ * @param deltaBytes
+ */
+ public void setDeltaBytes(byte[] deltaBytes) {
+ this.deltaBytes = deltaBytes;
+ }
+
+ // TODO (ashetkar) Can this.op.isCreate() be used instead?
+ public boolean isCreate() {
+ return testEventFlag(EventFlags.FLAG_ISCREATE);
+ }
+
+ /**
+ * this is used to distinguish an event that merely has Operation.CREATE
+ * from one that originated from Region.create() for delta processing
+ * purposes.
+ */
+ public EntryEventImpl setCreate(boolean isCreate) {
+ setEventFlag(EventFlags.FLAG_ISCREATE, isCreate);
+ return this;
+ }
+
+ public final void setContextObject(Object ctx) {
+ this.contextObj = ctx;
+ }
+
+ public final Object getContextObject() {
+ return this.contextObj;
+ }
+
+ /**
+ * @return the keyInfo
+ */
+ public KeyInfo getKeyInfo() {
+ return keyInfo;
+ }
+
+ public void setKeyInfo(KeyInfo keyInfo) {
+ this.keyInfo = keyInfo;
+ }
+
+ /**
+ * establish the old value in this event as the current cache value,
+ * whether in memory or on disk
+ */
+ public void setOldValueForQueryProcessing() {
+ RegionEntry reentry = this.region.entries.getEntry(this.getKey());
+ if (reentry != null) {
+ @Retained Object v = reentry.getValueOffHeapOrDiskWithoutFaultIn(this.region);
+ if ( !(v instanceof Token) ) {
+ // v has already been retained.
+ basicSetOldValue(v);
+ // this event now owns the retention of v.
+ }
+ }
+ }
+
+ @Override
+ public Version[] getSerializationVersions() {
+ return null;
+ }
+
+ /**
+ * @param versionTag the versionTag to set
+ */
+ public void setVersionTag(VersionTag versionTag) {
+ this.versionTag = versionTag;
+ }
+
+ /**
+ * @return the concurrency versioning tag for this event, if any
+ */
+ public VersionTag getVersionTag() {
+ return this.versionTag;
+ }
+
+ /**
+ * @return if there's no valid version tag for this event
+ */
+ public boolean hasValidVersionTag() {
+ return this.versionTag != null && this.versionTag.hasValidVersion();
+ }
+
+ /**
+ * this method joins together version tag timestamps and the "lastModified"
+ * timestamps generated and stored in entries. If a change does not already
+ * carry a lastModified timestamp
+ * @param suggestedTime
+ * @return the timestamp to store in the entry
+ */
+ public long getEventTime(long suggestedTime) {
+ long result = suggestedTime;
+ if (this.versionTag != null) {
+ if (suggestedTime != 0) {
+ this.versionTag.setVersionTimeStamp(suggestedTime);
+ } else {
+ result = this.versionTag.getVersionTimeStamp();
+ }
+ }
+ if (result <= 0) {
+ LocalRegion region = this.getLocalRegion();
+ if (region != null) {
+ result = region.cacheTimeMillis();
+ } else {
+ result = System.currentTimeMillis();
+ }
+ }
+ return result;
+ }
+
+ public static final class SerializedCacheValueImpl
+ implements SerializedCacheValue, CachedDeserializable, Sendable
+ {
+ private final EntryEventImpl event;
+ @Unretained private final CachedDeserializable cd;
+ private final Region r;
+ private final RegionEntry re;
+ private final byte[] serializedValue;
+
+ SerializedCacheValueImpl(EntryEventImpl event, Region r, RegionEntry re, @Unretained CachedDeserializable cd, byte[] serializedBytes) {
+ if (cd instanceof Chunk) {
+ this.event = event;
+ } else {
+ this.event = null;
+ }
+ this.r = r;
+ this.re = re;
+ this.cd = cd;
+ this.serializedValue = serializedBytes;
+ }
+
+ public byte[] getSerializedValue() {
+ if(this.serializedValue != null){
+ return this.serializedValue;
+ }
+ return getCd().getSerializedValue();
+ }
+
+ private CachedDeserializable getCd() {
+ if (this.event != null && !this.event.offHeapOk) {
+ throw new IllegalStateException("Attempt to access off heap value after the EntryEvent was released.");
+ }
+ return this.cd;
+ }
+
+ public Object getDeserializedValue() {
+ return getDeserializedValue(this.r, this.re);
+ }
+ public Object getDeserializedForReading() {
+ // TODO OFFHEAP: returns off-heap PdxInstance
+ return OffHeapHelper.getHeapForm(getCd().getDeserializedForReading());
+ }
+ public Object getDeserializedWritableCopy(Region rgn, RegionEntry entry) {
+ // TODO OFFHEAP: returns off-heap PdxInstance
+ return OffHeapHelper.getHeapForm(getCd().getDeserializedWritableCopy(rgn, entry));
+ }
+
+ public Object getDeserializedValue(Region rgn, RegionEntry reentry) {
+ // TODO OFFHEAP: returns off-heap PdxInstance
+ return OffHeapHelper.getHeapForm(getCd().getDeserializedValue(rgn, reentry));
+ }
+ public Object getValue() {
+ if(this.serializedValue != null){
+ return this.serializedValue;
+ }
+ return getCd().getValue();
+ }
+ public void writeValueAsByteArray(DataOutput out) throws IOException {
+ if (this.serializedValue != null) {
+ DataSerializer.writeByteArray(this.serializedValue, out);
+ } else {
+ getCd().writeValueAsByteArray(out);
+ }
+ }
+ public void fillSerializedValue(BytesAndBitsForCompactor wrapper, byte userBits) {
+ if (this.serializedValue != null) {
+ wrapper.setData(this.serializedValue, userBits, this.serializedValue.length,
+ false /* Not Reusable as it refers to underlying value */);
+ } else {
+ getCd().fillSerializedValue(wrapper, userBits);
+ }
+ }
+ public int getValueSizeInBytes() {
+ return getCd().getValueSizeInBytes();
+ }
+ public int getSizeInBytes() {
+ return getCd().getSizeInBytes();
+ }
+
+ public String getStringForm() {
+ return getCd().getStringForm();
+ }
+
+ @Override
+ public void sendTo(DataOutput out) throws IOException {
+ DataSerializer.writeObject(getCd(), out);
+ }
+ }
+//////////////////////////////////////////////////////////////////////////////////////////
+
+ public void setTailKey(Long tailKey) {
+ this.tailKey = tailKey;
+ }
+
+ public Long getTailKey() {
+ return this.tailKey;
+ }
+
+ private Thread invokeCallbacksThread;
+
+ /**
+ * Mark this event as having its callbacks invoked by the current thread.
+ * Note this is done just before the actual invocation of the callbacks.
+ */
+ public void setCallbacksInvokedByCurrentThread() {
+ this.invokeCallbacksThread = Thread.currentThread();
+ }
+
+ /**
+ * Return true if this event was marked as having its callbacks invoked
+ * by the current thread.
+ */
+ public boolean getCallbacksInvokedByCurrentThread() {
+ if (this.invokeCallbacksThread == null) return false;
+ return Thread.currentThread().equals(this.invokeCallbacksThread);
+ }
+
+ /**
+ * Returns whether this event is on the PDX type region.
+ * @return whether this event is on the PDX type region
+ */
+ public boolean isOnPdxTypeRegion() {
+ return PeerTypeRegistration.REGION_FULL_PATH.equals(this.region.getFullPath());
+ }
+
+ /**
+ * returns true if it is okay to process this event even though it has
+ * a null version
+ */
+ public boolean noVersionReceivedFromServer() {
+ return versionTag == null
+ && region.concurrencyChecksEnabled
+ && region.getServerProxy() != null
+ && !op.isLocal()
+ && !isOriginRemote()
+ ;
+ }
+
+ /** returns a copy of this event with the additional fields for WAN conflict resolution */
+ public TimestampedEntryEvent getTimestampedEvent(
+ final int newDSID, final int oldDSID,
+ final long newTimestamp, final long oldTimestamp) {
+ return new TimestampedEntryEventImpl(this, newDSID, oldDSID, newTimestamp, oldTimestamp);
+ }
+
+ private void setSerializationDeferred(boolean serializationDeferred) {
+ setEventFlag(EventFlags.FLAG_SERIALIZATION_DEFERRED, serializationDeferred);
+ }
+
+ private boolean isSerializationDeferred() {
+ return testEventFlag(EventFlags.FLAG_SERIALIZATION_DEFERRED);
+ }
+
+ public boolean isSingleHop() {
+ return (this.causedByMessage != null && this.causedByMessage instanceof RemoteOperationMessage);
+ }
+
+ public boolean isSingleHopPutOp() {
+ return (this.causedByMessage != null && this.causedByMessage instanceof RemotePutMessage);
+ }
+
+ /**
+ * True if it is ok to use old/new values that are stored off heap.
+ * False if an exception should be thrown if an attempt is made to access old/new offheap values.
+ */
+ private transient boolean offHeapOk = true;
+
+ @Override
+ @Released({ENTRY_EVENT_NEW_VALUE, ENTRY_EVENT_OLD_VALUE})
+ public void release() {
+ // noop if already freed or values can not be off-heap
+ if (!this.offHeapOk) return;
+ // Note that this method does not set the old/new values to null but
+ // leaves them set to the off-heap value so that future calls to getOld/NewValue
+ // will fail with an exception.
+// LocalRegion lr = getLocalRegion();
+// if (lr != null) {
+// if (lr.isCacheClosing()) {
+// // to fix races during closing and recreating cache (see bug 47883) don't bother
+// // trying to decrement reference counts if we are closing the cache.
+// // TODO OFFHEAP: this will cause problems once offheap lives longer than a cache.
+// this.offHeapOk = false;
+// return;
+// }
+// }
+ Object ov = basicGetOldValue();
+ Object nv = basicGetNewValue();
+ this.offHeapOk = false;
+
+ if (ov instanceof Chunk) {
+ //this.region.getCache().getLogger().info("DEBUG freeing ref to old value on " + System.identityHashCode(ov));
+ if (ReferenceCountHelper.trackReferenceCounts()) {
+ ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
+ ((Chunk) ov).release();
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ } else {
+ ((Chunk) ov).release();
+ }
+ }
+ OffHeapHelper.releaseAndTrackOwner(nv, this);
+ }
+
+ /**
+ * Make sure that this event will never own an off-heap value.
+ * Once this is called on an event it does not need to have release called.
+ */
+ public void disallowOffHeapValues() {
+ if (this.newValue instanceof Chunk || this.oldValue instanceof Chunk) {
+ throw new IllegalStateException("This event does not support off-heap values");
+ }
+ this.offHeapOk = false;
+ }
+
+ /**
+ * This copies the off-heap new and/or old value to the heap.
+ * As a result the current off-heap new/old will be released.
+ * @throws IllegalStateException if called with an event for sqlf data.
+ */
+ @Released({ENTRY_EVENT_NEW_VALUE, ENTRY_EVENT_OLD_VALUE})
+ public void copyOffHeapToHeap() {
+ Object ov = basicGetOldValue();
+ if (ov instanceof Chunk) {
+ if (ReferenceCountHelper.trackReferenceCounts()) {
+ ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
+ this.oldValue = OffHeapHelper.copyAndReleaseIfNeeded(ov);
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ } else {
+ this.oldValue = OffHeapHelper.copyAndReleaseIfNeeded(ov);
+ }
+ }
+ Object nv = basicGetNewValue();
+ if (nv instanceof Chunk) {
+ ReferenceCountHelper.setReferenceCountOwner(this);
+ this.newValue = OffHeapHelper.copyAndReleaseIfNeeded(nv);
+ ReferenceCountHelper.setReferenceCountOwner(null);
+ }
+ if (this.newValue instanceof Chunk || this.oldValue instanceof Chunk) {
+ throw new IllegalStateException("event's old/new value still off-heap after calling copyOffHeapToHeap");
+ }
+ this.offHeapOk = false;
+ }
+
+ public boolean isOldValueOffHeap() {
+ return this.oldValue instanceof Chunk;
+ }
+ public final boolean isFetchFromHDFS() {
+ return fetchFromHDFS;
+ }
+
+ public final void setFetchFromHDFS(boolean fetchFromHDFS) {
+ this.fetchFromHDFS = fetchFromHDFS;
+ }
+
+ public final boolean isPutDML() {
+ return this.isPutDML;
+ }
+
+ public final void setPutDML(boolean val) {
+ this.isPutDML = val;
+ }
+
+ public final boolean isLoadedFromHDFS() {
+ return loadedFromHDFS;
+ }
+
+ public final void setLoadedFromHDFS(boolean loadedFromHDFS) {
+ this.loadedFromHDFS = loadedFromHDFS;
+ }
+}
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
new file mode 100644
index 000000000000..78d356f179a7
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -0,0 +1,12970 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.*;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.AbstractSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.CopyHelper;
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.DeltaSerializationException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.InternalGemFireException;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.admin.internal.SystemMemberCacheEventProcessor;
+import com.gemstone.gemfire.cache.AttributesMutator;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.cache.CacheEvent;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.CacheListener;
+import com.gemstone.gemfire.cache.CacheLoader;
+import com.gemstone.gemfire.cache.CacheLoaderException;
+import com.gemstone.gemfire.cache.CacheRuntimeException;
+import com.gemstone.gemfire.cache.CacheStatistics;
+import com.gemstone.gemfire.cache.CacheWriter;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.CustomExpiry;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskAccessException;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.DiskWriteAttributes;
+import com.gemstone.gemfire.cache.DiskWriteAttributesFactory;
+import com.gemstone.gemfire.cache.EntryDestroyedException;
+import com.gemstone.gemfire.cache.EntryExistsException;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.FailedSynchronizationException;
+import com.gemstone.gemfire.cache.InterestRegistrationEvent;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+import com.gemstone.gemfire.cache.LoaderHelper;
+import com.gemstone.gemfire.cache.LowMemoryException;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.PartitionedRegionStorageException;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.RegionEvent;
+import com.gemstone.gemfire.cache.RegionExistsException;
+import com.gemstone.gemfire.cache.RegionMembershipListener;
+import com.gemstone.gemfire.cache.RegionReinitializedException;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.StatisticsDisabledException;
+import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.cache.TransactionException;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.cache.client.ServerOperationException;
+import com.gemstone.gemfire.cache.client.SubscriptionNotEnabledException;
+import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.Endpoint;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
+import com.gemstone.gemfire.cache.control.ResourceManager;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache.hdfs.internal.HDFSBucketRegionQueue;
+import com.gemstone.gemfire.cache.hdfs.internal.HDFSIntegrationUtil;
+import com.gemstone.gemfire.cache.hdfs.internal.HoplogListenerForRegion;
+import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector;
+import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector.HdfsRegionManager;
+import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
+import com.gemstone.gemfire.cache.query.FunctionDomainException;
+import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
+import com.gemstone.gemfire.cache.query.IndexType;
+import com.gemstone.gemfire.cache.query.MultiIndexCreationException;
+import com.gemstone.gemfire.cache.query.NameResolutionException;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.query.TypeMismatchException;
+import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
+import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
+import com.gemstone.gemfire.cache.query.internal.ExecutionContext;
+import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
+import com.gemstone.gemfire.cache.query.internal.cq.CqService;
+import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
+import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
+import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
+import com.gemstone.gemfire.cache.query.internal.index.IndexUtils;
+import com.gemstone.gemfire.cache.util.ObjectSizer;
+import com.gemstone.gemfire.cache.wan.GatewaySender;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.Profile;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.ResourceEvent;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.ClassLoadUtil;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalStatisticsDisabledException;
+import com.gemstone.gemfire.internal.NanoTimer;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor.CacheProfile;
+import com.gemstone.gemfire.internal.cache.DiskInitFile.DiskRegionFlag;
+import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
+import com.gemstone.gemfire.internal.cache.InitialImageOperation.GIIStatus;
+import com.gemstone.gemfire.internal.cache.PutAllPartialResultException.PutAllPartialResult;
+import com.gemstone.gemfire.internal.cache.control.InternalResourceManager;
+import com.gemstone.gemfire.internal.cache.control.InternalResourceManager.ResourceType;
+import com.gemstone.gemfire.internal.cache.control.MemoryEvent;
+import com.gemstone.gemfire.internal.cache.control.MemoryThresholds;
+import com.gemstone.gemfire.internal.cache.control.ResourceListener;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionExecutor;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionResultSender;
+import com.gemstone.gemfire.internal.cache.execute.LocalResultCollector;
+import com.gemstone.gemfire.internal.cache.execute.RegionFunctionContextImpl;
+import com.gemstone.gemfire.internal.cache.execute.ServerToClientFunctionResultSender;
+import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier;
+import com.gemstone.gemfire.internal.cache.lru.LRUEntry;
+import com.gemstone.gemfire.internal.cache.partitioned.RedundancyAlreadyMetException;
+import com.gemstone.gemfire.internal.cache.persistence.DiskExceptionHandler;
+import com.gemstone.gemfire.internal.cache.persistence.DiskRecoveryStore;
+import com.gemstone.gemfire.internal.cache.persistence.DiskRegionView;
+import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
+import com.gemstone.gemfire.internal.cache.persistence.query.IndexMap;
+import com.gemstone.gemfire.internal.cache.persistence.query.mock.IndexMapImpl;
+import com.gemstone.gemfire.internal.cache.tier.InterestType;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientHealthMonitor;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientTombstoneMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientUpdateMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
+import com.gemstone.gemfire.internal.cache.tier.sockets.VersionedObjectList;
+import com.gemstone.gemfire.internal.cache.versions.ConcurrentCacheModificationException;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionHolder;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
+import com.gemstone.gemfire.internal.cache.versions.VersionSource;
+import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
+import com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventCallbackArgument;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
+import com.gemstone.gemfire.internal.util.concurrent.FutureResult;
+import com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch;
+import com.gemstone.gemfire.internal.util.concurrent.StoppableReadWriteLock;
+import com.gemstone.gemfire.i18n.StringId;
+
+/**
+ * Implementation of a local scoped-region. Note that this class has a different
+ * meaning starting with 3.0. In previous versions, a LocalRegion was the
+ * representation of a region in the VM. Starting with 3.0, a LocalRegion is a
+ * non-distributed region. The subclass DistributedRegion adds distribution
+ * behavior.
+ *
+ */
+@SuppressWarnings("deprecation")
+public class LocalRegion extends AbstractRegion
+ implements LoaderHelperFactory, ResourceListener,
+ DiskExceptionHandler, DiskRecoveryStore
+{
+ private static final Logger logger = LogService.getLogger();
+
+ /**
+ * Internal interface used to simulate failures when performing entry operations
+ * @since 5.7
+ */
+ public interface TestCallable {
+ public void call(LocalRegion r, Operation op, RegionEntry re);
+ }
+
+ // view types for iterators
+ public enum IteratorType {
+ KEYS, VALUES, ENTRIES
+ }
+
+ // iniitialization level
+ public static final int AFTER_INITIAL_IMAGE = 0;
+
+ public static final int BEFORE_INITIAL_IMAGE = 1;
+
+ public static final int ANY_INIT = 2;
+
+ /**
+ * thread local to indicate that this thread should bypass the initialization
+ * Latch
+ */
+ private static final ThreadLocal initializationThread = new ThreadLocal();
+
+ /* thread local to indicate its for persist data convert tool */
+ protected static final ThreadLocal isConversion = new ThreadLocal();
+
+ // user attributes //
+ private Object regionUserAttribute;
+
+ protected Map entryUserAttributes; // @todo darrel: shouldn't this be an
+
+ // identity map whose key is a RegionEntry?
+
+ private final String regionName;
+
+ protected final LocalRegion parentRegion;
+
+ // set to true only if isDestroyed is also true
+ // and region is about to be recreated due to reinitialization by loading
+ // of a snapshot, etc.
+ private volatile boolean reinitialized_old = false;
+
+ protected volatile boolean isDestroyed = false;
+
+ // In case of parallel wan, when a destroy is called on userPR, it waits for
+ // parallelQueue to drain and then destroys paralleQueue. In this time if
+ // operation like put happens on userPR then it will keep on building parallel
+ // queue increasing time of userPR to get destroyed.this volatile boolean will
+ // block such put operation by throwing RegionDestroyedException
+ protected volatile boolean isDestroyedForParallelWAN = false;
+
+ // set to true after snapshot is loaded, to help get initial image
+ // make sure this is the right incarnation of this region
+ private volatile boolean reinitialized_new = false;
+
+ /** Lock used to prevent multiple concurrent destroy region operations */
+ private Semaphore destroyLock;
+
+ //guarded by regionExpiryLock.
+ private RegionTTLExpiryTask regionTTLExpiryTask = null;
+ //guarded by regionExpiryLock.
+ private RegionIdleExpiryTask regionIdleExpiryTask = null;
+
+ private final Object regionExpiryLock = new Object();
+ // guarded by regionExpiryLock. Keeps track of how many txs are writing to this region.
+ private int txRefCount;
+
+ private final ConcurrentHashMap entryExpiryTasks = new ConcurrentHashMap();
+
+ /**
+ * Set to true after an invalidate region expiration so we don't get multiple
+ * expirations
+ */
+ volatile boolean regionInvalid = false;
+
+ public final RegionMap entries;
+
+ /**
+ * Set to true if this region supports transaction else false.
+ */
+ private final boolean supportsTX;
+
+ /** tracks threadID->seqno information for this region */
+ protected EventTracker eventTracker;
+
+ /** tracks region-level version information for members. See
+ * https://wiki.gemstone.com/display/gfe70/Consistency+in+Replicated+Regions+and+WAN
+ */
+ private RegionVersionVector versionVector;
+
+ private static final Pattern[] QUERY_PATTERNS = new Pattern[] {
+ Pattern.compile("^\\(*select .*", Pattern.CASE_INSENSITIVE
+ | Pattern.UNICODE_CASE | Pattern.DOTALL),
+ Pattern.compile("^import .*", Pattern.CASE_INSENSITIVE
+ | Pattern.UNICODE_CASE | Pattern.DOTALL) };
+
+
+ public static final String EXPIRY_MS_PROPERTY = "gemfire.EXPIRY_UNITS_MS";
+
+ /**
+ * Used by unit tests to set expiry to milliseconds instead of the default
+ * seconds. Used in ExpiryTask.
+ *
+ * @since 5.0
+ */
+ final boolean EXPIRY_UNITS_MS;
+
+ // Indicates that the entries are in fact initialized. It turns out
+ // you can't trust the assignment of a volatile (as indicated above)
+ // to mean that the the thing being assigned is fully formed, only
+ // those things *before* the assignment are fully formed. mthomas 10/02/2005
+ private volatile boolean entriesInitialized;
+
+ /**
+ * contains Regions themselves // marked volatile to make sure it is fully
+ * initialized before being // accessed; (actually should be final)
+ */
+ protected volatile ConcurrentMap subregions;
+
+ private final Object subregionsLock = new Object();
+
+ // Used for synchronizzing access to client Cqs
+// private final Object clientCqsSync = new Object();
+
+ /**
+ * Prevents access to this region until it is done initializing, except for
+ * some special initializing operations such as replying to create region
+ * messages In JDK 1.5 we will use java.util.concurrent.CountDownLatch instead
+ * of com.gemstone.gemfire.internal.util.CountDownLatch.
+ */
+ protected final StoppableCountDownLatch initializationLatchBeforeGetInitialImage;
+
+ protected final StoppableCountDownLatch initializationLatchAfterGetInitialImage;
+
+ /**
+ * Used to hold off cache listener events until the afterRegionCreate is
+ * called
+ *
+ * @since 5.0
+ */
+ private final StoppableCountDownLatch afterRegionCreateEventLatch;
+
+ /**
+ * Set to true the first time isInitialized returns true.
+ */
+ private volatile boolean initialized = false; // added for bug 30223
+
+ /** Used for accessing region data on disk */
+ private final DiskRegion diskRegion;
+
+ /**
+ * Used by transactions to suspend entry expiration while a transaction is in
+ * progress on a region. This field is only initialized if expiration is
+ * configured and transactions are possible.
+ */
+ private volatile StoppableReadWriteLock txExpirationLock;
+
+ /**
+ * Used for serializing netSearch and netLoad on a per key basis.
+ * CM
+ */
+ protected final ConcurrentMap getFutures = new ConcurrentHashMap();
+
+ /*
+ * Asif: This boolean needs to be made true if the test needs to receive a
+ * synchronous callback just after clear on map is done. Its visibility is
+ * default so that only tests present in com.gemstone.gemfire.internal.cache
+ * will be able to see it
+ */
+ public static boolean ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
+
+ /**
+ * A flag used to indicate that this Region is being used as an administrative
+ * Region, holding meta-data for a PartitionedRegion
+ */
+ final private boolean isUsedForPartitionedRegionAdmin;
+
+ final private boolean isUsedForPartitionedRegionBucket;
+
+ final private boolean isUsedForMetaRegion;
+
+ final private boolean isMetaRegionWithTransactions;
+
+ final private boolean isUsedForSerialGatewaySenderQueue;
+
+ final private boolean isUsedForParallelGatewaySenderQueue;
+
+ final private AbstractGatewaySender serialGatewaySender;
+
+ /**
+ * The factory used to create the LoaderHelper when a loader is invoked
+ */
+ protected final LoaderHelperFactory loaderHelperFactory;
+
+ /**
+ * Allow for different cacheperfstats locations... primarily for PartitionedRegions
+ */
+ private final CachePerfStats cachePerfStats;
+ private final boolean hasOwnStats;
+
+
+ private final ImageState imageState;
+ /**
+ * Register interest count to track if any register interest is in progress for
+ * this region. This count will be incremented when register interest starts
+ * and decremented when register interest finishes.
+ * @guarded.By {@link #imageState}
+ */
+ private int riCnt = 0; /*
+ * since always written while holding an exclusive write lock
+ * and only read while holding a read lock
+ * it does not need to be atomic or
+ * protected by any other sync.
+ */
+
+
+ /**
+ * Map of subregion full paths to serial numbers. These are subregions that
+ * were destroyed when this region was destroyed. This map remains null until
+ * this region is destroyed.
+ */
+ private volatile HashMap destroyedSubregionSerialNumbers;
+
+ /**
+ * This boolean is true when a member who has this region is running low on memory.
+ * It is used to reject region operations.
+ */
+ public final AtomicBoolean memoryThresholdReached = new AtomicBoolean(false);
+
+ // Lock for updating PR MetaData on client side
+ public final Lock clientMetaDataLock = new ReentrantLock();
+
+
+ protected HdfsRegionManager hdfsManager;
+ protected HoplogListenerForRegion hoplogListener;
+
+ /**
+ * There seem to be cases where a region can be created and yet the
+ * distributed system is not yet in place...
+ *
+ *
+ */
+ protected class Stopper extends CancelCriterion {
+
+ @Override
+ public String cancelInProgress() {
+ // ---
+ // This grossness is necessary because there are instances where the
+ // region can exist without having a cache (XML creation)
+ checkFailure();
+ Cache c = LocalRegion.this.getCache();
+ if (c == null) {
+ return LocalizedStrings.LocalRegion_THE_CACHE_IS_NOT_AVAILABLE.toLocalizedString();
+ }
+ // --- end of grossness
+ return c.getCancelCriterion().cancelInProgress();
+ }
+
+ /* (non-Javadoc)
+ * @see com.gemstone.gemfire.CancelCriterion#generateCancelledException(java.lang.Throwable)
+ */
+ @Override
+ public RuntimeException generateCancelledException(Throwable e) {
+ // ---
+ // This grossness is necessary because there are instances where the
+ // region can exist without having a cache (XML creation)
+ checkFailure();
+ Cache c = LocalRegion.this.getCache();
+ if (c == null) {
+ return new CacheClosedException("No cache", e);
+ }
+ // --- end of grossness
+ return c.getCancelCriterion().generateCancelledException(e);
+ }
+
+ }
+
+ protected final CancelCriterion stopper = createStopper();
+
+ protected CancelCriterion createStopper() {
+ return new Stopper();
+ }
+
+ private final TestCallable testCallable;
+
+ /**
+ * ThreadLocal used to set the current region being initialized.
+ *
+ * Currently used by the OpLog layer to initialize the
+ * {@link KeyWithRegionContext} if required.
+ */
+ private final static ThreadLocal initializingRegion =
+ new ThreadLocal();
+
+ /**
+ * Set to true if the region contains keys implementing
+ * {@link KeyWithRegionContext} that require setting up of region specific
+ * context after deserialization or recovery from disk.
+ */
+ private boolean keyRequiresRegionContext;
+
+ /**
+ * Get the current initializing region as set in the ThreadLocal.
+ *
+ * Note that this value is cleared after the initialization of LocalRegion is
+ * done so is valid only for the duration of region creation and
+ * initialization.
+ */
+ public static LocalRegion getInitializingRegion() {
+ return initializingRegion.get();
+ }
+
+ /**
+ * Return true if the keys of this region implement
+ * {@link KeyWithRegionContext} that require region specific context
+ * initialization after deserialization or recovery from disk.
+ *
+ * Currently used by SQLFabric for the optimized
+ * CompactCompositeRegionKey
that points to the raw row bytes and
+ * so requires a handle to table schema for interpretation of those bytes.
+ */
+ public final boolean keyRequiresRegionContext() {
+ return this.keyRequiresRegionContext;
+ }
+
+ /**
+ * Set the {@link #keyRequiresRegionContext} flag to given value.
+ */
+ public final void setKeyRequiresRegionContext(boolean v) {
+ this.keyRequiresRegionContext = v;
+ }
+
+ public CancelCriterion getCancelCriterion() {
+ return this.stopper;
+ }
+
+ ////////////////// Public Methods ///////////////////////////////////////////
+
+ static String calcFullPath(String regionName, LocalRegion parentRegion) {
+ StringBuilder buf = null;
+ if (parentRegion == null) {
+ buf = new StringBuilder(regionName.length() + 1);
+ }
+ else {
+ String parentFull = parentRegion.getFullPath();
+ buf = new StringBuilder(parentFull.length() + regionName.length() + 1);
+ buf.append(parentFull);
+ }
+ buf.append(SEPARATOR).append(regionName);
+ return buf.toString();
+ }
+
+ /**
+ * Creates new region
+ */
+ protected LocalRegion(String regionName, RegionAttributes attrs,
+ LocalRegion parentRegion, GemFireCacheImpl cache,
+ InternalRegionArguments internalRegionArgs) throws DiskAccessException {
+ super(cache, attrs,regionName, internalRegionArgs);
+ // Initialized here (and defers to parent) to fix GEODE-128
+ this.EXPIRY_UNITS_MS = parentRegion != null ? parentRegion.EXPIRY_UNITS_MS : Boolean.getBoolean(EXPIRY_MS_PROPERTY);
+
+ Assert.assertTrue(regionName != null, "regionName must not be null");
+ this.sharedDataView = buildDataView();
+ this.regionName = regionName;
+ this.parentRegion = parentRegion;
+ this.fullPath = calcFullPath(regionName, parentRegion);
+
+ String myName = getFullPath();
+ if (internalRegionArgs.getPartitionedRegion() != null) {
+ myName = internalRegionArgs.getPartitionedRegion().getFullPath();
+ }
+ this.offHeap = attrs.getOffHeap() || Boolean.getBoolean(myName+":OFF_HEAP");
+ if (getOffHeap()) {
+ if (cache.getOffHeapStore() == null) {
+ throw new IllegalStateException(LocalizedStrings.
+ LocalRegion_THE_REGION_0_WAS_CONFIGURED_TO_USE_OFF_HEAP_MEMORY_BUT_OFF_HEAP_NOT_CONFIGURED.toLocalizedString(myName));
+ }
+ }
+
+ this.initializationLatchBeforeGetInitialImage = new StoppableCountDownLatch(this.stopper, 1);
+ this.initializationLatchAfterGetInitialImage = new StoppableCountDownLatch(this.stopper, 1);
+ this.afterRegionCreateEventLatch = new StoppableCountDownLatch(this.stopper, 1);
+
+ // set the user-attribute object upfront for SQLFabric
+ if (internalRegionArgs.getUserAttribute() != null) {
+ setUserAttribute(internalRegionArgs.getUserAttribute());
+ }
+ setKeyRequiresRegionContext(internalRegionArgs.keyRequiresRegionContext());
+ initializingRegion.set(this);
+
+ if (internalRegionArgs.getCachePerfStatsHolder() != null) {
+ this.hasOwnStats = false;
+ this.cachePerfStats = internalRegionArgs.getCachePerfStatsHolder()
+ .getCachePerfStats();
+ }
+ else {
+ if (attrs.getPartitionAttributes() != null || isInternalRegion()
+ || internalRegionArgs.isUsedForMetaRegion()) {
+ this.hasOwnStats = false;
+ this.cachePerfStats = cache.getCachePerfStats();
+ }
+ else {
+ this.hasOwnStats = true;
+ this.cachePerfStats = new RegionPerfStats(cache, cache.getCachePerfStats(), regionName);
+ }
+ }
+
+ this.hdfsManager = initHDFSManager();
+ this.dsi = findDiskStore(attrs, internalRegionArgs);
+ this.diskRegion = createDiskRegion(internalRegionArgs);
+ this.entries = createRegionMap(internalRegionArgs);
+ this.entriesInitialized = true;
+ this.subregions = new ConcurrentHashMap();
+ // we only need a destroy lock if this is a root
+ if (parentRegion == null) {
+ initRoot();
+ }
+ if (internalRegionArgs.getLoaderHelperFactory() != null) {
+ this.loaderHelperFactory = internalRegionArgs.getLoaderHelperFactory();
+ }
+ else {
+ this.loaderHelperFactory = this;
+ }
+
+ this.isUsedForPartitionedRegionAdmin = internalRegionArgs
+ .isUsedForPartitionedRegionAdmin();
+ this.isUsedForPartitionedRegionBucket = internalRegionArgs
+ .isUsedForPartitionedRegionBucket();
+ this.isUsedForMetaRegion = internalRegionArgs
+ .isUsedForMetaRegion();
+ this.isMetaRegionWithTransactions = internalRegionArgs.isMetaRegionWithTransactions();
+ this.isUsedForSerialGatewaySenderQueue = internalRegionArgs.isUsedForSerialGatewaySenderQueue();
+ this.isUsedForParallelGatewaySenderQueue = internalRegionArgs.isUsedForParallelGatewaySenderQueue();
+ this.serialGatewaySender = internalRegionArgs.getSerialGatewaySender();
+
+ if (!isUsedForMetaRegion && !isUsedForPartitionedRegionAdmin
+ && !isUsedForPartitionedRegionBucket
+ && !isUsedForSerialGatewaySenderQueue
+ && !isUsedForParallelGatewaySenderQueue) {
+ this.filterProfile = new FilterProfile(this);
+ }
+
+ // initialize client to server proxy
+ this.srp = (this.getPoolName() != null)
+ ? new ServerRegionProxy(this)
+ : null;
+ this.imageState =
+ new UnsharedImageState(this.srp != null,
+ getDataPolicy().withReplication() || getDataPolicy().isPreloaded(),
+ getAttributes().getDataPolicy().withPersistence(),
+ this.stopper);
+
+ createEventTracker();
+
+ // prevent internal regions from participating in a TX, bug 38709
+ this.supportsTX = !isSecret() && !isUsedForPartitionedRegionAdmin()
+ && !isUsedForMetaRegion() || isMetaRegionWithTransactions();
+
+ this.testCallable = internalRegionArgs.getTestCallable();
+
+ }
+
+ private HdfsRegionManager initHDFSManager() {
+ HdfsRegionManager hdfsMgr = null;
+ if (this.getHDFSStoreName() != null) {
+ this.hoplogListener = new HoplogListenerForRegion();
+ HDFSRegionDirector.getInstance().setCache(cache);
+ hdfsMgr = HDFSRegionDirector.getInstance().manageRegion(this,
+ this.getHDFSStoreName(), hoplogListener);
+ }
+ return hdfsMgr;
+ }
+
+ private RegionMap createRegionMap(InternalRegionArguments internalRegionArgs) {
+ RegionMap result = null;
+ if ((internalRegionArgs.isReadWriteHDFSRegion()) && this.diskRegion != null) {
+ this.diskRegion.setEntriesMapIncompatible(true);
+ }
+ if (this.diskRegion != null) {
+ result = this.diskRegion.useExistingRegionMap(this);
+ }
+ if (result == null) {
+ RegionMap.Attributes ma = new RegionMap.Attributes();
+ ma.statisticsEnabled = this.statisticsEnabled;
+ ma.loadFactor = this.loadFactor;
+ ma.initialCapacity = this.initialCapacity;
+ ma.concurrencyLevel = this.concurrencyLevel;
+ result = RegionMapFactory.createVM(this, ma, internalRegionArgs);
+ }
+ return result;
+ }
+
+ protected InternalDataView buildDataView() {
+ return new LocalRegionDataView();
+ }
+
+ /**
+ * initialize the event tracker. Not all region implementations want or
+ * need one of these. Regions that require one should reimplement this method
+ * and create one like so:
+ *
+ * this.eventTracker = new EventTracker(this.cache);
+ * this.eventTracker.start();
+ *
+ */
+ void createEventTracker() {
+ // if LocalRegion is changed to have an event tracker, then the initialize()
+ // method should be changed to set it to "initialized" state when the
+ // region finishes initialization
+ }
+
+
+ /**
+ * Test method for getting the event tracker.
+ *
+ * this method is for testing only. Other region classes may track events using
+ * different mechanisms than EventTrackers
+ */
+ protected EventTracker getEventTracker() {
+ return this.eventTracker;
+ }
+
+ /** returns the regions version-vector */
+ public RegionVersionVector getVersionVector() {
+ return this.versionVector;
+ }
+
+ /** returns object used to guard the size() operation during tombstone removal */
+ public Object getSizeGuard() {
+ if (!this.concurrencyChecksEnabled) {
+ return new Object();
+ } else {
+ return this.fullPath; // avoids creating another sync object - could be anything unique to this region
+ }
+ }
+
+ /** initializes a new version vector for this region */
+ protected void createVersionVector() {
+
+ this.versionVector = RegionVersionVector.create(getVersionMember(), this);
+
+ if (dataPolicy.withPersistence()) {
+ //copy the versions that we have recovered from disk into
+ //the version vector.
+ RegionVersionVector diskVector = this.diskRegion.getRegionVersionVector();
+ this.versionVector.recordVersions(diskVector.getCloneForTransmission());
+ } else if (!dataPolicy.withStorage()) {
+ // version vectors are currently only necessary in empty regions for
+ // tracking canonical member IDs
+ this.versionVector.turnOffRecordingForEmptyRegion();
+ }
+ if (this.srp != null) {
+ this.versionVector.setIsClientVector();
+ }
+ this.cache.getDistributionManager().addMembershipListener(this.versionVector);
+ }
+
+ @Override
+ protected void updateEntryExpiryPossible()
+ {
+ super.updateEntryExpiryPossible();
+ if (!isEntryExpiryPossible()) {
+ // since expiration is no longer possible cleanup the tasks
+ cancelAllEntryExpiryTasks();
+ }
+ }
+
+ public IndexUpdater getIndexUpdater() {
+ return this.entries.getIndexUpdater();
+ }
+
+ boolean isCacheClosing()
+ {
+ return this.cache.isClosed();
+ }
+
+ public RegionEntry getRegionEntry(Object key) {
+ return this.entries.getEntry(key);
+ }
+
+ /**
+ * Test hook - returns the version stamp for an entry in the form of a
+ * version tag
+ * @param key
+ * @return the entry version information
+ */
+ public VersionTag getVersionTag(Object key) {
+ Region.Entry entry = getEntry(key, true);
+ VersionTag tag = null;
+ if (entry != null && entry instanceof EntrySnapshot) {
+ tag = ((EntrySnapshot)entry).getVersionTag();
+ } else if (entry != null && entry instanceof NonTXEntry) {
+ tag = ((NonTXEntry)entry).getRegionEntry().getVersionStamp().asVersionTag();
+ }
+ return tag;
+ }
+
+ /** removes any destroyed entries from the region and clear the destroyedKeys
+ * assert: Caller must be holding writeLock on is
+ */
+ private void destroyEntriesAndClearDestroyedKeysSet() {
+ ImageState is = getImageState();
+ Iterator iter = is.getDestroyedEntries();
+ while (iter.hasNext()) {
+ Object key = iter.next();
+ // destroy the entry which has value Token.DESTROYED
+ // If it is Token.DESTROYED then only destroy it.
+ this.entries.removeIfDestroyed(key); // fixes bug 41957
+ }
+ }
+
+ /**
+ * @since 5.7
+ */
+ protected final ServerRegionProxy srp;
+
+ private final InternalDataView sharedDataView;
+
+ public final ServerRegionProxy getServerProxy() {
+ return this.srp;
+ }
+
+ public final boolean hasServerProxy() {
+ return this.srp != null;
+ }
+
+ /** Returns true if the ExpiryTask is currently allowed to expire. */
+ protected boolean isExpirationAllowed(ExpiryTask expiry)
+ {
+ return true;
+ }
+
+ void performExpiryTimeout(ExpiryTask p_task) throws CacheException
+ {
+ if (p_task != null) {
+ p_task.basicPerformTimeout(false);
+ }
+ }
+
+ private void initRoot()
+ {
+ this.destroyLock = new Semaphore(1);
+ }
+
+ public void handleMarker() {
+
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.MARKER, null, false, getMyId(),
+ false /* generate EventID */);
+
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REGION_LIVE, event);
+ }
+
+ public AttributesMutator getAttributesMutator()
+ {
+ checkReadiness();
+ return this;
+ }
+
+ public Region createSubregion(String subregionName,
+ RegionAttributes regionAttributes) throws RegionExistsException,
+ TimeoutException
+ {
+ try {
+ return createSubregion(subregionName, regionAttributes,
+ new InternalRegionArguments().setDestroyLockFlag(true)
+ .setRecreateFlag(false));
+ }
+ catch (IOException e) {
+ // only happens when loading a snapshot, not here
+ InternalGemFireError assErr = new InternalGemFireError(LocalizedStrings.LocalRegion_UNEXPECTED_EXCEPTION.toLocalizedString());
+ assErr.initCause(e);
+ throw assErr;
+
+ }
+ catch (ClassNotFoundException e) {
+ // only happens when loading a snapshot, not here
+ InternalGemFireError assErr = new InternalGemFireError(LocalizedStrings.LocalRegion_UNEXPECTED_EXCEPTION.toLocalizedString());
+ assErr.initCause(e);
+ throw assErr;
+
+ }
+ }
+
+ /**
+ * Returns the member id of my distributed system
+ *
+ * @since 5.0
+ */
+ @Override
+ protected InternalDistributedMember getMyId()
+ {
+ return this.cache.getMyId();
+ }
+
+ public VersionSource getVersionMember()
+ {
+ if(dataPolicy.withPersistence()) {
+ return getDiskStore().getDiskStoreID();
+ } else {
+ return this.cache.getMyId();
+ }
+ }
+
+ public Region createSubregion(String subregionName,
+ RegionAttributes attrs,
+ InternalRegionArguments internalRegionArgs) throws RegionExistsException,
+ TimeoutException, IOException, ClassNotFoundException
+ {
+ checkReadiness();
+ LocalRegion newRegion = null;
+ RegionAttributes regionAttributes = attrs;
+ attrs = cache.invokeRegionBefore(this, subregionName, attrs, internalRegionArgs);
+ final InputStream snapshotInputStream = internalRegionArgs
+ .getSnapshotInputStream();
+ final boolean getDestroyLock = internalRegionArgs.getDestroyLockFlag();
+ final InternalDistributedMember imageTarget = internalRegionArgs
+ .getImageTarget();
+ try {
+ if (getDestroyLock)
+ acquireDestroyLock();
+ LocalRegion existing = null;
+ try {
+ if (isDestroyed()) {
+ if (this.reinitialized_old) {
+ throw new RegionReinitializedException(toString(), getFullPath());
+ }
+ throw new RegionDestroyedException(toString(), getFullPath());
+ }
+ validateRegionName(subregionName);
+
+ validateSubregionAttributes(regionAttributes);
+ String regionPath = calcFullPath(subregionName, this);
+
+ // lock down the subregionsLock
+ // to prevent other threads from adding a region to it in toRegion
+ // but don't wait on initialization while synchronized (distributed
+ // deadlock)
+ synchronized (this.subregionsLock) {
+
+ existing = (LocalRegion)this.subregions.get(subregionName);
+
+ if (existing == null) {
+ // create the async queue for HDFS if required.
+ HDFSIntegrationUtil.createAndAddAsyncQueue(regionPath,
+ regionAttributes, this.cache);
+ regionAttributes = cache.setEvictionAttributesForLargeRegion(
+ regionAttributes);
+ if (regionAttributes.getScope().isDistributed()
+ && internalRegionArgs.isUsedForPartitionedRegionBucket()) {
+ final PartitionedRegion pr = internalRegionArgs
+ .getPartitionedRegion();
+ internalRegionArgs.setIndexUpdater(pr.getIndexUpdater());
+ internalRegionArgs.setUserAttribute(pr.getUserAttribute());
+ internalRegionArgs.setKeyRequiresRegionContext(pr
+ .keyRequiresRegionContext());
+ if (pr.isShadowPR()) {
+ if (!pr.isShadowPRForHDFS()) {
+ newRegion = new BucketRegionQueue(subregionName, regionAttributes,
+ this, this.cache, internalRegionArgs);
+ }
+ else {
+ newRegion = new HDFSBucketRegionQueue(subregionName, regionAttributes,
+ this, this.cache, internalRegionArgs);
+ }
+
+ } else {
+ newRegion = new BucketRegion(subregionName, regionAttributes,
+ this, this.cache, internalRegionArgs);
+ }
+ }
+ else if (regionAttributes.getPartitionAttributes() != null) {
+ newRegion = new PartitionedRegion(subregionName,
+ regionAttributes, this, this.cache, internalRegionArgs);
+ }
+ else {
+ boolean local = regionAttributes.getScope().isLocal();
+ newRegion = local ? new LocalRegion(subregionName,
+ regionAttributes, this, this.cache, internalRegionArgs)
+ : new DistributedRegion(subregionName, regionAttributes,
+ this, this.cache, internalRegionArgs);
+ }
+ Object o = this.subregions.putIfAbsent(subregionName, newRegion);
+
+ Assert.assertTrue(o == null);
+
+ Assert.assertTrue(!newRegion.isInitialized());
+
+ //
+ if (logger.isDebugEnabled()) {
+ logger.debug("Subregion created: {}", newRegion.getFullPath());
+ }
+ if (snapshotInputStream != null || imageTarget != null
+ || internalRegionArgs.getRecreateFlag()) {
+ this.cache.regionReinitialized(newRegion); // fix for bug 33534
+ }
+
+ } // endif: existing == null
+ } // end synchronization
+ }
+ finally {
+ if (getDestroyLock)
+ releaseDestroyLock();
+ }
+
+ //Fix for bug 42127 - moved to outside of the destroy lock.
+ if (existing != null) {
+ // now outside of synchronization we must wait for appropriate
+ // initialization on existing region before returning a reference to
+ // it
+ existing.waitOnInitialization();
+ // fix for bug 32570
+ throw new RegionExistsException(existing);
+ }
+
+
+ boolean success = false;
+ try {
+ newRegion.checkReadiness();
+ this.cache.setRegionByPath(newRegion.getFullPath(), newRegion);
+ if (regionAttributes instanceof UserSpecifiedRegionAttributes){
+ internalRegionArgs.setIndexes((
+ (UserSpecifiedRegionAttributes)regionAttributes).getIndexes());
+ }
+ newRegion.initialize(snapshotInputStream, imageTarget, internalRegionArgs); // releases initialization Latches
+ //register the region with resource manager to get memory events
+ if(!newRegion.isInternalRegion()){
+ if (!newRegion.isDestroyed) {
+ cache.getResourceManager().addResourceListener(ResourceType.MEMORY, newRegion);
+
+ if (!newRegion.getOffHeap()) {
+ newRegion.initialCriticalMembers(cache.getResourceManager().getHeapMonitor().getState().isCritical(), cache
+ .getResourceAdvisor().adviseCritialMembers());
+ } else {
+ newRegion.initialCriticalMembers(cache.getResourceManager().getHeapMonitor().getState().isCritical()
+ || cache.getResourceManager().getOffHeapMonitor().getState().isCritical(), cache.getResourceAdvisor()
+ .adviseCritialMembers());
+ }
+
+ // synchronization would be done on ManagementAdapter.regionOpLock
+ // instead of destroyLock in LocalRegion? ManagementAdapter is one
+ // of the Resource Event listeners
+
+ InternalDistributedSystem system = this.cache
+ .getDistributedSystem();
+ system.handleResourceEvent(ResourceEvent.REGION_CREATE, newRegion);
+ }
+ }
+ success = true;
+ } catch (CancelException | RegionDestroyedException | RedundancyAlreadyMetException e) {
+ // don't print a call stack
+ throw e;
+ } catch (final RuntimeException validationException) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_INITIALIZATION_FAILED_FOR_REGION_0,
+ getFullPath()), validationException);
+ throw validationException;
+ }
+ finally {
+ if (!success) {
+ this.cache.setRegionByPath(newRegion.getFullPath(), null);
+ initializationFailed(newRegion);
+ cache.getResourceManager(false).removeResourceListener(newRegion);
+ }
+ }
+
+ newRegion.postCreateRegion();
+ }
+ finally {
+ // make sure region initialization latch is open regardless
+ // before returning;
+ // if the latch is not open at this point, then an exception must
+ // have occurred
+ if (newRegion != null && !newRegion.isInitialized()) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Region initialize latch is closed, Error must have occurred");
+ }
+ }
+ }
+
+ cache.invokeRegionAfter(newRegion);
+ return newRegion;
+ }
+
+ public final void create(Object key, Object value, Object aCallbackArgument)
+ throws TimeoutException, EntryExistsException, CacheWriterException {
+ long startPut = CachePerfStats.getStatTime();
+ EntryEventImpl event = newCreateEntryEvent(key, value, aCallbackArgument);
+ validatedCreate(event, startPut);
+ // TODO OFFHEAP: validatedCreate calls freeOffHeapResources
+ }
+
+ public final void validatedCreate(EntryEventImpl event, long startPut)
+ throws TimeoutException, EntryExistsException, CacheWriterException {
+
+ try {
+ if (event.getEventId() == null && generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ assert event.isFetchFromHDFS() : "validatedPut() should have been called";
+ // Fix for 42448 - Only make create with null a local invalidate for
+ // normal regions. Otherwise, it will become a distributed invalidate.
+ if (getDataPolicy() == DataPolicy.NORMAL) {
+ event.setLocalInvalid(true);
+ }
+ discoverJTA();
+ if (!basicPut(event, true, // ifNew
+ false, // ifOld
+ null, // expectedOldValue
+ true // requireOldValue TODO txMerge why is oldValue required for
+ // create? I think so that the EntryExistsException will have it.
+ )) {
+ throw new EntryExistsException(event.getKey().toString(),
+ event.getOldValue());
+ } else {
+ if (!getDataView().isDeferredStats()) {
+ getCachePerfStats().endPut(startPut, false);
+ }
+ }
+ } finally {
+
+ event.release();
+
+ }
+ }
+
+ // split into a separate newCreateEntryEvent since SQLFabric may need to
+ // manipulate event before doing the put (e.g. posDup flag)
+ public final EntryEventImpl newCreateEntryEvent(Object key, Object value,
+ Object aCallbackArgument) {
+
+ validateArguments(key, value, aCallbackArgument);
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+
+ return EntryEventImpl.create(this, Operation.CREATE, key,
+ value, aCallbackArgument, false, getMyId())
+ /* to distinguish genuine create */.setCreate(true);
+ }
+
+ /**
+ * The default Region implementation will generate EvenTID in the EntryEvent
+ * object. This method is overridden in special Region objects like HARegion
+ * or SingleWriteSingleReadRegionQueue.SingleReadWriteMetaRegion to return
+ * false as the event propagation from those regions do not need EventID
+ * objects
+ *
+ * author Asif
+ * @return boolean indicating whether to generate eventID or not
+ */
+ @Override
+ public boolean generateEventID()
+ {
+ return !(isUsedForPartitionedRegionAdmin()
+ || isUsedForPartitionedRegionBucket() );
+ }
+
+ public final Object destroy(Object key, Object aCallbackArgument)
+ throws TimeoutException, EntryNotFoundException, CacheWriterException {
+ EntryEventImpl event = newDestroyEntryEvent(key, aCallbackArgument);
+ return validatedDestroy(key, event);
+ // TODO OFFHEAP: validatedDestroy calls freeOffHeapResources
+ }
+
+ /**
+ * Destroys entry without performing validations. Call this after validating
+ * key, callback arg, and runtime state.
+ */
+ public Object validatedDestroy(Object key, EntryEventImpl event)
+ throws TimeoutException, EntryNotFoundException, CacheWriterException
+ {
+ try {
+ if (event.getEventId() == null && generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ basicDestroy(event, true, // cacheWrite
+ null); // expectedOldValue
+ if (event.isOldValueOffHeap()) {
+ return null;
+ } else {
+ return handleNotAvailable(event.getOldValue());
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+ // split into a separate newDestroyEntryEvent since SQLFabric may need to
+ // manipulate event before doing the put (e.g. posDup flag)
+ public final EntryEventImpl newDestroyEntryEvent(Object key,
+ Object aCallbackArgument) {
+ validateKey(key);
+ validateCallbackArg(aCallbackArgument);
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+
+ return EntryEventImpl.create(this, Operation.DESTROY, key,
+ null/* newValue */, aCallbackArgument, false, getMyId());
+ }
+
+ public void destroyRegion(Object aCallbackArgument)
+ throws CacheWriterException, TimeoutException
+ {
+ getDataView().checkSupportsRegionDestroy();
+ checkForLimitedOrNoAccess();
+
+ RegionEventImpl event = new RegionEventImpl(this, Operation.REGION_DESTROY,
+ aCallbackArgument, false, getMyId(), generateEventID());
+ basicDestroyRegion(event, true);
+ }
+
+ public InternalDataView getDataView() {
+ final TXStateInterface tx = getTXState();
+ if (tx == null) {
+ return this.sharedDataView;
+ }
+ return tx;
+ }
+
+ /**
+ * Fetch the de-serialized value from non-transactional state.
+ * @param keyInfo to which the value is associated
+ * @param updateStats true if the entry stats should be updated.
+ * @param disableCopyOnRead if true then disable copy on read
+ * @param preferCD true if the preferred result form is CachedDeserializable
+ * @param clientEvent client's event, if any (for version tag retrieval)
+ * @param returnTombstones whether destroyed entries should be returned
+ * @param retainResult if true then the result may be a retained off-heap reference
+ * @return the value for the given key
+ */
+ public final Object getDeserializedValue(RegionEntry re, final KeyInfo keyInfo, final boolean updateStats, boolean disableCopyOnRead,
+ boolean preferCD, EntryEventImpl clientEvent, boolean returnTombstones, boolean allowReadFromHDFS, boolean retainResult) {
+ if (this.diskRegion != null) {
+ this.diskRegion.setClearCountReference();
+ }
+ try {
+ if (re == null) {
+ if (allowReadFromHDFS) {
+ re = this.entries.getEntry(keyInfo.getKey());
+ } else {
+ re = this.entries.getOperationalEntryInVM(keyInfo.getKey());
+ }
+ }
+ //skip updating the stats if the value is null
+ // TODO - We need to clean up the callers of the this class so that we can
+ // update the statistics here, where it would make more sense.
+ if (re == null) {
+ return null;
+ }
+ final Object value;
+ if (clientEvent != null && re.getVersionStamp() != null) {
+ // defer the lruUpdateCallback to prevent a deadlock (see bug 51121).
+ final boolean disabled = this.entries.disableLruUpdateCallback();
+ try {
+ synchronized(re) { // bug #51059 value & version must be obtained atomically
+ clientEvent.setVersionTag(re.getVersionStamp().asVersionTag());
+ value = getDeserialized(re, updateStats, disableCopyOnRead, preferCD, retainResult);
+ }
+ } finally {
+ if (disabled) {
+ this.entries.enableLruUpdateCallback();
+ }
+ try {
+ this.entries.lruUpdateCallback();
+ }catch( DiskAccessException dae) {
+ this.handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+ } else {
+ value = getDeserialized(re, updateStats, disableCopyOnRead, preferCD, retainResult);
+ }
+ if (logger.isTraceEnabled() && !(this instanceof HARegion)) {
+ logger.trace("getDeserializedValue for {} returning version: {} returnTombstones: {} value: {}",
+ keyInfo.getKey(), (re.getVersionStamp()==null? "null" : re.getVersionStamp().asVersionTag()), returnTombstones, value);
+ }
+ return value;
+ }
+ finally {
+ if (this.diskRegion != null) {
+ this.diskRegion.removeClearCountReference();
+ }
+ }
+ }
+
+ /**
+ *
+ * @param re
+ * @param updateStats
+ * @param disableCopyOnRead if true then do not make a copy on read
+ * @param preferCD true if the preferred result form is CachedDeserializable
+ * @param retainResult if true then the result may be a retained off-heap reference
+ * @return the value found, which can be
+ *
+ * null if the value was removed from the region entry
+ * Token.INVALID if the value of the region entry is invalid
+ * Token.LOCAL_INVALID if the value of the region entry is local invalid
+ *
+ */
+ @Retained
+ protected final Object getDeserialized(RegionEntry re, boolean updateStats, boolean disableCopyOnRead, boolean preferCD, boolean retainResult) {
+ assert !retainResult || preferCD;
+ try {
+ @Retained Object v = null;
+ try {
+ if (retainResult) {
+ v = re.getValueRetain(this);
+ } else {
+ v = re.getValue(this);
+ }
+ } catch(DiskAccessException dae) {
+ this.handleDiskAccessException(dae);
+ throw dae;
+ }
+
+ //skip updating the stats if the value is null
+ if (v == null) {
+ return null;
+ }
+ if (v instanceof CachedDeserializable) {
+ if (!preferCD) {
+ if (isCopyOnRead()) {
+ if (disableCopyOnRead) {
+ v = ((CachedDeserializable)v).getDeserializedForReading();
+ } else {
+ v = ((CachedDeserializable)v).getDeserializedWritableCopy(this, re);
+ }
+ } else {
+ v = ((CachedDeserializable)v).getDeserializedValue(this, re);
+ }
+ }
+ }
+ else if (!disableCopyOnRead) {
+ v = conditionalCopy(v);
+ }
+
+ if (updateStats) {
+ updateStatsForGet(re, v != null && !Token.isInvalid(v));
+ }
+ return v;
+ } catch(IllegalArgumentException i) {
+ IllegalArgumentException iae = new IllegalArgumentException(LocalizedStrings.DONT_RELEASE.toLocalizedString("Error while deserializing value for key="+re.getKey()));
+ iae.initCause(i);
+ throw iae;
+ }
+ }
+
+ @Override
+ public Object get(Object key, Object aCallbackArgument,
+ boolean generateCallbacks, EntryEventImpl clientEvent) throws TimeoutException, CacheLoaderException
+ {
+ Object result = get(key, aCallbackArgument, generateCallbacks, false, false, null, clientEvent, false, true/*allowReadFromHDFS*/);
+ if (Token.isInvalid(result)) {
+ result = null;
+ }
+ return result;
+ }
+
+ /*
+ * @see BucketRegion#getSerialized(KeyInfo, boolean, boolean)
+ */
+ public Object get(Object key, Object aCallbackArgument,
+ boolean generateCallbacks, boolean disableCopyOnRead, boolean preferCD,
+ ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones, boolean allowReadFromHDFS) throws TimeoutException, CacheLoaderException {
+ return get(key, aCallbackArgument,
+ generateCallbacks, disableCopyOnRead, preferCD,requestingClient, clientEvent, returnTombstones, false, allowReadFromHDFS, false);
+ }
+
+ /**
+ * The result of this operation may be an off-heap reference that the caller must release
+ */
+ @Retained
+ public Object getRetained(Object key, Object aCallbackArgument,
+ boolean generateCallbacks, boolean disableCopyOnRead,
+ ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones) throws TimeoutException, CacheLoaderException {
+ return getRetained(key, aCallbackArgument,
+ generateCallbacks, disableCopyOnRead, requestingClient, clientEvent, returnTombstones, false);
+ }
+
+ /**
+ * The result of this operation may be an off-heap reference that the caller must release.
+ * @param opScopeIsLocal if true then just check local storage for a value; if false then try to find the value if it is not local
+ */
+ @Retained
+ public Object getRetained(Object key, Object aCallbackArgument,
+ boolean generateCallbacks, boolean disableCopyOnRead,
+ ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones, boolean opScopeIsLocal) throws TimeoutException, CacheLoaderException {
+ return get(key, aCallbackArgument, generateCallbacks, disableCopyOnRead, true, requestingClient, clientEvent, returnTombstones, opScopeIsLocal, true, false);
+ }
+ /**
+ * @param opScopeIsLocal if true then just check local storage for a value; if false then try to find the value if it is not local
+ * @param retainResult if true then the result may be a retained off-heap reference.
+ */
+ public Object get(Object key, Object aCallbackArgument,
+ boolean generateCallbacks, boolean disableCopyOnRead, boolean preferCD,
+ ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones,
+ boolean opScopeIsLocal, boolean allowReadFromHDFS, boolean retainResult) throws TimeoutException, CacheLoaderException
+ {
+ assert !retainResult || preferCD;
+ validateKey(key);
+ validateCallbackArg(aCallbackArgument);
+ checkReadiness();
+ checkForNoAccess();
+ discoverJTA();
+ CachePerfStats stats = getCachePerfStats();
+ long start = stats.startGet();
+ boolean isMiss = true;
+ try {
+ KeyInfo keyInfo = getKeyInfo(key, aCallbackArgument);
+ Object value = getDataView().getDeserializedValue(keyInfo, this, true, disableCopyOnRead, preferCD, clientEvent, returnTombstones, allowReadFromHDFS, retainResult);
+ final boolean isCreate = value == null;
+ isMiss = value == null || Token.isInvalid(value)
+ || (!returnTombstones && value == Token.TOMBSTONE);
+ // Note: if the value was Token.DESTROYED then getDeserialized
+ // returns null so we don't need the following in the above expression:
+ // || (isRegInterestInProgress() && Token.isDestroyed(value))
+ // because (value == null) will be true in this case.
+ if (isMiss) {
+ // to fix bug 51509 raise the precedence of opScopeIsLocal
+ // if scope is local and there is no loader, then
+ // don't go further to try and get value
+ if (!opScopeIsLocal
+ && ((getScope().isDistributed() && !isHDFSRegion())
+ || hasServerProxy()
+ || basicGetLoader() != null)) {
+ // serialize search/load threads if not in txn
+ // TODO OFFHEAP OPTIMIZE: findObject can be enhanced to use the retainResult flag
+ value = getDataView().findObject(keyInfo,
+ this, isCreate, generateCallbacks, value, disableCopyOnRead,
+ preferCD, requestingClient, clientEvent, returnTombstones, false/*allowReadFromHDFS*/);
+ if (!returnTombstones && value == Token.TOMBSTONE) {
+ value = null;
+ }
+ }
+ else { // local scope with no loader, still might need to update stats
+ if (isCreate) {
+ recordMiss(null, key);
+ }
+ value = null;
+ }
+ }
+ return value;
+ }
+ finally {
+ stats.endGet(start, isMiss);
+ }
+ }
+
+ /**
+ * Update region and potentially entry stats for the miss case
+ * @param re optional region entry, fetched if null
+ * @param key the key used to fetch the region entry
+ */
+ final public void recordMiss(final RegionEntry re, Object key) {
+ final RegionEntry e;
+ if (re == null && !isTX() && !isHDFSRegion()) {
+ e = basicGetEntry(key);
+ } else {
+ e = re;
+ }
+ updateStatsForGet(e, false);
+ }
+
+ /**
+ * @return true if this region has been configured for HDFS persistence
+ */
+ public boolean isHDFSRegion() {
+ return false;
+ }
+
+ /**
+ * @return true if this region is configured to read and write data from HDFS
+ */
+ public boolean isHDFSReadWriteRegion() {
+ return false;
+ }
+
+ /**
+ * @return true if this region is configured to only write to HDFS
+ */
+ protected boolean isHDFSWriteOnly() {
+ return false;
+ }
+
+ /**
+ * FOR TESTING ONLY
+ */
+ public HoplogListenerForRegion getHoplogListener() {
+ return hoplogListener;
+ }
+
+ /**
+ * FOR TESTING ONLY
+ */
+ public HdfsRegionManager getHdfsRegionManager() {
+ return hdfsManager;
+ }
+
+ /**
+ * optimized to only allow one thread to do a search/load, other threads wait
+ * on a future
+ *
+ * @param keyInfo
+ * @param p_isCreate
+ * true if call found no entry; false if updating an existing
+ * entry
+ * @param generateCallbacks
+ * @param p_localValue
+ * the value retrieved from the region for this object.
+ * @param disableCopyOnRead if true then do not make a copy
+ * @param preferCD true if the preferred result form is CachedDeserializable
+ * @param clientEvent the client event, if any
+ * @param returnTombstones whether to return tombstones
+ */
+ @Retained
+ Object nonTxnFindObject(KeyInfo keyInfo, boolean p_isCreate,
+ boolean generateCallbacks, Object p_localValue, boolean disableCopyOnRead, boolean preferCD,
+ ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones, boolean allowReadFromHDFS)
+ throws TimeoutException, CacheLoaderException
+ {
+ final Object key = keyInfo.getKey();
+
+ Object localValue = p_localValue;
+ boolean isCreate = p_isCreate;
+ Object[] valueAndVersion = null;
+ @Retained Object result = null;
+ FutureResult thisFuture = new FutureResult(this.stopper);
+ Future otherFuture = (Future)this.getFutures.putIfAbsent(keyInfo.getKey(), thisFuture);
+ // only one thread can get their future into the map for this key at a time
+ if (otherFuture != null) {
+ try {
+ valueAndVersion = (Object[])otherFuture.get();
+ if (valueAndVersion != null) {
+ result = valueAndVersion[0];
+ if (clientEvent != null) {
+ clientEvent.setVersionTag((VersionTag)valueAndVersion[1]);
+ }
+ if (!preferCD && result instanceof CachedDeserializable) {
+ CachedDeserializable cd = (CachedDeserializable)result;
+ // fix for bug 43023
+ if (!disableCopyOnRead && isCopyOnRead()) {
+ result = cd.getDeserializedWritableCopy(null, null);
+ } else {
+ result = cd.getDeserializedForReading();
+ }
+
+ } else if (!disableCopyOnRead) {
+ result = conditionalCopy(result);
+ }
+ //For sqlf since the deserialized value is nothing but chunk
+ // before returning the found value increase its use count
+ if(GemFireCacheImpl.sqlfSystem() && result instanceof Chunk) {
+ if(!((Chunk)result).retain()) {
+ return null;
+ }
+ }
+ // what was a miss is now a hit
+ RegionEntry re = null;
+ if (isCreate) {
+ re = basicGetEntry(keyInfo.getKey());
+ updateStatsForGet(re, true);
+ }
+ return result;
+ }
+ // if value == null, try our own search/load
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ // TODO check a CancelCriterion here?
+ return null;
+ }
+ catch (ExecutionException e) {
+ // unexpected since there is no background thread
+ InternalGemFireError err = new InternalGemFireError(LocalizedStrings.LocalRegion_UNEXPECTED_EXCEPTION.toLocalizedString());
+ err.initCause(err);
+ throw err;
+ }
+ }
+ // didn't find a future, do one more probe for the entry to catch a race
+ // condition where the future was just removed by another thread
+ try {
+ boolean partitioned = this.getDataPolicy().withPartitioning();
+ if (!partitioned) {
+ localValue = getDeserializedValue(null, keyInfo, isCreate, disableCopyOnRead, preferCD, clientEvent, false, false/*allowReadFromHDFS*/, false);
+
+ // stats have now been updated
+ if (localValue != null && !Token.isInvalid(localValue)) {
+ result = localValue;
+ return result;
+ }
+ isCreate = localValue == null;
+ result = findObjectInSystem(keyInfo, isCreate, null, generateCallbacks,
+ localValue, disableCopyOnRead, preferCD, requestingClient, clientEvent, returnTombstones, false/*allowReadFromHDFS*/);
+
+ } else {
+
+ // This code was moved from PartitionedRegion.nonTxnFindObject(). That method has been removed.
+ // For PRs we don't want to deserialize the value and we can't use findObjectInSystem because
+ // it can invoke code that is transactional.
+ result = getSharedDataView().findObject(keyInfo, this, true/*isCreate*/, generateCallbacks,
+ localValue, disableCopyOnRead, preferCD, null, null, false, allowReadFromHDFS);
+ // TODO why are we not passing the client event or returnTombstones in the above invokation?
+ }
+
+ if (result == null && localValue != null) {
+ if (localValue != Token.TOMBSTONE || returnTombstones) {
+ result = localValue;
+ }
+ }
+ // findObjectInSystem does not call conditionalCopy
+ }
+ finally {
+ if (result != null) {
+ VersionTag tag = (clientEvent==null)? null : clientEvent.getVersionTag();
+ thisFuture.set(new Object[]{result, tag});
+ } else {
+ thisFuture.set(null);
+ }
+ this.getFutures.remove(keyInfo.getKey());
+ }
+ if (!disableCopyOnRead) {
+ result = conditionalCopy(result);
+ }
+ return result;
+ }
+
+ /**
+ * Returns true if get should give a copy; false if a reference.
+ *
+ * @since 4.0
+ */
+ protected boolean isCopyOnRead()
+ {
+ return this.compressor == null
+ && this.cache.isCopyOnRead()
+ && ! this.isUsedForPartitionedRegionAdmin
+ && ! this.isUsedForMetaRegion
+ && ! getOffHeap()
+ && ! isSecret();
+ }
+
+ /**
+ * Makes a copy, if copy-on-get is enabled, of the specified object.
+ *
+ * @since 4.0
+ */
+ protected Object conditionalCopy(Object o)
+ {
+ if (isCopyOnRead() && !Token.isInvalid(o)) {
+ return CopyHelper.copy(o);
+ }
+ else {
+ return o;
+ }
+ }
+
+ private final String fullPath;
+
+ public String getFullPath()
+ {
+ return this.fullPath;
+ }
+
+ // public String getFullPath() {
+ // // work way up to root region, prepending
+ // // the region names to a buffer
+ // StringBuffer buf = new StringBuffer(SEPARATOR);
+ // Assert.assertTrue(this.regionName != null);
+ // buf.append(this.regionName);
+ // LocalRegion r = this;
+ // while ((r = r.parentRegion) != null) {
+ // buf.insert(0, r.regionName);
+ // buf.insert(0, SEPARATOR_CHAR);
+ // }
+ // return buf.toString();
+ // }
+
+ public Region getParentRegion()
+ {
+ //checkReadiness();
+ return this.parentRegion;
+ }
+
+ public Region getSubregion(String path)
+ {
+ checkReadiness();
+ return getSubregion(path, false);
+ }
+
+ public void invalidateRegion(Object aCallbackArgument)
+ throws TimeoutException
+ {
+ getDataView().checkSupportsRegionInvalidate();
+ validateCallbackArg(aCallbackArgument);
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ RegionEventImpl event = new RegionEventImpl(this, Operation.REGION_INVALIDATE,
+ aCallbackArgument, false, getMyId(), generateEventID());
+
+ basicInvalidateRegion(event);
+ }
+
+ public Object put(Object key, Object value, Object aCallbackArgument)
+ throws TimeoutException, CacheWriterException {
+ long startPut = CachePerfStats.getStatTime();
+ EntryEventImpl event = newUpdateEntryEvent(key, value, aCallbackArgument);
+ //Since Sqlfire directly calls validatedPut, the freeing is done in
+ // validatedPut
+ return validatedPut(event, startPut);
+ // TODO OFFHEAP: validatedPut calls freeOffHeapResources
+
+ }
+
+ public final Object validatedPut(EntryEventImpl event, long startPut)
+ throws TimeoutException, CacheWriterException {
+
+ try {
+ if (event.getEventId() == null && generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ Object oldValue = null;
+ // Sqlf changes begin
+ // see #40294.
+
+ // Rahul: this has to be an update.
+ // so executing it as an update.
+ boolean forceUpdateForDelta = event.hasDelta();
+ // Sqlf Changes end.
+ if (basicPut(event, false, // ifNew
+ forceUpdateForDelta, // ifOld
+ null, // expectedOldValue
+ false // requireOldValue
+ )) {
+ if (!event.isOldValueOffHeap()) {
+ // don't copy it to heap just to return from put.
+ // TODO: come up with a better way to do this.
+ oldValue = event.getOldValue();
+ }
+ if (!getDataView().isDeferredStats()) {
+ getCachePerfStats().endPut(startPut, false);
+ }
+ }
+ return handleNotAvailable(oldValue);
+ } finally {
+ event.release();
+ }
+ }
+
+ // split into a separate newUpdateEntryEvent since SQLFabric may need to
+ // manipulate event before doing the put (e.g. posDup flag)
+ public final EntryEventImpl newUpdateEntryEvent(Object key, Object value,
+ Object aCallbackArgument) {
+
+ validateArguments(key, value, aCallbackArgument);
+ if (value == null) {
+ throw new NullPointerException(LocalizedStrings
+ .LocalRegion_VALUE_MUST_NOT_BE_NULL.toLocalizedString());
+ }
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ discoverJTA();
+
+ // This used to call the constructor which took the old value. It
+ // was modified to call the other EntryEventImpl constructor so that
+ // an id will be generated by default. Null was passed in anyway.
+ // generate EventID
+ final EntryEventImpl event = EntryEventImpl.create(
+ this, Operation.UPDATE, key,
+ value, aCallbackArgument, false, getMyId());
+ boolean eventReturned = false;
+ try {
+ extractDeltaIntoEvent(value, event);
+ eventReturned = true;
+ return event;
+ } finally {
+ if (!eventReturned) event.release();
+ }
+ }
+ /**
+ * Creates an EntryEventImpl that is optimized to not fetch data from HDFS.
+ * This is meant to be used by PUT dml from GemFireXD.
+ */
+ public final EntryEventImpl newPutEntryEvent(Object key, Object value,
+ Object aCallbackArgument) {
+ EntryEventImpl ev = newUpdateEntryEvent(key, value, aCallbackArgument);
+ ev.setFetchFromHDFS(false);
+ ev.setPutDML(true);
+ return ev;
+ }
+ private void extractDeltaIntoEvent(Object value, EntryEventImpl event) {
+ // 1. Check for DS-level delta property.
+ // 2. Default value for operation type is UPDATE, so no need to check that here.
+ // 3. Check if it has server region proxy.
+ // We do not have a handle to event in PutOpImpl to check if we have
+ // delta bytes calculated already. So no need to calculate it here.
+ // 4. Check if value is instanceof com.gemstone.gemfire.Delta
+ // 5. Check if Region in PR with redundantCopies > 0. Set extractDelta.
+ // 6. Check if Region has peers. Set extractDelta.
+ // 7. Check if it has any delta proxies attached to it. Set extractDelta.
+ // 8. If extractDelta is set, check if it has delta.
+ // 9. If delta is found, extract it and set it into the event.
+ // 10. If any exception is caught while invoking the delta callbacks, throw it back.
+ // 11. Wrap any checked exception in InternalGemFireException before throwing it.
+ try {
+ boolean extractDelta = false;
+ // How costly is this if check?
+ if (this.getSystem().getConfig().getDeltaPropagation()
+ && value instanceof com.gemstone.gemfire.Delta) {
+ if (!this.hasServerProxy()) {
+ if ((this instanceof PartitionedRegion)) {
+ if (((PartitionedRegion)this).getRedundantCopies() > 0) {
+ extractDelta = true;
+ } else {
+ InternalDistributedMember ids = (InternalDistributedMember)PartitionRegionHelper
+ .getPrimaryMemberForKey(this, event.getKey());
+ if (ids != null) {
+ if (this.getSystem().getMemberId().equals(ids.getId())) {
+ extractDelta = hasAdjunctRecipientsNeedingDelta(event);
+ } else {
+ extractDelta = true;
+ }
+ } else {
+ extractDelta = true;
+ }
+ }
+ } else if ((this instanceof DistributedRegion)
+ && !((DistributedRegion)this).scope.isDistributedNoAck()
+ && ((DistributedRegion)this).getCacheDistributionAdvisor()
+ .adviseCacheOp().size() > 0) {
+ extractDelta = true;
+ }
+ if (!extractDelta && ClientHealthMonitor.getInstance() != null) {
+ extractDelta = ClientHealthMonitor.getInstance().hasDeltaClients();
+ }
+ } else if (HandShake.isDeltaEnabledOnServer()) {
+ // This is a client region
+ extractDelta = true;
+ }
+ if (extractDelta && ((com.gemstone.gemfire.Delta)value).hasDelta()) {
+ HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
+ long start = DistributionStats.getStatTime();
+ try {
+ ((com.gemstone.gemfire.Delta)value).toDelta(hdos);
+ } catch (RuntimeException re) {
+ throw re;
+ } catch (Exception e) {
+ throw new DeltaSerializationException(
+ LocalizedStrings.DistributionManager_CAUGHT_EXCEPTION_WHILE_SENDING_DELTA
+ .toLocalizedString(), e);
+ }
+ event.setDeltaBytes(hdos.toByteArray());
+ this.getCachePerfStats().endDeltaPrepared(start);
+ }
+ }
+ } catch (RuntimeException re) {
+ throw re;
+ } catch (Exception e) {
+ throw new InternalGemFireException(e);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private boolean hasAdjunctRecipientsNeedingDelta(EntryEventImpl event) {
+ PartitionedRegion pr = ((PartitionedRegion)this);
+ BucketRegion br = null;
+ FilterRoutingInfo filterRouting = null;
+ Set twoMessages = Collections.EMPTY_SET;
+ Set adjunctRecipients = Collections.EMPTY_SET;
+ Set cacheservers = null;
+
+ int bId = event.getKeyInfo().getBucketId();
+ try {
+ br = pr.dataStore.getInitializedBucketForId(event.getKey(), bId);
+ } catch (ForceReattemptException fre) {
+ return true;
+ }
+ Set recipients = br.getCacheDistributionAdvisor().adviseUpdate(event);
+ twoMessages = br.getBucketAdvisor().adviseRequiresTwoMessages();
+ CacheDistributionAdvisor cda = pr.getCacheDistributionAdvisor();
+ filterRouting = cda.adviseFilterRouting(event, recipients);
+ adjunctRecipients = br.getAdjunctReceivers(event, recipients, twoMessages, filterRouting);
+ cacheservers = cda.adviseCacheServers();
+ return !Collections.disjoint(adjunctRecipients, cacheservers);
+ }
+
+ public Region.Entry getEntry(Object key)
+ {
+ validateKey(key);
+ checkReadiness();
+ checkForNoAccess();
+ discoverJTA();
+ return getDataView().getEntry(getKeyInfo(key), this, false);
+ }
+
+ /** internally we often need to get an entry whether it is a tombstone or not */
+ public Region.Entry getEntry(Object key, boolean allowTombstones) {
+ return getDataView().getEntry(getKeyInfo(key), this, allowTombstones);
+ }
+
+ /**
+ * Just like getEntry but also updates the stats that get would have depending
+ * on a flag. See bug 42410. Also skips discovering JTA
+ *
+ * @param key
+ * @return the entry if it exists; otherwise null.
+ */
+ public Entry accessEntry(Object key, boolean updateStats) {
+ validateKey(key);
+ checkReadiness();
+ checkForNoAccess();
+ if(updateStats) {
+ return getDataView().accessEntry(getKeyInfo(key), this);
+ } else {
+ return getDataView().getEntry(getKeyInfo(key), this, false);
+ }
+ }
+
+ protected boolean includeHDFSResults() {
+ return isUsedForPartitionedRegionBucket()
+ && isHDFSReadWriteRegion()
+ && getPartitionedRegion().includeHDFSResults();
+ }
+
+
+ /** a fast estimate of total number of entries locally in the region */
+ public long getEstimatedLocalSize() {
+ RegionMap rm;
+ if (!this.isDestroyed) {
+ long size;
+ if (isHDFSReadWriteRegion() && this.initialized) {
+ // this size is not used by HDFS region iterators
+ // fixes bug 49239
+ return 0;
+ }
+ // if region has not been initialized yet, then get the estimate from
+ // disk region's recovery map if available
+ if (!this.initialized && this.diskRegion != null
+ && (rm = this.diskRegion.getRecoveredEntryMap()) != null
+ && (size = rm.size()) > 0) {
+ return size;
+ }
+ if ((rm = getRegionMap()) != null) {
+ return rm.size();
+ }
+ }
+ return 0;
+ }
+ /**
+ * @param keyInfo
+ * @param access
+ * true if caller wants last accessed time updated
+ * @param allowTombstones whether an entry with a TOMBSTONE value can be returned
+ * @return TODO
+ */
+ protected Region.Entry nonTXGetEntry(KeyInfo keyInfo, boolean access, boolean allowTombstones) {
+ final Object key = keyInfo.getKey();
+ RegionEntry re = this.entries.getEntry(key);
+ boolean miss = (re == null || re.isDestroyedOrRemoved());
+ if (access) {
+ updateStatsForGet(re, !miss);
+ }
+ if (re == null) {
+ return null;
+ }
+ if (re.isTombstone()) {
+ if (!allowTombstones) {
+ return null;
+ } // else return an entry (client GII / putAll results)
+ } else if (miss) {
+ return null;
+ }
+
+ Region.Entry ren = new NonTXEntry(re);
+ //long start=0, end=0;
+ //start = System.currentTimeMillis();
+ //end = System.currentTimeMillis();
+ //System.out.println("getEntry: " + (end-start));
+ return ren;
+ }
+
+ /**
+ * @return boolean
+ */
+ protected boolean isClosed()
+ {
+ return this.cache.isClosed();
+ }
+
+ /**
+ * Returns true if this region is or has been closed or destroyed.
+ * Note that unlike {@link #isDestroyed()} this method will not
+ * return true if the cache is closing but has not yet started closing
+ * this region.
+ */
+ public boolean isThisRegionBeingClosedOrDestroyed() {
+ return this.isDestroyed;
+ }
+
+ /** returns true if this region has been destroyed */
+ public boolean isDestroyed()
+ {
+ if (isClosed()) {
+ return true; // for bug 42328
+ }
+ boolean isTraceEnabled = logger.isTraceEnabled();
+ // boolean result = false;
+ if (this.isDestroyed) {
+ if (isTraceEnabled) {
+ logger.trace("isDestroyed: true, this.isDestroyed: {}", getFullPath());
+ }
+ return true;
+ }
+ // if (!isInitialized()) { // don't return true if still initializing
+ // if (finestEnabled) {
+ // log.finest("isDestroyed: false, not initialized: " + getFullPath());
+ // }
+ // return false;
+ // }
+ // @todo we could check parents here if we want this to be more accurate,
+ // and the isDestroyed field could be made volatile as well.
+ // if (this.parentRegion != null) return this.parentRegion.isDestroyed();
+ if (isTraceEnabled) {
+ logger.trace("isDestroyed: false : {}", getFullPath());
+ }
+ return false;
+ }
+
+ /** a variant of subregions() that does not perform a readiness check */
+ protected Set basicSubregions(boolean recursive) {
+ return new SubregionsSet(recursive);
+ }
+
+ public Set subregions(boolean recursive) {
+ checkReadiness();
+ return new SubregionsSet(recursive);
+ }
+
+ public Set entries(boolean recursive) {
+ checkReadiness();
+ checkForNoAccess();
+ return basicEntries(recursive);
+ }
+
+ /** Returns set of entries without performing validation checks. */
+ public Set basicEntries(boolean recursive) {
+ return new EntriesSet(this, recursive, IteratorType.ENTRIES, false);
+ }
+
+ /**
+ * Flavor of keys that will not do repeatable read
+ * @since 5.5
+ */
+ public Set testHookKeys()
+ {
+ checkReadiness();
+ checkForNoAccess();
+ return new EntriesSet(this, false, IteratorType.KEYS,
+ false /* dontRememberReads */, false /* skipTxCheckInIteration */,
+ false /* allowTombstones */);
+ }
+
+ public Set keys()
+ {
+ checkReadiness();
+ checkForNoAccess();
+ return new EntriesSet(this, false, IteratorType.KEYS, false);
+ }
+
+ /**
+ * return a set of the keys in this region
+ * @param allowTombstones whether destroyed entries should be included
+ * @return the keys
+ */
+ public Set keySet(boolean allowTombstones) {
+ checkReadiness();
+ checkForNoAccess();
+ return new EntriesSet(this, false, IteratorType.KEYS, allowTombstones);
+ }
+
+ public Collection values()
+ {
+ checkReadiness();
+ checkForNoAccess();
+ return new EntriesSet(this, false, IteratorType.VALUES, false);
+ }
+
+ public Object getUserAttribute()
+ {
+ return this.regionUserAttribute;
+ }
+
+ public void setUserAttribute(Object value)
+ {
+ checkReadiness();
+ this.regionUserAttribute = value;
+ }
+
+ public boolean containsKey(Object key)
+ {
+ checkReadiness();
+ checkForNoAccess();
+ return getDataView().containsKey(getKeyInfo(key), this);
+ }
+
+ public boolean containsTombstone(Object key)
+ {
+ checkReadiness();
+ checkForNoAccess();
+ if (!this.concurrencyChecksEnabled) {
+ return false;
+ } else {
+ try {
+ Entry entry = getDataView().getEntry(getKeyInfo(key), this, true);
+ if (entry == null) {
+ return false;
+ } else {
+ return (entry.getValue() == Token.TOMBSTONE);
+ }
+ } catch (EntryDestroyedException e) {
+ return true;
+ }
+ }
+ }
+
+ protected boolean nonTXContainsKey(KeyInfo keyInfo) {
+ boolean contains = getRegionMap().containsKey(keyInfo.getKey());
+ if (contains && this.imageState.isClient()) {
+ // fix for bug #40871 - concurrent RI causes containsKey for destroyed entry
+ // to return true
+ RegionEntry re = this.entries.getEntry(keyInfo.getKey());
+ // TODO:KIRK:OK if (re == null || Token.isRemoved(re.getValueInVM(this))) {
+ if (re == null || re.isDestroyedOrRemoved()) {
+ contains = false;
+ }
+ }
+ return contains;
+ }
+
+ public boolean containsValueForKey(Object key)
+ {
+ discoverJTA();
+ return getDataView().containsValueForKey(getKeyInfo(key), this);
+ }
+
+ /**
+ * @param keyInfo
+ * @return TODO
+ */
+ protected boolean nonTXContainsValueForKey(KeyInfo keyInfo) {
+ checkReadiness();
+ checkForNoAccess();
+ if (this.diskRegion != null) {
+ this.diskRegion.setClearCountReference();
+ }
+ try {
+ RegionEntry entry = this.entries.getEntry(keyInfo.getKey());
+ boolean result = entry != null;
+ if (result) {
+ ReferenceCountHelper.skipRefCountTracking();
+ Object val = entry.getTransformedValue(); // no need to decompress since we only want to know if we have an existing value
+ if (val instanceof StoredObject) {
+ OffHeapHelper.release(val);
+ ReferenceCountHelper.unskipRefCountTracking();
+ return true;
+ }
+ ReferenceCountHelper.unskipRefCountTracking();
+ // No need to to check CachedDeserializable because of Bruce's fix in r30960 for bug 42162. See bug 42732.
+ // this works because INVALID and LOCAL_INVALID will never be faulted out of mem
+ // If val is NOT_AVAILABLE that means we have a valid value on disk.
+ result = !Token.isInvalidOrRemoved(val);
+ }
+ return result;
+ }
+ finally {
+ if (this.diskRegion != null) {
+ this.diskRegion.removeClearCountReference();
+ }
+ }
+ }
+
+ public RegionAttributes getAttributes()
+ {
+ // to fix bug 35134 allow attribute access on closed regions
+ //checkReadiness();
+ return this;
+ }
+
+ public String getName()
+ {
+ return this.regionName;
+ }
+
+ /**
+ * Convenience method to get region name for logging/exception messages.
+ * if this region is an instanceof bucket region, it returns the
+ * bucket region name
+ * @return name of the region or the owning partitioned region
+ */
+ public String getDisplayName() {
+ if (this.isUsedForPartitionedRegionBucket()) {
+ return this.getPartitionedRegion().getName();
+ }
+ return this.regionName;
+ }
+
+ /**
+ * Returns the number of entries in this region. Note that because of the
+ * concurrency properties of the {@link RegionMap}, the number of entries is
+ * only an approximate. That is, other threads may change the number of
+ * entries in this region while this method is being invoked.
+ *
+ * @see RegionMap#size
+ *
+ * author David Whitlock
+ */
+ public final int entryCount() {
+ return getDataView().entryCount(this);
+ }
+
+ public int entryCount(Set buckets) {
+ return entryCount(buckets, false);
+ }
+
+ protected int entryCount( Set buckets, boolean estimate) {
+ assert buckets == null: "unexpected buckets " + buckets + " for region "
+ + toString();
+
+ return getDataView().entryCount(this);
+ }
+
+ public int entryCountEstimate(final TXStateInterface tx, Set buckets, boolean entryCountEstimate) {
+ return entryCount(buckets, entryCountEstimate);
+ }
+
+ /**
+ * @return size after considering imageState
+ */
+ protected int getRegionSize() {
+ synchronized(getSizeGuard()) {
+ int result = getRegionMap().size();
+ // if this is a client with no tombstones then we subtract the number
+ // of entries being affected by register-interest refresh
+ if (this.imageState.isClient() && !this.concurrencyChecksEnabled) {
+ return result - this.imageState.getDestroyedEntriesCount();
+ }
+ if (includeHDFSResults()) {
+ return result;
+ }
+ return result - this.tombstoneCount.get();
+ }
+ }
+
+ /**
+ * Returns the DiskRegion
that this region uses to access data
+ * on disk.
+ *
+ * @return null
if disk regions are not being used
+ *
+ * @since 3.2
+ */
+ public DiskRegion getDiskRegion()
+ {
+ return this.diskRegion;
+ }
+ public DiskRegionView getDiskRegionView() {
+ return getDiskRegion();
+ }
+
+ /**
+ * Lets the customer do an explicit evict of a value to disk and removes the value
+ * from memory.
+ */
+ public void evictValue(Object key) {
+ if (getDiskRegion() != null) {
+ this.entries.evictValue(key);
+ }
+ }
+
+ /**
+ *
+ * Initially called by EvictorThread.run
+ *
+ * @since 3.5.1
+ */
+ public void checkLRU()
+ {
+ if (this.entriesInitialized) {
+ try {
+ this.entries.lruUpdateCallback();
+ }catch( DiskAccessException dae) {
+ this.handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+ }
+
+ protected boolean isOverflowEnabled() {
+ EvictionAttributes ea = getAttributes().getEvictionAttributes();
+ return ea != null && ea.getAction().isOverflowToDisk();
+ }
+
+ public void writeToDisk()
+ {
+ if (this.diskRegion == null) {
+ DataPolicy dp = getDataPolicy();
+ if (dp.isEmpty()) {
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_CANNOT_WRITE_A_REGION_WITH_DATAPOLICY_0_TO_DISK.toLocalizedString(dp));
+ }
+ else if (!dp.withPersistence() && !isOverflowEnabled()) {
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_CANNOT_WRITE_A_REGION_THAT_IS_NOT_CONFIGURED_TO_ACCESS_DISKS.toLocalizedString());
+ }
+ }
+ else {
+ this.diskRegion.asynchForceFlush();
+ }
+ }
+
+ /**
+ * Used by tests to force everything out to disk.
+ */
+ public void forceFlush()
+ {
+ if (this.diskRegion != null) {
+ this.diskRegion.flushForTesting();
+ }
+ }
+
+ /**
+ * This implementation only checks readiness and scope
+ */
+ public Lock getRegionDistributedLock() throws IllegalStateException
+ {
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ Scope theScope = getAttributes().getScope();
+ Assert.assertTrue(theScope == Scope.LOCAL);
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_ONLY_SUPPORTED_FOR_GLOBAL_SCOPE_NOT_LOCAL.toLocalizedString());
+ }
+
+ /**
+ * This implementation only checks readiness and scope
+ */
+ public Lock getDistributedLock(Object key) throws IllegalStateException
+ {
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ Scope theScope = getAttributes().getScope();
+ Assert.assertTrue(theScope == Scope.LOCAL);
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_ONLY_SUPPORTED_FOR_GLOBAL_SCOPE_NOT_LOCAL.toLocalizedString());
+ }
+
+ public void invalidate(Object key, Object aCallbackArgument)
+ throws TimeoutException, EntryNotFoundException
+ {
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ validatedInvalidate(key, aCallbackArgument);
+ }
+
+ /**
+ * Destroys entry without performing validations. Call this after validating
+ * key, callback arg, and runtime state.
+ */
+ protected void validatedInvalidate(Object key, Object aCallbackArgument)
+ throws TimeoutException, EntryNotFoundException
+ {
+ EntryEventImpl event = EntryEventImpl.create(
+ this, Operation.INVALIDATE,
+ key, null, aCallbackArgument, false, getMyId());
+ try {
+ if (generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ basicInvalidate(event);
+ } finally {
+ event.release();
+ }
+ }
+
+ public void localDestroy(Object key, Object aCallbackArgument)
+ throws EntryNotFoundException
+ {
+ validateKey(key);
+ checkReadiness();
+ checkForNoAccess();
+ EntryEventImpl event = EntryEventImpl.create(
+ this,
+ Operation.LOCAL_DESTROY, key, null, aCallbackArgument, false, getMyId());
+ if (generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ try {
+ basicDestroy(event,
+ false,
+ null); // expectedOldValue
+ }
+ catch (CacheWriterException e) {
+ // cache writer not called
+ throw new Error(LocalizedStrings.LocalRegion_CACHE_WRITER_SHOULD_NOT_HAVE_BEEN_CALLED_FOR_LOCALDESTROY.toLocalizedString(), e);
+ }
+ catch (TimeoutException e) {
+ // no distributed lock
+ throw new Error(LocalizedStrings.LocalRegion_NO_DISTRIBUTED_LOCK_SHOULD_HAVE_BEEN_ATTEMPTED_FOR_LOCALDESTROY.toLocalizedString(), e);
+ } finally {
+ event.release();
+ }
+ }
+
+ public void localDestroyRegion(Object aCallbackArgument)
+ {
+ getDataView().checkSupportsRegionDestroy();
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.REGION_LOCAL_DESTROY, aCallbackArgument, false, getMyId(),
+ generateEventID()/* generate EventID */);
+ try {
+ basicDestroyRegion(event, false);
+ }
+ catch (CacheWriterException e) {
+ // not possible with local operation, CacheWriter not called
+ throw new Error(LocalizedStrings.LocalRegion_CACHEWRITEREXCEPTION_SHOULD_NOT_BE_THROWN_IN_LOCALDESTROYREGION.toLocalizedString(), e);
+ }
+ catch (TimeoutException e) {
+ // not possible with local operation, no distributed locks possible
+ throw new Error(LocalizedStrings.LocalRegion_TIMEOUTEXCEPTION_SHOULD_NOT_BE_THROWN_IN_LOCALDESTROYREGION.toLocalizedString(), e);
+ }
+ }
+
+ public void close()
+ {
+
+ RegionEventImpl event = new RegionEventImpl(this, Operation.REGION_CLOSE,
+ null, false, getMyId(), generateEventID()/* generate EventID */);
+ try {
+ // NOTE: the 422dynamicRegions branch added the callbackEvents argument
+ // to basicDestroyRegion and inhibited events on region.close. This
+ // clashed with the new SystemMemberCacheListener functionality in
+ // 5.0, causing unit tests to fail
+ basicDestroyRegion(event, false, true, true);
+ }
+ catch (CacheWriterException e) {
+ // not possible with local operation, CacheWriter not called
+ throw new Error(LocalizedStrings.LocalRegion_CACHEWRITEREXCEPTION_SHOULD_NOT_BE_THROWN_IN_LOCALDESTROYREGION.toLocalizedString(), e);
+ }
+ catch (TimeoutException e) {
+ // not possible with local operation, no distributed locks possible
+ throw new Error(LocalizedStrings.LocalRegion_TIMEOUTEXCEPTION_SHOULD_NOT_BE_THROWN_IN_LOCALDESTROYREGION.toLocalizedString(), e);
+ }
+ }
+
+ public void localInvalidate(Object key, Object callbackArgument)
+ throws EntryNotFoundException
+ {
+ validateKey(key);
+ checkReadiness();
+ checkForNoAccess();
+
+ EntryEventImpl event = EntryEventImpl.create(
+ this,
+ Operation.LOCAL_INVALIDATE, key, null/* newValue */, callbackArgument,
+ false, getMyId());
+ try {
+ if (generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ event.setLocalInvalid(true);
+ basicInvalidate(event);
+ } finally {
+ event.release();
+ }
+ }
+
+ public void localInvalidateRegion(Object aCallbackArgument)
+ {
+ getDataView().checkSupportsRegionInvalidate();
+ checkReadiness();
+ checkForNoAccess();
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.REGION_LOCAL_INVALIDATE, aCallbackArgument, false, getMyId());
+ basicInvalidateRegion(event);
+ }
+
+ /**
+ * Look up the LocalRegion with the specified full path.
+ *
+ * @param system
+ * the distributed system whose cache contains the root of interest
+ * @return the LocalRegion or null if not found
+ */
+ public static LocalRegion getRegionFromPath(DistributedSystem system,
+ String path)
+ {
+ Cache c = GemFireCacheImpl.getInstance();
+ if(c==null) {
+ return null;
+ } else {
+ return (LocalRegion)c.getRegion(path);
+ }
+ }
+
+ // public void dumpEntryMapStats(PrintStream out) {
+ // ((ConcurrentHashMap)this.entries).dumpStats(out);
+ // }
+
+ ////////////////// Protected Methods ////////////////////////////////////////
+
+ /**
+ * Do any extra initialization required. Region is already visible in parent's
+ * subregion map. This method releases the initialization Latches, so
+ * subclasses should call this super method last after performing additional
+ * initialization.
+ *
+ * @param imageTarget
+ * ignored, used by subclass for get initial image
+ * @param internalRegionArgs
+ * @see DistributedRegion#initialize(InputStream, InternalDistributedMember, InternalRegionArguments)
+ */
+ protected void initialize(InputStream snapshotInputStream,
+ InternalDistributedMember imageTarget, InternalRegionArguments internalRegionArgs) throws TimeoutException,
+ IOException, ClassNotFoundException
+ {
+ if (!isInternalRegion()) {
+ // Subclasses may have already called this method, but this is
+ // acceptable because addResourceListener won't add it twice
+ if (!this.isDestroyed) {
+ cache.getResourceManager().addResourceListener(ResourceType.MEMORY, this);
+ }
+ }
+
+ // if we're versioning entries we need a region-level version vector
+ if (this.concurrencyChecksEnabled && this.versionVector == null) {
+ createVersionVector();
+ }
+ // if not local, then recovery happens in InitialImageOperation
+ if (this.scope.isLocal()) {
+ createOQLIndexes(internalRegionArgs);
+ if (this.diskRegion != null) {
+ try {
+ this.diskRegion.initializeOwner(this);
+ this.diskRegion.finishInitializeOwner(this, GIIStatus.NO_GII);
+ { // This block was added so that early recovery could figure out that
+ // this data needs to be recovered from disk. Local regions used to
+ // not bother assigning a memberId but that is what the early
+ // recovery
+ // code uses to figure out that a region needs to be recovered.
+ PersistentMemberID oldId = this.diskRegion.getMyInitializingID();
+ if (oldId == null) {
+ oldId = this.diskRegion.getMyPersistentID();
+ }
+ if (oldId == null) {
+ PersistentMemberID newId = this.diskRegion.generatePersistentID();
+ this.diskRegion.setInitializing(newId);
+ this.diskRegion.setInitialized();
+ }
+ }
+ } catch (DiskAccessException dae) {
+ releaseAfterRegionCreateEventLatch();
+ this.handleDiskAccessException(dae, true);
+ throw dae;
+ }
+ }
+ }
+
+ // make sure latches are released if they haven't been by now already
+ releaseBeforeGetInitialImageLatch();
+ if (snapshotInputStream != null && this.scope.isLocal()) {
+ try {
+ loadSnapshotDuringInitialization(snapshotInputStream);
+ }catch(DiskAccessException dae) {
+ releaseAfterRegionCreateEventLatch();
+ this.handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+ releaseAfterGetInitialImageLatch();
+ if (logger.isDebugEnabled()) {
+ logger.debug("Calling addExpiryTasks for {}", this);
+ }
+ // these calls can throw RegionDestroyedException if this region is
+ // destroyed
+ // at this point
+ try {
+ addIdleExpiryTask();
+ addTTLExpiryTask();
+ if (isEntryExpiryPossible()) {
+ rescheduleEntryExpiryTasks(); // called after gii to fix bug 35214
+ }
+ initialized();
+ }
+ catch (RegionDestroyedException e) {
+ // whether it is this region or a parent region that is destroyed,
+ // then so must we be
+ Assert.assertTrue(isDestroyed());
+ // just proceed, a destroyed region will be returned to caller
+ }
+ }
+
+ protected void createOQLIndexes(InternalRegionArguments internalRegionArgs) {
+ createOQLIndexes(internalRegionArgs, false);
+ }
+
+ protected void createOQLIndexes(InternalRegionArguments internalRegionArgs, boolean recoverFromDisk) {
+
+ if (internalRegionArgs == null || internalRegionArgs.getIndexes() == null || internalRegionArgs.getIndexes().size() == 0){
+ return;
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("LocalRegion.createOQLIndexes on region {}", this.getFullPath());
+ }
+ long start = getCachePerfStats().startIndexInitialization();
+ List oqlIndexes = internalRegionArgs.getIndexes();
+
+ if (this.indexManager == null) {
+ this.indexManager = IndexUtils.getIndexManager(this, true);
+ }
+ Set indexes = new HashSet();
+ Set prIndexes = new HashSet();
+ int initLevel = 0;
+ DiskRegion dr = this.getDiskRegion();
+ boolean isOverflowToDisk = false;
+ if(dr != null) {
+ isOverflowToDisk = dr.isOverflowEnabled();
+ if(recoverFromDisk && !isOverflowToDisk) {
+ // Refer bug #44119
+ // For disk regions, index creation should wait for async value creation to complete before it starts its iteration
+ // In case of disk overflow regions the waitForAsyncRecovery is done in populateOQLIndexes method via getBestIterator()
+ dr.waitForAsyncRecovery();
+ }
+ }
+ try {
+ // Release the initialization latch for index creation.
+ initLevel = LocalRegion.setThreadInitLevelRequirement(ANY_INIT);
+ for (Object o : oqlIndexes) {
+ IndexCreationData icd = (IndexCreationData)o;
+ try {
+ if (icd.getPartitionedIndex() != null) {
+ ExecutionContext externalContext = new ExecutionContext(null, this.cache);
+ if (internalRegionArgs.getPartitionedRegion() != null) {
+ externalContext.setBucketRegion(internalRegionArgs.getPartitionedRegion(), (BucketRegion)this);
+ }
+ if (logger.isDebugEnabled()) {
+ logger.debug("IndexManager Index creation process for {}", icd.getIndexName());
+ }
+
+ //load entries during initialization only for non overflow regions
+ indexes.add(this.indexManager.createIndex(icd.getIndexName(), icd.getIndexType(),
+ icd.getIndexExpression(), icd.getIndexFromClause(),
+ icd.getIndexImportString(), externalContext, icd.getPartitionedIndex(), !isOverflowToDisk));
+ prIndexes.add(icd.getPartitionedIndex());
+ } else {
+ if (logger.isDebugEnabled()) {
+ logger.debug("QueryService Index creation process for {}" + icd.getIndexName());
+ }
+ DefaultQueryService qs = (DefaultQueryService) this.getGemFireCache().getLocalQueryService();
+ String fromClause = (icd.getIndexType() == IndexType.FUNCTIONAL || icd.getIndexType() == IndexType.HASH)? icd.getIndexFromClause() : this.getFullPath();
+ //load entries during initialization only for non overflow regions
+ indexes.add(qs.createIndex(icd.getIndexName(), icd.getIndexType(), icd.getIndexExpression(), fromClause, icd.getIndexImportString(), !isOverflowToDisk));
+ }
+
+ } catch (Exception ex) {
+ logger.error("Failed to create index {} on region {} with exception: {}", icd.getIndexName(), this.getFullPath(), ex);
+
+ // Check if the region index creation is from cache.xml, in that case throw exception.
+ // Other case is when bucket regions are created dynamically, in that case ignore the exception.
+ if (internalRegionArgs.getDeclarativeIndexCreation()) {
+ InternalGemFireError err = new InternalGemFireError(LocalizedStrings.GemFireCache_INDEX_CREATION_EXCEPTION_1.toLocalizedString(new Object[] {icd.getIndexName(), this.getFullPath()}) );
+ err.initCause(ex);
+ throw err;
+ }
+ }
+ }
+ } finally {
+ // Reset the initialization lock.
+ LocalRegion.setThreadInitLevelRequirement(initLevel);
+ }
+ // Load data into OQL indexes in case of disk recovery and disk overflow
+ if(isOverflowToDisk){
+ if(recoverFromDisk) {
+ populateOQLIndexes(indexes);
+ } else {
+ // Empty indexes are created for overflow regions but not populated at this stage
+ // since this is not recovery.
+ // Setting the populate flag to true so that the indexes can apply updates.
+ this.indexManager.setPopulateFlagForIndexes(indexes);
+ }
+ //due to bug #52096, the pr index populate flags were not being set
+ //we should revisit and clean up the index creation code paths
+ this.indexManager.setPopulateFlagForIndexes(prIndexes);
+ }
+ getCachePerfStats().endIndexInitialization(start);
+ }
+
+ /**
+ * Populate the indexes with region entries
+ */
+ protected void populateOQLIndexes(Set indexes) {
+ logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_INDEX_LOADING));
+ try {
+ this.indexManager.populateIndexes(indexes);
+ } catch (MultiIndexCreationException ex) {
+ logger.info("Failed to update index on region {}: {}", this.getFullPath(), ex.getMessage());
+ }
+ }
+
+ /**
+ * The region is now fully initialized, as far as LocalRegion is concerned
+ */
+ protected void initialized() {
+ // does nothing in LocalRegion at this time
+ }
+
+ protected void releaseLatches()
+ {
+ releaseBeforeGetInitialImageLatch();
+ releaseAfterGetInitialImageLatch();
+ releaseAfterRegionCreateEventLatch();
+ }
+
+ protected void releaseBeforeGetInitialImageLatch()
+ {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Releasing Initialization Latch (before initial image) for {}", getFullPath());
+ }
+ releaseLatch(this.initializationLatchBeforeGetInitialImage);
+ }
+
+ protected final void releaseAfterGetInitialImageLatch() {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Releasing Initialization Latch (after initial image) for {}", getFullPath());
+ }
+ releaseLatch(this.initializationLatchAfterGetInitialImage);
+ }
+
+ /**
+ * Called after we have delivered our REGION_CREATE event.
+ *
+ * @since 5.0
+ */
+ private void releaseAfterRegionCreateEventLatch()
+ {
+ releaseLatch(this.afterRegionCreateEventLatch);
+ }
+
+ /**
+ * Used to cause cache listener events to wait until the after region create
+ * event is delivered.
+ *
+ * @since 5.0
+ */
+ private void waitForRegionCreateEvent()
+ {
+ StoppableCountDownLatch l = this.afterRegionCreateEventLatch;
+ if (l != null && l.getCount() == 0) {
+ return;
+ }
+ waitOnInitialization(l);
+ }
+
+ private static void releaseLatch(StoppableCountDownLatch latch)
+ {
+ if (latch == null)
+ return;
+ latch.countDown();
+ }
+
+ /**
+ * Removes entries and recursively destroys subregions.
+ *
+ * @param eventSet
+ * collects the events for all destroyed regions if null, then we're
+ * closing so don't send events to callbacks or destroy the disk
+ * region
+ */
+ private void recursiveDestroyRegion(Set eventSet, RegionEventImpl p_event,
+ boolean cacheWrite) throws CacheWriterException, TimeoutException
+ {
+ RegionEventImpl event = p_event;
+ final boolean isClose = event.getOperation().isClose();
+ // do the cacheWriter beforeRegionDestroy first to fix bug 47736
+ if (eventSet != null && cacheWrite) {
+ try {
+ cacheWriteBeforeRegionDestroy(event);
+ }
+ catch (CancelException e) {
+ // I don't think this should ever happens: bulletproofing for bug 39454
+ if (!cache.forcedDisconnect()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_RECURSIVEDESTROYREGION_PROBLEM_IN_CACHEWRITEBEFOREREGIONDESTROY), e);
+ }
+ }
+ }
+
+ if (this.eventTracker != null) {
+ this.eventTracker.stop();
+ }
+ if (logger.isTraceEnabled(LogMarker.RVV) && getVersionVector() != null) {
+ logger.trace(LogMarker.RVV, "version vector for {} is {}", getName(), getVersionVector().fullToString());
+ }
+ cancelTTLExpiryTask();
+ cancelIdleExpiryTask();
+ cancelAllEntryExpiryTasks();
+ if (!isInternalRegion()) {
+ getCachePerfStats().incRegions(-1);
+ }
+ cache.getResourceManager(false).removeResourceListener(this);
+ if (getMembershipAttributes().hasRequiredRoles()) {
+ if (!isInternalRegion()) {
+ getCachePerfStats().incReliableRegions(-1);
+ }
+ }
+
+ // Note we need to do this even if we don't have a listener
+ // because of the SystemMemberCacheEventProcessor. Once we have
+ // a way to check for existence of SystemMemberCacheEventProcessor listeners
+ // then the add only needs to be done if hasListener || hasAdminListener
+ if (eventSet != null) { // && hasListener())
+ eventSet.add(event);
+ }
+
+ try {
+ // call recursiveDestroyRegion on each subregion and remove it
+ // from this subregion map
+ Collection values = this.subregions.values();
+ for (Iterator itr = values.iterator(); itr.hasNext();) {
+ Object element = itr.next(); // element is a LocalRegion
+ LocalRegion rgn;
+ try {
+ LocalRegion.setThreadInitLevelRequirement(LocalRegion.BEFORE_INITIAL_IMAGE);
+ try {
+ rgn = toRegion(element); // converts to a LocalRegion
+ } finally {
+ LocalRegion.setThreadInitLevelRequirement(LocalRegion.AFTER_INITIAL_IMAGE);
+ }
+ }
+ catch (CancelException e) {
+ rgn = (LocalRegion)element; // ignore, keep going through the motions though
+ }
+ catch (RegionDestroyedException rde) {
+ // SharedRegionData was destroyed
+ continue;
+ }
+
+ // if the region is destroyed, then it is a race condition with
+ // failed initialization removing it from the parent subregion map
+ if (rgn.isDestroyed) {
+ continue;
+ }
+ /** ** BEGIN operating on subregion of this region (rgn) *** */
+ if (eventSet != null) {
+ event = (RegionEventImpl)event.clone();
+ event.region = rgn;
+ }
+
+ try {
+ rgn.recursiveDestroyRegion(eventSet, event, cacheWrite);
+ if (!rgn.isInternalRegion()) {
+ InternalDistributedSystem system = rgn.cache.getDistributedSystem();
+ system.handleResourceEvent(ResourceEvent.REGION_REMOVE, rgn);
+ }
+ }
+ catch (CancelException e) {
+ // I don't think this should ever happen: bulletproofing for bug 39454
+ if (!cache.forcedDisconnect()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_RECURSIVEDESTROYREGION_RECURSION_FAILED_DUE_TO_CACHE_CLOSURE_REGION_0,
+ rgn.getFullPath()), e);
+ }
+ }
+ itr.remove(); // remove from this subregion map;
+ /** ** END operating on subregion of this region *** */
+ } // for
+
+ try {
+ if (this.indexManager != null) {
+ try {
+ if (this instanceof BucketRegion) {
+ this.indexManager.removeBucketIndexes(getPartitionedRegion());
+ }
+ this.indexManager.destroy();
+ }
+ catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }
+ catch (CancelException e) {
+ // I don't think this should ever happens: bulletproofing for bug 39454
+ if (!cache.forcedDisconnect()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_BASICDESTROYREGION_INDEX_REMOVAL_FAILED_DUE_TO_CACHE_CLOSURE_REGION_0,
+ getFullPath()), e);
+ }
+ }
+ }
+ finally {
+ // mark this region as destroyed.
+ if (event.isReinitializing()) {
+ this.reinitialized_old = true;
+ }
+ this.cache.setRegionByPath(getFullPath(), null);
+
+ if (this.eventTracker != null) {
+ this.eventTracker.stop();
+ }
+
+ if (this.diskRegion != null) {
+ this.diskRegion.prepareForClose(this);
+ }
+
+ this.isDestroyed = true;
+ // after isDestroyed is set to true call removeResourceListener to fix bug 49555
+ this.cache.getResourceManager(false).removeResourceListener(this);
+ closeEntries();
+ if (logger.isDebugEnabled()) {
+ logger.debug("recursiveDestroyRegion: Region Destroyed: {}", getFullPath());
+ }
+
+ // if eventSet is null then we need to close the listener as well
+ // otherwise, the listener will be closed after the destroy event
+ try {
+ postDestroyRegion(!isClose, event);
+ }
+ catch (CancelException e) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_RECURSIVEDESTROYREGION_POSTDESTROYREGION_FAILED_DUE_TO_CACHE_CLOSURE_REGION_0,
+ getFullPath()), e);
+ }
+
+ // Destroy cqs created aganist this Region in a server cache.
+ // fix for bug #47061
+ if (getServerProxy() == null) {
+ closeCqs();
+ }
+
+ detachPool();
+
+ if (eventSet != null) {
+ closeCallbacksExceptListener();
+ }
+ else {
+ closeAllCallbacks();
+ }
+ if (this.concurrencyChecksEnabled && this.dataPolicy.withReplication() && !this.cache.isClosed()) {
+ this.cache.getTombstoneService().unscheduleTombstones(this);
+ }
+ if (this.hasOwnStats) {
+ this.cachePerfStats.close();
+ }
+ }
+ }
+
+ public void closeEntries() {
+ this.entries.close();
+ }
+ public Set clearEntries(RegionVersionVector rvv) {
+ return this.entries.clear(rvv);
+ }
+
+ @Override
+ public void checkReadiness()
+ {
+ checkRegionDestroyed(true);
+ }
+
+ /**
+ * This method should be called when the caller cannot locate an entry and that condition
+ * is unexpected. This will first double check the cache and region state before throwing
+ * an EntryNotFoundException. EntryNotFoundException should be a last resort exception.
+ *
+ * @param entryKey the missing entry's key.
+ */
+ public void checkEntryNotFound(Object entryKey) {
+ checkReadiness();
+ // Localized string for partitioned region is generic enough for general use
+ throw new EntryNotFoundException(LocalizedStrings.PartitionedRegion_ENTRY_NOT_FOUND_FOR_KEY_0.toLocalizedString(entryKey));
+ }
+
+ /**
+
+ * Search for the value in a server (if one exists),
+ * then try a loader.
+ *
+ * If we find a value, we put it in the cache.
+ * @param preferCD return the CacheDeserializable, if that's what the value is.
+ * @param requestingClient the client making the request, if any
+ * @param clientEvent the client's event, if any. If not null, we set the version tag
+ * @param returnTombstones TODO
+ * @return the deserialized value
+ * @see DistributedRegion#findObjectInSystem(KeyInfo, boolean, TXStateInterface, boolean, Object, boolean, boolean, ClientProxyMembershipID, EntryEventImpl, boolean, boolean )
+ */
+ protected Object findObjectInSystem(KeyInfo keyInfo, boolean isCreate,
+ TXStateInterface tx, boolean generateCallbacks, Object localValue, boolean disableCopyOnRead, boolean preferCD, ClientProxyMembershipID requestingClient,
+ EntryEventImpl clientEvent, boolean returnTombstones, boolean allowReadFromHDFS)
+ throws CacheLoaderException, TimeoutException
+ {
+ final Object key = keyInfo.getKey();
+ final Object aCallbackArgument = keyInfo.getCallbackArg();
+ Object value = null;
+ boolean fromServer = false;
+ EntryEventImpl holder = null;
+
+ /*
+ * First lets try the server
+ */
+ {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ holder = EntryEventImpl.createVersionTagHolder();
+ try {
+ value = mySRP.get(key, aCallbackArgument, holder);
+ fromServer = value != null;
+ } finally {
+ holder.release();
+ }
+ }
+ }
+
+ /*
+ * If we didn't get anything from the server, try the loader
+ */
+ if (!fromServer || value == Token.TOMBSTONE) {
+ // copy into local var to prevent race condition
+ CacheLoader loader = basicGetLoader();
+ if (loader != null) {
+ final LoaderHelper loaderHelper
+ = loaderHelperFactory.createLoaderHelper(key, aCallbackArgument,
+ false /* netSearchAllowed */,
+ true /* netloadAllowed */,
+ null /* searcher */);
+ CachePerfStats stats = getCachePerfStats();
+ long statStart = stats.startLoad();
+ try {
+ value = loader.load(loaderHelper);
+ fromServer = false;
+ }
+ finally {
+ stats.endLoad(statStart);
+ }
+ }
+ }
+
+ // don't allow tombstones into a client cache if it doesn't
+ // have concurrency checks enabled
+ if (fromServer &&
+ value == Token.TOMBSTONE && !this.concurrencyChecksEnabled) {
+ value = null;
+ }
+
+ /*
+ * If we got a value back, let's put it in the cache.
+ */
+ RegionEntry re = null;
+ if (value != null && !isMemoryThresholdReachedForLoad()) {
+
+ long startPut = CachePerfStats.getStatTime();
+ validateKey(key);
+ Operation op;
+ if (isCreate) {
+ op = Operation.LOCAL_LOAD_CREATE;
+ }
+ else {
+ op = Operation.LOCAL_LOAD_UPDATE;
+ }
+
+ EntryEventImpl event
+ = EntryEventImpl.create(this, op, key, value, aCallbackArgument,
+ false, getMyId(), generateCallbacks);
+ try {
+
+ // bug #47716 - do not put an invalid entry into the cache if there's
+ // already one there with the same version
+ if (fromServer) {
+ if (alreadyInvalid(key, event)) {
+ return null;
+ }
+ event.setFromServer(fromServer);
+ event.setVersionTag(holder.getVersionTag());
+ if (clientEvent != null) {
+ clientEvent.setVersionTag(holder.getVersionTag());
+ }
+ }
+
+ //set the event id so that we can progagate
+ //the value to the server
+ if (!fromServer) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ try {
+ try {
+ re = basicPutEntry(event, 0L);
+ if (!fromServer && clientEvent != null) {
+ clientEvent.setVersionTag(event.getVersionTag());
+ clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
+ }
+ if (fromServer && (event.getRawNewValue() == Token.TOMBSTONE)) {
+ return null; // tombstones are destroyed entries
+ }
+ } catch (ConcurrentCacheModificationException e) {
+ // this means the value attempted to overwrite a newer modification and was rejected
+ if (logger.isDebugEnabled()) {
+ logger.debug("caught concurrent modification attempt when applying {}", event);
+ }
+ notifyBridgeClients(event);
+ }
+ if (!getDataView().isDeferredStats()) {
+ getCachePerfStats().endPut(startPut, event.isOriginRemote());
+ }
+ }
+ catch (CacheWriterException cwe) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("findObjectInSystem: writer exception putting entry {}", event, cwe);
+ }
+ }
+ } finally {
+ event.release();
+ }
+ }
+ if (isCreate) {
+ recordMiss(re, key);
+ }
+ return value;
+ }
+
+ protected boolean isMemoryThresholdReachedForLoad() {
+ return this.memoryThresholdReached.get();
+ }
+
+ /**
+ * Returns true if the cache already has this key as an invalid entry
+ * with a version >= the one in the given event. This is used in
+ * cache-miss processing to avoid overwriting the entry when it is
+ * not necessary, so that we avoid invoking cache listeners.
+ *
+ * @param key
+ * @param event
+ * @return whether the entry is already invalid
+ */
+ protected boolean alreadyInvalid(Object key, EntryEventImpl event) {
+ @Unretained(ENTRY_EVENT_NEW_VALUE)
+ Object newValue = event.getRawNewValue();
+ if (newValue == null || Token.isInvalid(newValue)) {
+ RegionEntry entry = this.entries.getEntry(key);
+ if (entry != null) {
+ synchronized(entry) {
+ if (entry.isInvalid()) {
+ VersionStamp stamp = entry.getVersionStamp();
+ if (stamp == null || event.getVersionTag() == null) {
+ return true;
+ }
+ if (stamp.getEntryVersion() >= event.getVersionTag().getEntryVersion()) {
+ return true;
+ }
+ }
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * @return true if cacheWrite was performed
+ * @see DistributedRegion#cacheWriteBeforeDestroy(EntryEventImpl, Object)
+ */
+ boolean cacheWriteBeforeDestroy(EntryEventImpl event, Object expectedOldValue)
+ throws CacheWriterException, EntryNotFoundException, TimeoutException
+ {
+ boolean result = false;
+ // copy into local var to prevent race condition
+ CacheWriter writer = basicGetWriter();
+ if (writer != null && event.getOperation() != Operation.REMOVE &&
+ !event.inhibitAllNotifications()) {
+ final long start = getCachePerfStats().startCacheWriterCall();
+ try {
+ writer.beforeDestroy(event);
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ result = true;
+ }
+ serverDestroy(event, expectedOldValue);
+ return result;
+ }
+
+ /** @return true if this was a client region; false if not */
+ protected boolean bridgeWriteBeforeDestroy(EntryEventImpl event, Object expectedOldValue)
+ throws CacheWriterException, EntryNotFoundException, TimeoutException
+ {
+ if (hasServerProxy()) {
+ serverDestroy(event, expectedOldValue);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ * @since 5.7
+ */
+ protected void serverRegionDestroy(RegionEventImpl regionEvent) {
+ if (regionEvent.getOperation().isDistributed()) {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ EventID eventId = regionEvent.getEventId();
+ Object callbackArg = regionEvent.getRawCallbackArgument();
+ mySRP.destroyRegion(eventId, callbackArg);
+ }
+ }
+ }
+
+ /**
+ * @since 5.7
+ */
+ protected void serverRegionClear(RegionEventImpl regionEvent) {
+ if (regionEvent.getOperation().isDistributed()) {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ EventID eventId = regionEvent.getEventId();
+ Object callbackArg = regionEvent.getRawCallbackArgument();
+ mySRP.clear(eventId, callbackArg);
+ }
+ }
+ }
+ /**
+ * @since 5.7
+ */
+ protected void serverRegionInvalidate(RegionEventImpl regionEvent) {
+ if (regionEvent.getOperation().isDistributed()) {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ // @todo grid: add a client to server Op message for this
+ }
+ }
+ }
+
+ /**
+ * @since 5.7
+ */
+ protected void serverInvalidate(EntryEventImpl event, boolean invokeCallbacks,
+ boolean forceNewEntry) {
+ if (event.getOperation().isDistributed()) {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ mySRP.invalidate(event);
+ }
+ }
+ }
+
+ /**
+ * @since 5.7
+ */
+ protected void serverPut(EntryEventImpl event,
+ boolean requireOldValue, Object expectedOldValue) {
+ if (event.getOperation().isDistributed() && !event.isFromServer()) {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ if (event.isBulkOpInProgress()) {
+ // this is a put all, ignore this!
+ return;
+ }
+ Operation op = event.getOperation();
+ // @todo grid: is the newEntry flag needed?
+ Object key = event.getKey();
+ Object value = event.getRawNewValue();
+ // serverPut is called by cacheWriteBeforePut so the new value will not yet be off-heap
+ // TODO OFFHEAP: verify that the above assertion is true
+ Object callbackArg = event.getRawCallbackArgument();
+ boolean isCreate = event.isCreate();
+ Object result = mySRP.put(key, value, event.getDeltaBytes(), event,
+ op, requireOldValue, expectedOldValue,
+ callbackArg, isCreate);
+ // bug #42296, serverproxy returns null when cache is closing
+ getCancelCriterion().checkCancelInProgress(null);
+ // if concurrent map operations failed we don't want the region map
+ // to apply the operation and need to throw an exception
+ if (op.guaranteesOldValue()) {
+ if (op != Operation.REPLACE || requireOldValue) {
+ event.setConcurrentMapOldValue(result);
+ }
+ if (op == Operation.PUT_IF_ABSENT) {
+ if (result != null) {
+ throw new EntryNotFoundException("entry existed for putIfAbsent"); // customers don't see this exception
+ }
+ } else if (op == Operation.REPLACE) {
+ if (requireOldValue && result == null) {
+ throw new EntryNotFoundException("entry not found for replace");
+ } else if (!requireOldValue) {
+ if ( !((Boolean)result).booleanValue() ) {
+ throw new EntryNotFoundException("entry found with wrong value"); // customers don't see this exception
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Destroy an entry on the server given its event.
+ * @since 5.7
+ */
+ protected void serverDestroy(EntryEventImpl event, Object expectedOldValue) {
+ if (event.getOperation().isDistributed()) {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ // send to server
+ Object key = event.getKey();
+ Object callbackArg = event.getRawCallbackArgument();
+ Object result = mySRP.destroy(key, expectedOldValue, event.getOperation(), event, callbackArg);
+ if (result instanceof EntryNotFoundException) {
+ throw (EntryNotFoundException)result;
+ }
+ }
+ }
+ }
+
+ /**
+ * @return true if cacheWrite was performed
+ * @see DistributedRegion#cacheWriteBeforeRegionDestroy(RegionEventImpl)
+ */
+ boolean cacheWriteBeforeRegionDestroy(RegionEventImpl event)
+ throws CacheWriterException, TimeoutException
+ {
+ boolean result = false;
+ // copy into local var to prevent race condition
+ CacheWriter writer = basicGetWriter();
+ if (writer != null) {
+ final long start = getCachePerfStats().startCacheWriterCall();
+ try {
+ writer.beforeRegionDestroy(event);
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ result = true;
+ }
+ serverRegionDestroy(event);
+ return result;
+ }
+
+ protected boolean cacheWriteBeforeRegionClear(RegionEventImpl event)
+ throws CacheWriterException, TimeoutException
+ {
+ boolean result = false;
+ // copy into local var to prevent race condition
+ CacheWriter writer = basicGetWriter();
+ if (writer != null) {
+ final long start = getCachePerfStats().startCacheWriterCall();
+ try {
+ writer.beforeRegionClear(event);
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ result = true;
+ }
+ serverRegionClear(event);
+ return result;
+ }
+
+ /**
+ * @since 5.7
+ */
+ void cacheWriteBeforeInvalidate(EntryEventImpl event, boolean invokeCallbacks, boolean forceNewEntry) {
+ if (!event.getOperation().isLocal() && !event.isOriginRemote()) {
+ serverInvalidate(event, invokeCallbacks, forceNewEntry);
+ }
+ }
+
+ /**
+ * @see DistributedRegion#cacheWriteBeforePut(EntryEventImpl, Set, CacheWriter, boolean, Object)
+ * @param event
+ * @param netWriteRecipients
+ * @param localWriter
+ * @param requireOldValue
+ * @param expectedOldValue
+ * @throws CacheWriterException
+ * @throws TimeoutException
+ */
+ void cacheWriteBeforePut(EntryEventImpl event, Set netWriteRecipients,
+ CacheWriter localWriter,
+ boolean requireOldValue,
+ Object expectedOldValue)
+ throws CacheWriterException, TimeoutException
+ {
+ Assert.assertTrue(netWriteRecipients == null);
+ Operation op = event.getOperation();
+ if (!(op == Operation.PUT_IF_ABSENT
+ || op == Operation.REPLACE) && (localWriter != null) &&
+ !((EntryEventImpl)event).inhibitAllNotifications()) {
+ final long start = getCachePerfStats().startCacheWriterCall();
+ final boolean newEntry = event.getOperation().isCreate();
+ try {
+ if (!newEntry) {
+ localWriter.beforeUpdate(event);
+ }
+ else {
+ localWriter.beforeCreate(event);
+ }
+ }
+ finally {
+ getCachePerfStats().endCacheWriterCall(start);
+ }
+ }
+ serverPut(event, requireOldValue, expectedOldValue);
+ }
+
+ protected void validateArguments(Object key, Object value,
+ Object aCallbackArgument)
+ {
+ validateKey(key);
+ validateValue(value);
+ validateCallbackArg(aCallbackArgument);
+
+ }
+
+ protected void validateKey(Object key)
+ {
+ if (key == null) {
+ throw new NullPointerException(LocalizedStrings.LocalRegion_KEY_CANNOT_BE_NULL.toLocalizedString());
+ }
+
+ // check validity of key against keyConstraint
+ if (this.keyConstraint != null) {
+ if (!this.keyConstraint.isInstance(key))
+ throw new ClassCastException(LocalizedStrings.LocalRegion_KEY_0_DOES_NOT_SATISFY_KEYCONSTRAINT_1.toLocalizedString(new Object[] {key.getClass().getName(), this.keyConstraint.getName()}));
+ }
+
+ // We don't need to check that the key is Serializable. Instead,
+ // we let the lower-level (data) serialization mechanism take care
+ // of this for us. See bug 32394.
+ }
+
+ /**
+ * Starting in 3.5, we don't check to see if the callback argument is
+ * Serializable
. We instead rely on the actual serialization
+ * (which happens in-thread with the put) to tell us if there are any
+ * problems.
+ */
+ protected void validateCallbackArg(Object aCallbackArgument)
+ {
+
+ }
+
+ /**
+ * @since 5.0.2
+ */
+ private final boolean DO_EXPENSIVE_VALIDATIONS = Boolean.getBoolean("gemfire.DO_EXPENSIVE_VALIDATIONS");
+
+ /**
+ * the number of tombstone entries in the RegionMap
+ */
+ protected AtomicInteger tombstoneCount = new AtomicInteger();
+
+ /** a boolean for issuing a client/server configuration mismatch message */
+ private boolean concurrencyMessageIssued;
+
+ /**
+ * Starting in 3.5, we don't check to see if the value is
+ * Serializable
. We instead rely on the actual serialization
+ * (which happens in-thread with the put) to tell us if there are any
+ * problems.
+ */
+ protected void validateValue(Object p_value)
+ {
+ Object value = p_value;
+ // check validity of value against valueConstraint
+ if (this.valueConstraint != null) {
+ if (value != null) {
+ if (value instanceof CachedDeserializable) {
+ if (DO_EXPENSIVE_VALIDATIONS) {
+ value = ((CachedDeserializable)value).getDeserializedValue(null, null);
+ } else {
+ return;
+ }
+ }
+ if (!this.valueConstraint.isInstance(value))
+ throw new ClassCastException(LocalizedStrings.LocalRegion_VALUE_0_DOES_NOT_SATISFY_VALUECONSTRAINT_1.toLocalizedString(new Object[] {value.getClass().getName(), this.valueConstraint.getName()}));
+ }
+ }
+ }
+
+ public CachePerfStats getCachePerfStats() {
+ // return this.cache.getCachePerfStats();
+ return this.cachePerfStats;
+ }
+ public CachePerfStats getRegionPerfStats() {
+ return this.cachePerfStats;
+ }
+
+ /** regions track the number of tombstones their map holds for size calculations */
+ public void incTombstoneCount(int delta) {
+ this.tombstoneCount.addAndGet(delta);
+ this.cachePerfStats.incTombstoneCount(delta);
+
+ //Fix for 45204 - don't include the tombstones in
+ //any of our entry count stats.
+ this.cachePerfStats.incEntryCount(-delta);
+ if(getDiskRegion() != null) {
+ getDiskRegion().incNumEntriesInVM(-delta);
+ }
+ DiskEntry.Helper.incrementBucketStats(this, -delta/*InVM*/, 0/*OnDisk*/, 0);
+ }
+
+ public int getTombstoneCount() {
+ return this.tombstoneCount.get();
+ }
+
+ public void scheduleTombstone(RegionEntry entry, VersionTag destroyedVersion) {
+ if (destroyedVersion == null) {
+ throw new NullPointerException("destroyed version tag cannot be null");
+ }
+// Object sync = TombstoneService.DEBUG_TOMBSTONE_COUNT? TombstoneService.debugSync : new Object();
+// lastUnscheduled.set(null);
+// synchronized(sync) {
+ incTombstoneCount(1);
+// if (entry instanceof AbstractRegionEntry) {
+// AbstractRegionEntry are = (AbstractRegionEntry)entry;
+// if (are.isTombstoneScheduled()) {
+// log.severe(LocalizedStrings.DEBUG, "Scheduling a tombstone for an entry that is already a tombstone: " + entry, new Exception("stack trace"));
+// throw new IllegalStateException("Attempt to schedule a tombstone for a destroyed entry that is already scheduled for expiration");
+// }
+// are.setTombstoneScheduled(true);
+// }
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE_COUNT)) {
+ logger.trace(LogMarker.TOMBSTONE_COUNT, "scheduling tombstone for {} version={} count is {} entryMap size is {}",
+ entry.getKey(), entry.getVersionStamp().asVersionTag(), this.tombstoneCount.get(), this.entries.size()/*, new Exception("stack trace")*/);
+ // this can be useful for debugging tombstone count problems if there aren't a lot of concurrent threads
+// if (TombstoneService.DEBUG_TOMBSTONE_COUNT && this.entries instanceof AbstractRegionMap) {
+// ((AbstractRegionMap)this.entries).verifyTombstoneCount(tombstoneCount);
+// }
+ }
+ getGemFireCache().getTombstoneService().scheduleTombstone(this, entry,
+ destroyedVersion);
+// }
+ }
+
+// ThreadLocal lastUnscheduled = new ThreadLocal();
+// ThreadLocal lastUnscheduledPlace = new ThreadLocal();
+
+ public void rescheduleTombstone(RegionEntry entry, VersionTag version) {
+ Object sync = TombstoneService.DEBUG_TOMBSTONE_COUNT? TombstoneService.debugSync : new Object();
+ synchronized(sync) {
+ unscheduleTombstone(entry, false); // count is off by one, so don't allow validation to take place
+ scheduleTombstone(entry, version);
+ }
+
+ }
+
+ public void unscheduleTombstone(RegionEntry entry) {
+ unscheduleTombstone(entry, true);
+ }
+
+ private void unscheduleTombstone(RegionEntry entry, boolean validate) {
+ incTombstoneCount(-1);
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE)) {
+ logger.trace(LogMarker.TOMBSTONE, "unscheduling tombstone for {} count is {} entryMap size is {}",
+ entry.getKey(), this.tombstoneCount.get(), this.entries.size()/*, new Exception("stack trace")*/);
+ }
+ getRegionMap().unscheduleTombstone(entry);
+ if (logger.isTraceEnabled(LogMarker.TOMBSTONE_COUNT) && validate) {
+ if (this.entries instanceof AbstractRegionMap) {
+ ((AbstractRegionMap) this.entries).verifyTombstoneCount(this.tombstoneCount);
+ }
+ }
+ // we don't have to remove the entry from the sweeper since the version has
+ // changed. It would be costly to iterate over the tombstone list for
+ // every tombstone exhumed while holding the entry's lock
+ //this.cache.getTombstoneService().unscheduleTombstone(entry);
+ }
+
+ /** remove any tombstones from the given member that are <= the given version
+ * @param eventID event identifier for the GC operation
+ * @param clientRouting routing info (if null a routing is computed)
+ */
+ public void expireTombstones(Map regionGCVersions, EventID eventID, FilterInfo clientRouting) {
+ Set keys = null;
+ if (!this.concurrencyChecksEnabled) {
+ return;
+ }
+ if (!this.versionVector.containsTombstoneGCVersions(regionGCVersions)) {
+ keys = this.cache.getTombstoneService().gcTombstones(this, regionGCVersions);
+ if (keys == null) {
+ // deltaGII prevented tombstone GC
+ return;
+ }
+ }
+ if (eventID != null) { // bug #50683 - old members might not send an eventID
+ notifyClientsOfTombstoneGC(regionGCVersions, keys, eventID, clientRouting);
+ }
+ }
+
+ public void expireTombstoneKeys(Set tombstoneKeys) {
+ if (this.concurrencyChecksEnabled) {
+ this.cache.getTombstoneService().gcTombstoneKeys(this, tombstoneKeys);
+ }
+ }
+
+
+ /** pass tombstone garbage-collection info to clients
+ * @param eventID the ID of the event (see bug #50683)
+ * @param routing routing info (routing is computed if this is null)
+ */
+ protected void notifyClientsOfTombstoneGC(Map regionGCVersions, SetkeysRemoved, EventID eventID, FilterInfo routing) {
+ if (CacheClientNotifier.getInstance() != null) {
+ // Only route the event to clients interested in the partitioned region.
+ // We do this by constructing a region-level event and then use it to
+ // have the filter profile ferret out all of the clients that have interest
+ // in this region
+ FilterProfile fp = getFilterProfile();
+ if (fp != null || routing != null) { // null check - fix for bug #45614
+ RegionEventImpl regionEvent = new RegionEventImpl(this, Operation.REGION_DESTROY, null, true, getMyId());
+ regionEvent.setEventID(eventID);
+ FilterInfo clientRouting = routing;
+ if (clientRouting == null) {
+ clientRouting = fp.getLocalFilterRouting(regionEvent);
+ }
+ regionEvent.setLocalFilterInfo(clientRouting);
+ ClientUpdateMessage clientMessage = ClientTombstoneMessage.gc(this, regionGCVersions,
+ eventID);
+ CacheClientNotifier.notifyClients(regionEvent, clientMessage);
+ }
+ }
+ }
+
+
+ /** local regions do not perform versioning */
+ protected boolean shouldGenerateVersionTag(RegionEntry entry, EntryEventImpl event) {
+ if (this.getDataPolicy().withPersistence()) {
+ return true;
+ } else {
+ return this.concurrencyChecksEnabled && (entry.getVersionStamp().hasValidVersion() || this.dataPolicy.withReplication());
+ }
+ }
+
+ protected void enableConcurrencyChecks() {
+ this.concurrencyChecksEnabled = true;
+ if (this.dataPolicy.withStorage()) {
+ RegionEntryFactory versionedEntryFactory = this.entries.getEntryFactory().makeVersioned();
+ Assert.assertTrue(this.entries.size() == 0, "RegionMap should be empty but was of size:"+this.entries.size());
+ this.entries.setEntryFactory(versionedEntryFactory);
+ createVersionVector();
+ }
+ }
+
+ /**
+ * validate attributes of subregion being created, sent to parent
+ *
+ * @throws IllegalArgumentException
+ * if attrs is null
+ * @throws IllegalStateException
+ * if attributes are invalid
+ */
+ protected void validateSubregionAttributes(RegionAttributes attrs)
+ {
+ if (attrs == null) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_REGION_ATTRIBUTES_MUST_NOT_BE_NULL.toLocalizedString());
+ }
+ if (this.scope == Scope.LOCAL && attrs.getScope() != Scope.LOCAL) {
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_A_REGION_WITH_SCOPELOCAL_CAN_ONLY_HAVE_SUBREGIONS_WITH_SCOPELOCAL.toLocalizedString());
+ }
+ }
+
+ /**
+ * Returns the value of the entry with the given key as it is stored in the
+ * VM. This means that if the value is invalid, the invalid token will be
+ * returned. If the value is a {@link CachedDeserializable}received from
+ * another VM, that object will be returned. If the value does not reside in
+ * the VM because it has been overflowed to disk, null
will be
+ * returned. This method is intended for testing.testing purposes only.
+ *
+ * @throws EntryNotFoundException
+ * No entry with key
exists
+ *
+ * @see RegionMap#getEntry
+ *
+ * @since 3.2
+ */
+ public Object getValueInVM(Object key) throws EntryNotFoundException
+ {
+ return basicGetValueInVM(key, true/*rememberRead*/);
+ }
+
+ public Object getValueInVM(EntryEventImpl event)
+ throws EntryNotFoundException {
+ return basicGetValueInVM(event.getKey(), true/*rememberRead*/);
+ }
+
+ /**
+ * @param rememberRead true if read should be remembered in a transaction
+ * @since 5.5
+ */
+ private Object basicGetValueInVM(Object key, boolean rememberRead)
+ throws EntryNotFoundException
+ {
+ return getDataView().getValueInVM(getKeyInfo(key), this, rememberRead);
+ }
+
+ /**
+ * @param keyInfo
+ * @return TODO
+ */
+ @Retained
+ protected Object nonTXbasicGetValueInVM(KeyInfo keyInfo) {
+ RegionEntry re = this.entries.getEntry(keyInfo.getKey());
+ if (re == null) {
+ checkEntryNotFound(keyInfo.getKey());
+ }
+ Object v = re.getValueInVM(this); // OFFHEAP returned to callers
+ if (Token.isRemoved(v)) {
+ checkEntryNotFound(keyInfo.getKey());
+ }
+ if (v == Token.NOT_AVAILABLE) {
+ return null;
+ }
+ return v;
+ }
+
+ /**
+ * This is a test hook method used to find out what keys the current tx
+ * has read or written.
+ * @return an unmodifiable set of keys that have been read or written
+ * by the transaction on this thread.
+ * @throws IllegalStateException if not tx in progress
+ * @since 5.5
+ */
+ public Set testHookTXKeys() {
+ if (!isTX()) throw new IllegalStateException(LocalizedStrings.LocalRegion_TX_NOT_IN_PROGRESS.toLocalizedString());
+ TXStateProxyImpl tx = (TXStateProxyImpl) getTXState();
+ if (!tx.isRealDealLocal()) {
+ return Collections.EMPTY_SET;
+ }
+ TXRegionState txr = txReadRegion();
+ if (txr == null) {
+ return Collections.EMPTY_SET;
+ } else {
+ return txr.getEntryKeys();
+ }
+ }
+
+ /**
+ * Returns the value of the entry with the given key as it is stored on disk.
+ * While the value may be read from disk, it is not stored into the
+ * entry in the VM. This method is intended for testing purposes only.
+ * DO NOT use in product code else it will break SQLFabric that has cases
+ * where routing object is not part of only the key.
+ *
+ * @throws EntryNotFoundException
+ * No entry with key
exists
+ * @throws IllegalStateException
+ * If this region does not write to disk
+ *
+ * @see RegionEntry#getValueOnDisk
+ *
+ * @since 3.2
+ */
+ public Object getValueOnDisk(Object key) throws EntryNotFoundException
+ {
+ // Ok for this to ignore tx state
+ RegionEntry re = this.entries.getEntry(key);
+ if (re == null) {
+ throw new EntryNotFoundException(key.toString());
+ }
+ return re.getValueOnDisk(this);
+ }
+
+ /**
+ * Gets the value from VM, if present, otherwise from disk without fault in.
+ *
+ * @param key
+ * @return Object
+ * @throws EntryNotFoundException
+ */
+ public Object getValueInVMOrDiskWithoutFaultIn(Object key) throws EntryNotFoundException {
+ RegionEntry re = this.entries.getEntry(key);
+ if (re == null) {
+ throw new EntryNotFoundException(key.toString());
+ }
+ return re.getValueInVMOrDiskWithoutFaultIn(this);
+ }
+
+ /**
+ * Get the serialized bytes from disk. This method only looks for the value on
+ * the disk, ignoring heap data. This method is intended for testing purposes
+ * only. DO NOT use in product code else it will break SQLFabric that has
+ * cases where routing object is not part of only the key.
+ *
+ * @param key the object whose hashCode is used to find the value
+ * @return either a byte array, a CacheDeserializable with the serialized value,
+ * or null if the entry exists but no value data exists.
+ * @throws IllegalStateException when the region is not persistent
+ * @throws EntryNotFoundException if there is no entry for the given key
+ * @since gemfire5.7_hotfix
+ */
+ public Object getSerializedValueOnDisk(Object key) throws EntryNotFoundException
+ {
+ // Ok for this to ignore tx state
+ RegionEntry re = this.entries.getEntry(key);
+ if (re == null) {
+ throw new EntryNotFoundException(key.toString());
+ }
+ Object result = re.getSerializedValueOnDisk(this);
+ if (Token.isInvalid(result)) {
+ result = null;
+ } else if (Token.isRemoved(result)) {
+ throw new EntryNotFoundException(key.toString());
+ }
+ return result;
+ }
+
+ /**
+ * Returns the value of the entry with the given key as it is stored present
+ * in the buffer or disk. While the value may be read from disk or buffer,
+ * it is not
+ * stored into the entry in the VM. This is different from getValueonDisk in that
+ * it checks for a value both in asynch buffers ( subject to asynch mode
+ * enabled) as well as Disk
+ *
+ * @throws EntryNotFoundException
+ * No entry with key
exists
+ * @throws IllegalStateException
+ * If this region does not write to disk
+ *
+ * @see RegionEntry#getValueOnDisk
+ *
+ * @since 5.1
+ */
+ public Object getValueOnDiskOrBuffer(Object key)
+ throws EntryNotFoundException {
+ // Ok for this to ignore tx state
+ RegionEntry re = this.entries.getEntry(key);
+ if (re == null) {
+ throw new EntryNotFoundException(key.toString());
+ }
+ return re.getValueOnDiskOrBuffer(this);
+ }
+
+ /**
+ * Does a get that attempts to not fault values in from disk or make the entry
+ * the most recent in the LRU.
+ *
+ * Originally implemented in WAN gateway code and moved here in the sqlfire
+ * "cheetah" branch.
+ * @param adamant fault in and affect LRU as a last resort
+ * @param allowTombstone also return Token.TOMBSTONE if the entry is deleted
+ * @param serializedFormOkay if the serialized form can be returned
+ */
+ public Object getNoLRU(Object k, boolean adamant, boolean allowTombstone, boolean serializedFormOkay) {
+ Object o = null;
+ try {
+ o = getValueInVM(k); // OFFHEAP deserialize
+ if (o == null) {
+ // must be on disk
+ // fault it in w/o putting it back in the region
+ o = getValueOnDiskOrBuffer(k);
+ if (o == null) {
+ // try memory one more time in case it was already faulted back in
+ o = getValueInVM(k); // OFFHEAP deserialize
+ if (o == null) {
+ if (adamant) {
+ o = get(k);
+ }
+ } else {
+ if (!serializedFormOkay && (o instanceof CachedDeserializable)) {
+ o = ((CachedDeserializable)o).getDeserializedValue(this,
+ getRegionEntry(k));
+ }
+ }
+ }
+ } else {
+ if (!serializedFormOkay && (o instanceof CachedDeserializable)) {
+ o = ((CachedDeserializable)o).getDeserializedValue(this,
+ getRegionEntry(k));
+ }
+ }
+ } catch (EntryNotFoundException ok) {
+ // just return null;
+ }
+ if (o == Token.TOMBSTONE && !allowTombstone) {
+ o = null;
+ }
+ return o;
+ }
+
+ /**
+ * Bump this number any time an incompatible change is made to the snapshot
+ * format.
+ */
+ private static final byte SNAPSHOT_VERSION = 1;
+
+ private static final byte SNAPSHOT_VALUE_OBJ = 23;
+
+ private static final byte SNAPSHOT_VALUE_INVALID = 24;
+
+ private static final byte SNAPSHOT_VALUE_LOCAL_INVALID = 25;
+
+ public void saveSnapshot(OutputStream outputStream) throws IOException
+ {
+ if (isProxy()) {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_REGIONS_WITH_DATAPOLICY_0_DO_NOT_SUPPORT_SAVESNAPSHOT.toLocalizedString(getDataPolicy()));
+ }
+ checkForNoAccess();
+ DataOutputStream out = new DataOutputStream(outputStream);
+ try {
+ out.writeByte(SNAPSHOT_VERSION);
+ for (Iterator itr = entries(false).iterator(); itr.hasNext();) {
+ Region.Entry entry = (Region.Entry)itr.next();
+ try {
+ Object key = entry.getKey();
+ Object value = entry.getValue();
+ if (value == Token.TOMBSTONE) {
+ continue;
+ }
+ DataSerializer.writeObject(key, out);
+ if (value == null) { // fix for bug 33311
+ LocalRegion.NonTXEntry lre = (LocalRegion.NonTXEntry)entry;
+ RegionEntry re = lre.getRegionEntry();
+ value = re.getValue(this); // OFFHEAP: incrc, copy info heap cd for serialization, decrc
+ if (value == Token.INVALID) {
+ out.writeByte(SNAPSHOT_VALUE_INVALID);
+ }
+ else if (value == Token.LOCAL_INVALID) {
+ out.writeByte(SNAPSHOT_VALUE_LOCAL_INVALID);
+ }
+ else {
+ out.writeByte(SNAPSHOT_VALUE_OBJ);
+ DataSerializer.writeObject(value, out);
+ }
+ }
+ else {
+ out.writeByte(SNAPSHOT_VALUE_OBJ);
+ DataSerializer.writeObject(value, out);
+ }
+ }
+ catch (EntryDestroyedException e) {
+ // continue to next entry
+ }
+ }
+ // write NULL terminator
+ DataSerializer.writeObject(null, out);
+ }
+ finally {
+ out.close();
+ }
+ }
+
+ public void loadSnapshot4ConvertTo65(InputStream inputStream) throws CacheWriterException, TimeoutException, ClassNotFoundException, IOException {
+ isConversion.set(Boolean.valueOf(true));
+ try {
+ loadSnapshot(inputStream);
+ } finally {
+ isConversion.remove();
+ }
+ }
+
+ public void loadSnapshot(InputStream inputStream)
+ throws CacheWriterException, TimeoutException, ClassNotFoundException,
+ IOException
+ {
+ if (isProxy()) {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_REGIONS_WITH_DATAPOLICY_0_DO_NOT_SUPPORT_LOADSNAPSHOT.toLocalizedString(getDataPolicy()));
+ }
+ if (inputStream == null) {
+ throw new NullPointerException(LocalizedStrings.LocalRegion_INPUTSTREAM_MUST_NOT_BE_NULL.toLocalizedString());
+ }
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.REGION_LOAD_SNAPSHOT, null, false, getMyId(),
+ generateEventID()/* generate EventID */);
+ reinitialize(inputStream, event);
+ }
+
+ public void registerInterest(Object key)
+ {
+ registerInterest(key, false);
+ }
+
+ public void registerInterest(Object key, boolean isDurable) {
+ registerInterest(key, isDurable, true);
+ }
+
+ public void registerInterest(Object key, boolean isDurable,
+ boolean receiveValues)
+ {
+ registerInterest(key, InterestResultPolicy.DEFAULT, isDurable, receiveValues);
+ }
+
+ public void startRegisterInterest() {
+ getImageState().writeLockRI();
+ try {
+ cache.registerInterestStarted();
+ this.riCnt++;
+ } finally {
+ getImageState().writeUnlockRI();
+ }
+ }
+
+ public void finishRegisterInterest() {
+ if (Boolean.getBoolean("gemfire.testing.slow-interest-recovery")) {
+ if(logger.isDebugEnabled()) {
+ logger.debug("slowing interest recovery...");
+ }
+ try { Thread.sleep(20000); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; }
+ if(logger.isDebugEnabled()) {
+ logger.debug("done slowing interest recovery");
+ }
+ }
+ boolean gotLock = false;
+ try {
+ getImageState().writeLockRI();
+ gotLock = true;
+ this.riCnt--;
+ Assert.assertTrue(this.riCnt >= 0 , "register interest count can not be < 0 ");
+ if (this.riCnt == 0) {
+ // remove any destroyed entries from the region and clear the hashset
+ destroyEntriesAndClearDestroyedKeysSet();
+ }
+ } finally {
+ cache.registerInterestCompleted();
+ if (gotLock) {
+ getImageState().writeUnlockRI();
+ }
+ }
+ }
+
+ // TODO this is distressingly similar to code in the client.internal package
+ private void processSingleInterest(Object key, int interestType,
+ InterestResultPolicy pol, boolean isDurable,
+ boolean receiveUpdatesAsInvalidates)
+ {
+ final ServerRegionProxy proxy = getServerProxy();
+ if (proxy == null) {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_REGISTRATION_REQUIRES_A_POOL.toLocalizedString());
+ }
+ if (isDurable && !proxy.getPool().isDurableClient()) {
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_DURABLE_FLAG_ONLY_APPLICABLE_FOR_DURABLE_CLIENTS.toLocalizedString());
+ }
+ if (!proxy.getPool().getSubscriptionEnabled()) {
+ String msg = "Interest registration requires a pool whose queue is enabled.";
+ throw new SubscriptionNotEnabledException(msg);
+ }
+
+ if (getAttributes().getDataPolicy().withReplication() // fix for bug 36185
+ && !getAttributes().getScope().isLocal()) { // fix for bug 37692
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_REGISTRATION_NOT_SUPPORTED_ON_REPLICATED_REGIONS.toLocalizedString());
+ }
+
+ if (key == null)
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_INTEREST_KEY_MUST_NOT_BE_NULL.toLocalizedString());
+ // Sequence of events, on a single entry:
+ // 1. Client puts value (a).
+ // 2. Server updates with value (b). Client never gets the update,
+ // because it isn't interested in that key.
+ // 3. Client registers interest.
+ // At this point, there is an entry in the local cache, but it is
+ // inconsistent with the server.
+ //
+ // Because of this, we must _always_ destroy and refetch affected values
+ // during registerInterest.
+ startRegisterInterest();
+ try {
+ List serverKeys;
+
+ this.clearKeysOfInterest(key, interestType, pol);
+ // Checking for the Dunit test(testRegisterInterst_Destroy_Concurrent) flag
+ if (PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG) {
+ ClientServerObserver bo = ClientServerObserverHolder.getInstance();
+ bo.beforeInterestRegistration();
+ }// Test Code Ends
+ final byte regionDataPolicy = getAttributes().getDataPolicy().ordinal;
+ switch (interestType) {
+ case InterestType.FILTER_CLASS:
+ serverKeys = proxy.registerInterest(key, interestType, pol,
+ isDurable, receiveUpdatesAsInvalidates, regionDataPolicy);
+ break;
+ case InterestType.KEY:
+
+ if (key instanceof String && key.equals("ALL_KEYS")) {
+
+ serverKeys = proxy.registerInterest(".*",
+ InterestType.REGULAR_EXPRESSION,
+ pol,
+ isDurable,
+ receiveUpdatesAsInvalidates,
+ regionDataPolicy);
+ }
+ else {
+ if (key instanceof List) {
+ serverKeys = proxy.registerInterestList((List)key, pol,
+ isDurable, receiveUpdatesAsInvalidates, regionDataPolicy);
+ } else {
+ serverKeys = proxy.registerInterest(key, InterestType.KEY, pol,
+ isDurable, receiveUpdatesAsInvalidates, regionDataPolicy);
+ }
+ }
+ break;
+ case InterestType.OQL_QUERY:
+ serverKeys = proxy.registerInterest(key, InterestType.OQL_QUERY, pol,
+ isDurable, receiveUpdatesAsInvalidates, regionDataPolicy);
+ break;
+ case InterestType.REGULAR_EXPRESSION: {
+ String regex = (String)key;
+ // compile regex throws java.util.regex.PatternSyntaxException if invalid
+ // we do this before sending to the server because it's more efficient
+ // and the client is not receiving exception messages properly
+ Pattern.compile(regex);
+ serverKeys = proxy.registerInterest(regex,
+ InterestType.REGULAR_EXPRESSION,
+ pol,
+ isDurable,
+ receiveUpdatesAsInvalidates,
+ regionDataPolicy);
+ break;
+ }
+ default:
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_UNKNOWN_INTEREST_TYPE.toLocalizedString());
+ }
+ boolean finishedRefresh = false;
+ try {
+ refreshEntriesFromServerKeys(null, serverKeys, pol);
+
+ finishedRefresh = true;
+ }
+ finally {
+ if (!finishedRefresh) {
+ // unregister before throwing the exception caused by the refresh
+ switch (interestType) {
+ case InterestType.FILTER_CLASS:
+ proxy.unregisterInterest(key, interestType, false, false);
+ break;
+ case InterestType.KEY:
+ if (key instanceof String && key.equals("ALL_KEYS")) {
+ proxy.unregisterInterest(".*", InterestType.REGULAR_EXPRESSION, false, false);
+ }
+ else if (key instanceof List) {
+ proxy.unregisterInterestList((List)key, false, false);
+ } else {
+ proxy.unregisterInterest(key, InterestType.KEY, false, false);
+ }
+ break;
+ case InterestType.OQL_QUERY:
+ proxy.unregisterInterest(key, InterestType.OQL_QUERY, false, false);
+ break;
+ case InterestType.REGULAR_EXPRESSION: {
+ proxy.unregisterInterest(key, InterestType.REGULAR_EXPRESSION, false, false);
+ break;
+ }
+ default:
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_UNKNOWN_INTEREST_TYPE.toLocalizedString());
+ }
+ }
+ }
+ }
+ finally {
+ finishRegisterInterest();
+ }
+ }
+
+ public void registerInterest(Object key, InterestResultPolicy policy)
+ {
+ registerInterest(key, policy, false);
+ }
+ public void registerInterest(Object key, InterestResultPolicy policy,
+ boolean isDurable) {
+ registerInterest(key, policy, isDurable, true);
+ }
+ public void registerInterest(Object key, InterestResultPolicy policy,
+ boolean isDurable, boolean receiveValues)
+ {
+ processSingleInterest(key, InterestType.KEY, policy, isDurable, !receiveValues);
+ }
+
+ public void registerInterestRegex(String regex)
+ {
+ registerInterestRegex(regex, false);
+ }
+
+ public void registerInterestRegex(String regex, boolean isDurable) {
+ registerInterestRegex(regex, InterestResultPolicy.DEFAULT, isDurable, true);
+ }
+ public void registerInterestRegex(String regex, boolean isDurable,
+ boolean receiveValues)
+ {
+ registerInterestRegex(regex, InterestResultPolicy.DEFAULT, isDurable, receiveValues);
+ }
+ public void registerInterestRegex(String regex, InterestResultPolicy policy)
+ {
+ registerInterestRegex(regex, policy, false);
+ }
+
+ public void registerInterestRegex(String regex, InterestResultPolicy policy,
+ boolean isDurable) {
+ registerInterestRegex(regex, policy, isDurable, true);
+ }
+ public void registerInterestRegex(String regex, InterestResultPolicy policy,
+ boolean isDurable, boolean receiveValues)
+ {
+ processSingleInterest(regex, InterestType.REGULAR_EXPRESSION, policy,
+ isDurable, !receiveValues);
+ }
+ public void registerInterestFilter(String className)
+ {
+ registerInterestFilter(className, false);
+ }
+ public void registerInterestFilter(String className, boolean isDurable) {
+ registerInterestFilter(className, isDurable, true);
+ }
+ public void registerInterestFilter(String className, boolean isDurable,
+ boolean receiveValues)
+ {
+ processSingleInterest(className, InterestType.FILTER_CLASS,
+ InterestResultPolicy.DEFAULT, isDurable, !receiveValues);
+ }
+ public void registerInterestOQL(String query)
+ {
+ registerInterestOQL(query, false);
+ }
+ public void registerInterestOQL(String query, boolean isDurable) {
+ registerInterestOQL(query, isDurable, true);
+ }
+ public void registerInterestOQL(String query, boolean isDurable,
+ boolean receiveValues)
+ {
+ processSingleInterest(query, InterestType.OQL_QUERY,
+ InterestResultPolicy.DEFAULT, isDurable, !receiveValues);
+ }
+
+ public void unregisterInterest(Object key)
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ // Keep support for "ALL_KEYS" in 4.2.x
+ if (key instanceof String && key.equals("ALL_KEYS")) {
+ proxy.unregisterInterest(".*", InterestType.REGULAR_EXPRESSION, false, false);
+ }
+ else if (key instanceof List) {
+ proxy.unregisterInterestList((List)key, false, false);
+ }
+ else {
+ proxy.unregisterInterest(key, InterestType.KEY, false, false);
+ }
+ }
+ else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_UNREGISTRATION_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public void unregisterInterestRegex(String regex)
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ proxy.unregisterInterest(regex, InterestType.REGULAR_EXPRESSION, false, false);
+ }
+ else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_UNREGISTRATION_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public void unregisterInterestFilter(String className) {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ proxy.unregisterInterest(className, InterestType.FILTER_CLASS, false, false);
+ }
+ else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_UNREGISTRATION_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public void unregisterInterestOQL(String query)
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ proxy.unregisterInterest(query, InterestType.OQL_QUERY, false, false);
+ }
+ else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_UNREGISTRATION_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public List getInterestList()
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ return proxy.getInterestList(InterestType.KEY);
+ }
+ else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_INTEREST_UNREGISTRATION_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ /** finds the keys in this region using the given interestType and argument. Currently only
+ * InterestType.REGULAR_EXPRESSION and InterestType.KEY are supported
+ *
+ * @param interestType an InterestType value
+ * @param interestArg the associated argument (regex string, key or key list, etc)
+ * @param allowTombstones whether to return destroyed entries
+ * @return a set of the keys matching the given criterion
+ */
+ public Set getKeysWithInterest(int interestType, Object interestArg, boolean allowTombstones)
+ {
+ Set ret = null;
+ if (interestType == InterestType.REGULAR_EXPRESSION) {
+ if (interestArg == null || ".*".equals(interestArg)) {
+ ret = new HashSet(keySet(allowTombstones));
+ }
+ else {
+ ret = new HashSet();
+ // Handle the regex pattern
+ if (!(interestArg instanceof String)) {
+ throw new IllegalArgumentException(LocalizedStrings.AbstractRegion_REGULAR_EXPRESSION_ARGUMENT_WAS_NOT_A_STRING.toLocalizedString());
+ }
+ Pattern keyPattern = Pattern.compile((String)interestArg);
+ for (Iterator it = this.keySet(allowTombstones).iterator(); it.hasNext();) {
+ Object entryKey = it.next();
+ if(!(entryKey instanceof String)) {
+ //key is not a String, cannot apply regex to this entry
+ continue;
+ }
+ if(!keyPattern.matcher((String) entryKey).matches()) {
+ //key does not match the regex, this entry should not be returned.
+ continue;
+ }
+ ret.add(entryKey);
+ }
+ }
+ }
+ else if (interestType == InterestType.KEY) {
+ if (interestArg instanceof List) {
+ ret = new HashSet(); // TODO optimize initial size
+ List keyList = (List)interestArg;
+ for (Iterator it = keyList.iterator(); it.hasNext();) {
+ Object entryKey = it.next();
+ if (this.containsKey(entryKey) || (allowTombstones && this.containsTombstone(entryKey))) {
+ ret.add(entryKey);
+ }
+ }
+ }
+ else {
+ ret = new HashSet();
+ if (this.containsKey(interestArg) || (allowTombstones && this.containsTombstone(interestArg))) {
+ ret.add(interestArg);
+ }
+ }
+ }
+ else if (interestType == InterestType.FILTER_CLASS) {
+ throw new UnsupportedOperationException(LocalizedStrings.AbstractRegion_INTERESTTYPEFILTER_CLASS_NOT_YET_SUPPORTED.toLocalizedString());
+ }
+ else if (interestType == InterestType.OQL_QUERY) {
+ throw new UnsupportedOperationException(LocalizedStrings.AbstractRegion_INTERESTTYPEOQL_QUERY_NOT_YET_SUPPORTED.toLocalizedString());
+ }
+ else {
+ throw new IllegalArgumentException(LocalizedStrings.AbstractRegion_UNSUPPORTED_INTEREST_TYPE_0.toLocalizedString(Integer.valueOf(interestType)));
+ }
+ return ret;
+ }
+
+ public List getInterestListRegex()
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ return proxy.getInterestList(InterestType.REGULAR_EXPRESSION);
+ }
+ else {
+ throw new UnsupportedOperationException( LocalizedStrings.LocalRegion_INTEREST_LIST_RETRIEVAL_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public List getInterestListFilters()
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ return proxy.getInterestList(InterestType.FILTER_CLASS);
+ }
+ else {
+ throw new UnsupportedOperationException( LocalizedStrings.LocalRegion_INTEREST_LIST_RETRIEVAL_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public List getInterestListOQL()
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ return proxy.getInterestList(InterestType.OQL_QUERY);
+ }
+ else {
+ throw new UnsupportedOperationException( LocalizedStrings.LocalRegion_INTEREST_LIST_RETRIEVAL_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public Set keySetOnServer() {
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ return proxy.keySet();
+ } else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_SERVER_KEYSET_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ public boolean containsKeyOnServer(Object key) {
+ checkReadiness();
+ checkForNoAccess();
+ ServerRegionProxy proxy = getServerProxy();
+ if (proxy != null) {
+ return proxy.containsKey(key);
+ } else {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_SERVER_KEYSET_REQUIRES_A_POOL.toLocalizedString());
+ }
+ }
+
+ /**
+ * WARNING: this method is overridden in subclasses.
+ *
+ * @param key
+ * @see DistributedRegion#localDestroyNoCallbacks(Object)
+ */
+ protected void localDestroyNoCallbacks(Object key)
+ {
+ if(logger.isDebugEnabled()) {
+ logger.debug("localDestroyNoCallbacks key={}", key);
+ }
+ checkReadiness();
+ validateKey(key);
+ EntryEventImpl event = EntryEventImpl.create(this, Operation.LOCAL_DESTROY,
+ key, false, getMyId(), false /* generateCallbacks */, true);
+ try {
+ basicDestroy(event,
+ false,
+ null); // expectedOldValue
+ }
+ catch (CacheWriterException e) {
+ // cache writer not called
+ throw new Error(LocalizedStrings.LocalRegion_CACHE_WRITER_SHOULD_NOT_HAVE_BEEN_CALLED_FOR_LOCALDESTROY.toLocalizedString(), e);
+ }
+ catch (TimeoutException e) {
+ // no distributed lock
+ throw new Error(LocalizedStrings.LocalRegion_NO_DISTRIBUTED_LOCK_SHOULD_HAVE_BEEN_ATTEMPTED_FOR_LOCALDESTROY.toLocalizedString(), e);
+ }
+ catch (EntryNotFoundException e) {
+ // not a problem
+ } finally {
+ event.release();
+ }
+ }
+
+ /**
+ * Do localDestroy on a list of keys, if they exist
+ *
+ * @param keys
+ * the list of arrays of keys to invalidate
+ * @see #registerInterest(Object)
+ */
+ private void clearViaList(List keys)
+ {
+ for (Iterator it = this.entries(false).iterator(); it.hasNext();) {
+ Region.Entry entry = (Region.Entry)it.next();
+ try {
+ Object entryKey = entry.getKey();
+ boolean match = false;
+ for (Iterator it2 = keys.iterator(); it2.hasNext();) {
+ Object k = it2.next();
+ if (entryKey.equals(k)) {
+ match = true;
+ break;
+ }
+ } // for
+ if (!match) {
+ continue;
+ }
+ localDestroyNoCallbacks(entryKey);
+ }
+ catch (EntryDestroyedException ignore) {
+ // ignore to fix bug 35534
+ }
+ }
+ }
+
+ /**
+ * do a localDestroy on all matching keys
+ *
+ * @param key
+ * the regular expression to match on
+ * @see #registerInterestRegex(String)
+ */
+ private void clearViaRegEx(String key)
+ {
+ // @todo: if (key.equals(".*)) then cmnClearRegionNoCallbacks
+ Pattern keyPattern = Pattern.compile(key);
+ for (Iterator it = this.entries(false).iterator(); it.hasNext();) {
+ Region.Entry entry = (Region.Entry)it.next();
+ try {
+ Object entryKey = entry.getKey();
+ if (!(entryKey instanceof String))
+ continue;
+ if (!keyPattern.matcher((String)entryKey).matches()) {
+ //key does not match the regex, this entry should not be returned.
+ continue;
+ }
+ localDestroyNoCallbacks(entryKey);
+ }
+ catch (EntryDestroyedException ignore) {
+ // ignore to fix bug 35534
+ }
+ }
+ }
+
+ /**
+ * do a localDestroy on all matching keys
+ *
+ * @param key
+ * the regular expression to match on
+ * @see #registerInterestFilter(String)
+ */
+ private void clearViaFilterClass(String key)
+ {
+ Class filterClass;
+ InterestFilter filter;
+ try {
+ filterClass = ClassLoadUtil.classFromName(key);
+ filter = (InterestFilter)filterClass.newInstance();
+ }
+ catch (ClassNotFoundException cnfe) {
+ throw new RuntimeException(LocalizedStrings.LocalRegion_CLASS_0_NOT_FOUND_IN_CLASSPATH.toLocalizedString(key), cnfe);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(LocalizedStrings.LocalRegion_CLASS_0_COULD_NOT_BE_INSTANTIATED.toLocalizedString(key), e);
+ }
+
+ for (Iterator it = this.entries(false).iterator(); it.hasNext();) {
+ Region.Entry entry = (Region.Entry)it.next();
+ try {
+ Object entryKey = entry.getKey();
+ if (!(entryKey instanceof String))
+ continue;
+ InterestEvent e = new InterestEvent(entryKey, entry.getValue(), true);
+ if (!filter.notifyOnRegister(e)) {
+ //the filter does not want to know about this entry, so skip it.
+ continue;
+ }
+ localDestroyNoCallbacks(entryKey);
+ }
+ catch (EntryDestroyedException ignore) {
+ // ignore to fix bug 35534
+ }
+ }
+ }
+
+ /**
+ * Do a localDestroy of all matching keys
+ *
+ * @param query
+ * @see #registerInterestOQL(String)
+ */
+ private void clearViaQuery(String query)
+ {
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_NOT_YET_SUPPORTED.toLocalizedString());
+ }
+
+ /**
+ * Refresh local entries based on server's list of keys
+ * @param serverKeys
+ */
+ public void refreshEntriesFromServerKeys(Connection con, List serverKeys,
+ InterestResultPolicy pol)
+ {
+ ServerRegionProxy proxy = getServerProxy();
+ if (logger.isDebugEnabled()) {
+ logKeys(serverKeys, pol);
+ }
+
+ if (pol == InterestResultPolicy.NONE) {
+ return; // done
+ }
+
+ if(logger.isDebugEnabled()) {
+ logger.debug("refreshEntries region={}", getFullPath());
+ }
+ for (Iterator it = serverKeys.iterator(); it.hasNext();) {
+ ArrayList keysList = (ArrayList)it.next();
+ // The chunk can contain null data if there are no entries on the server
+ // corresponding to the requested keys
+ if (keysList == null) {
+ continue;
+ }
+ // int numberOfResults = keysList.size();
+ if(EntryLogger.isEnabled()) {
+ if(con != null) {
+ Endpoint endpoint = con.getEndpoint();
+ if(endpoint != null) {
+ EntryLogger.setSource(endpoint.getMemberId(), "RIGII");
+ }
+ }
+ }
+ try {
+ ArrayList list = new ArrayList(keysList);
+ if (pol != InterestResultPolicy.KEYS_VALUES) {
+ for (Iterator it2 = keysList.iterator(); it2.hasNext();) {
+ Object currentKey = it2.next();
+ // Dont apply riResponse if the entry was destroyed when
+ // ri is in progress
+ if (currentKey == null || getImageState().hasDestroyedEntry(currentKey)){
+ list.remove(currentKey);
+ }
+ }
+ }
+ if (pol == InterestResultPolicy.KEYS) {
+ // Attempt to create an invalid in without overwriting
+ if (!isProxy()) {
+ for (Iterator it2 = list.iterator(); it2.hasNext();) {
+ Object currentKey = it2.next();
+ entries.initialImagePut(currentKey, 0, Token.LOCAL_INVALID, false,
+ false, null, null, false);
+
+ }
+ }
+ // Size statistics don't take key into account, so we don't
+ // need to modify the region's size.
+ }
+ else if(!list.isEmpty()) {
+ Assert.assertTrue(pol == InterestResultPolicy.KEYS_VALUES);
+// for (Iterator it2 = list.iterator(); it2.hasNext();) {
+// Object currentKey = ((Entry)it2.next()).getKey();
+// localDestroyNoCallbacks(currentKey);
+// }
+ VersionedObjectList values = (VersionedObjectList)list.get(0); // proxy.getAllOnPrimaryForRegisterInterest(con, list);
+ if(logger.isDebugEnabled()) {
+ logger.debug("processing interest response: {}", values.size());
+ }
+ VersionedObjectList.Iterator listIt = values.iterator();
+ while (listIt.hasNext()) {
+ VersionedObjectList.Entry entry = listIt.next();
+ Object currentKey = entry.getKey();
+ if (currentKey == null || getImageState().hasDestroyedEntry(currentKey)) {
+ continue;
+ }
+ Object val = entry.getObject();
+ boolean isBytes = entry.isBytes();
+ boolean isKeyOnServer = !entry.isKeyNotOnServer();
+ boolean isTombstone = this.concurrencyChecksEnabled
+ && entry.isKeyNotOnServer()
+ && (entry.getVersionTag() != null);
+ final VersionTag tag = entry.getVersionTag();
+ if (val instanceof Throwable) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_CAUGHT_THE_FOLLOWING_EXCEPTION_FOR_KEY_0_WHILE_PERFORMING_A_REMOTE_GETALL,
+ currentKey), (Throwable)val);
+ localDestroyNoCallbacks(currentKey);
+ continue;
+ } else {
+ if(logger.isDebugEnabled()) {
+ logger.debug("refreshEntries key={} value={} version={}", currentKey, entry, tag);
+ }
+ if (tag == null) { // no version checks
+ localDestroyNoCallbacks(currentKey);
+ }
+ }
+
+ if(val instanceof byte[] && !isBytes) {
+ val = CachedDeserializableFactory.create((byte[]) val);
+ }
+
+ if (isTombstone) {
+ assert val == null : "server returned a value for a destroyed entry";
+ val = Token.TOMBSTONE;
+ }
+
+ if (val != null || isTombstone) {
+ // Sneakily drop in the value into our local cache,
+ // but don't overwrite
+ if (!isProxy()) {
+ entries.initialImagePut(currentKey, 0, val, false, false, tag, null, false);
+ }
+ }
+ else {
+ RegionEntry re = entries.getEntry(currentKey);
+ if (!isProxy() && isKeyOnServer) {
+ entries.initialImagePut(currentKey, 0, Token.LOCAL_INVALID,
+ false,false, tag, null, false);
+ }
+ else {
+ if (re != null) {
+ synchronized (re) {
+ if (re.isDestroyedOrRemovedButNotTombstone()) {
+ entries.removeEntry(currentKey, re, false);
+ }
+ }
+ }
+ }
+ // In this case, if we didn't overwrite, we don't have a local
+ // value, so no size change needs to be recorded.
+ }
+ }
+ }
+ }
+ catch (DiskAccessException dae) {
+ this.handleDiskAccessException(dae);
+ throw dae;
+ } finally {
+ EntryLogger.clearSource();
+ }
+ } // for
+ }
+
+ private void logKeys(List serverKeys, InterestResultPolicy pol)
+ {
+ int totalKeys = 0;
+ StringBuffer buffer = new StringBuffer();
+ for (Iterator it = serverKeys.iterator(); it.hasNext();) {
+ ArrayList keysList = (ArrayList)it.next();
+ // The chunk can contain null data if there are no entries on the server
+ // corresponding to the requested keys
+ // TODO is this still possible?
+ if (keysList == null)
+ continue;
+ int numThisChunk = keysList.size();
+ totalKeys += numThisChunk;
+ for (Iterator it2 = keysList.iterator(); it2.hasNext();) {
+ Object key = it2.next();
+ if (key != null) {
+ if (key instanceof VersionedObjectList) {
+ Set keys = ((VersionedObjectList)key).keySet();
+ for (Object k : keys) {
+ buffer.append(" " + k).append("\n");
+ }
+ } else {
+ buffer.append(" " + key).append("\n");
+ }
+ }
+ }
+ } // for
+ if(logger.isDebugEnabled()) {
+ logger.debug("{} refreshEntriesFromServerKeys count={} policy={}\n{}", this, totalKeys, pol, buffer);
+ }
+ }
+ /**
+ * Remove values in local cache before registering interest
+ *
+ * @param key
+ * the interest key
+ * @param interestType
+ * the interest type from {@link InterestType}
+ * @param pol
+ * the policy from {@link InterestResultPolicy}
+ */
+ public void clearKeysOfInterest(Object key, int interestType,
+ InterestResultPolicy pol)
+ {
+ switch (interestType) {
+ case InterestType.FILTER_CLASS:
+ clearViaFilterClass((String)key);
+ break;
+ case InterestType.KEY:
+ if (key instanceof String && key.equals("ALL_KEYS"))
+ clearViaRegEx(".*");
+ else if (key instanceof List)
+ clearViaList((List)key);
+ else
+ localDestroyNoCallbacks(key);
+ break;
+ case InterestType.OQL_QUERY:
+ clearViaQuery((String)key);
+ break;
+ case InterestType.REGULAR_EXPRESSION:
+ clearViaRegEx((String)key);
+ break;
+ default:
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_UNKNOWN_INTEREST_TYPE.toLocalizedString());
+ }
+ }
+
+ //////////////////// Package Methods ////////////////////////////////////////
+
+ /**
+ * Destroys and recreates this region. If this is triggered by loadSnapshot
+ * inputStream will be supplied. If this is triggered by LossAction of
+ * reinitialize then inputStream will be null, and the region will go through
+ * regular GetInitalImage if it is a mirrored replicate.
+ *
+ * Acquires and releases the DestroyLock.
+ *
+ * @since 5.0
+ */
+ void reinitialize(InputStream inputStream, RegionEventImpl event)
+ throws TimeoutException, IOException, ClassNotFoundException
+ {
+ acquireDestroyLock();
+ try {
+ reinitialize_destroy(event);
+ recreate(inputStream, null);
+ }
+ finally {
+ releaseDestroyLock();
+ }
+ }
+
+ /** must be holding destroy lock */
+ void reinitializeFromImageTarget(InternalDistributedMember imageTarget)
+ throws TimeoutException, IOException, ClassNotFoundException
+ {
+ Assert.assertTrue(imageTarget != null);
+ recreate(null, imageTarget);
+ }
+
+ /**
+ * Returns true if this region was reinitialized, e.g. a snapshot was loaded,
+ * and this is the recreated region
+ */
+ boolean reinitialized_new()
+ {
+ return this.reinitialized_new;
+ }
+
+ /** must be holding destroy lock */
+ void reinitialize_destroy(RegionEventImpl event) throws CacheWriterException,
+ TimeoutException
+ {
+ final boolean cacheWrite = !event.originRemote;
+ // register this region as reinitializing
+ this.cache.regionReinitializing(getFullPath());
+ basicDestroyRegion(event, cacheWrite, false/* lock */, true);
+ }
+
+ /** must be holding destroy lock */
+ private void recreate(InputStream inputStream,
+ InternalDistributedMember imageTarget) throws TimeoutException,
+ IOException, ClassNotFoundException
+ {
+ String thePath = getFullPath();
+ Region newRegion = null;
+ // recreate new region with snapshot data
+
+ try {
+ LocalRegion parent = this.parentRegion;
+ boolean getDestroyLock = false;
+ // If specified diskDir in DEFAULT diskstore, we should not use null
+ // as diskstore name any more
+ if (this.dsi!=null && this.dsi.getName().equals(DiskStoreFactory.DEFAULT_DISK_STORE_NAME)
+ && this.diskStoreName == null && !useDefaultDiskStore()) {
+ this.diskStoreName = this.dsi.getName();
+ }
+ RegionAttributes attrs = this;
+ InternalRegionArguments iargs = new InternalRegionArguments()
+ .setDestroyLockFlag(getDestroyLock)
+ .setSnapshotInputStream(inputStream)
+ .setImageTarget(imageTarget)
+ .setRecreateFlag(true);
+ if (this instanceof BucketRegion) {
+ BucketRegion me = (BucketRegion) this;
+ iargs.setPartitionedRegionBucketRedundancy(me.getRedundancyLevel());
+ }
+
+ if (parent == null) {
+ newRegion = this.cache.createVMRegion(this.regionName, attrs, iargs);
+ }
+ else {
+ newRegion = parent.createSubregion(this.regionName, attrs, iargs);
+ }
+
+ // note that createVMRegion and createSubregion now call
+ // regionReinitialized
+ }
+ catch (RegionExistsException e) {
+ // shouldn't happen since we're holding the destroy lock
+ InternalGemFireError error = new InternalGemFireError(LocalizedStrings.LocalRegion_GOT_REGIONEXISTSEXCEPTION_IN_REINITIALIZE_WHEN_HOLDING_DESTROY_LOCK.toLocalizedString());
+ error.initCause(e);
+ throw error;
+ }
+ finally {
+ if (newRegion == null) { // failed to create region
+ this.cache.unregisterReinitializingRegion(thePath);
+ }
+ }
+ }
+
+ void loadSnapshotDuringInitialization(InputStream inputStream)
+ throws IOException, ClassNotFoundException
+ {
+ DataInputStream in = new DataInputStream(inputStream);
+ try {
+ RegionMap map = getRegionMap();
+ byte snapshotVersion = in.readByte();
+ if (snapshotVersion != SNAPSHOT_VERSION) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_UNSUPPORTED_SNAPSHOT_VERSION_0_ONLY_VERSION_1_IS_SUPPORTED.toLocalizedString(new Object[] {Byte.valueOf(snapshotVersion), Byte.valueOf(SNAPSHOT_VERSION)}));
+ }
+ for (;;) {
+ Object key = DataSerializer.readObject(in);
+ if (key == null)
+ break;
+ byte b = in.readByte();
+ Object value;
+
+ if (b == SNAPSHOT_VALUE_OBJ) {
+ value = DataSerializer.readObject(in);
+ }
+ else if (b == SNAPSHOT_VALUE_INVALID) {
+ // Even though it was a distributed invalidate when the
+ // snapshot was created I think it is correct to turn it
+ // into a local invalidate when we load the snapshot since
+ // we don't do a distributed invalidate operation when loading.
+ value = Token.LOCAL_INVALID;
+ }
+ else if (b == SNAPSHOT_VALUE_LOCAL_INVALID) {
+ value = Token.LOCAL_INVALID;
+ }
+ else {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_UNEXPECTED_SNAPSHOT_CODE_0_THIS_SNAPSHOT_WAS_PROBABLY_WRITTEN_BY_AN_EARLIER_INCOMPATIBLE_RELEASE.toLocalizedString(new Byte(b)));
+ }
+
+ //If versioning is enabled, we will give the entry a "fake"
+ //version.
+ VersionTag tag = null;
+ if(this.concurrencyChecksEnabled) {
+ tag = VersionTag.create(getVersionMember());
+ }
+ map.initialImagePut(key, cacheTimeMillis(), value, false,
+ false, tag, null, false);
+ }
+ }
+ finally {
+ in.close();
+ }
+ this.reinitialized_new = true;
+ }
+
+ /** Package helper method */
+ @Retained
+ Object getEntryValue(RegionEntry entry)
+ {
+ if (entry == null) {
+ return null;
+ }
+
+ try {
+ return entry.getValue(this);
+ }catch(DiskAccessException dae) {
+ this.handleDiskAccessException(dae);
+ throw dae;
+ }
+
+ }
+
+ /**
+ * Blocks until initialization is complete.
+ *
+ * @param destroyedRegionOk
+ * true if it is okay to return a region that isDestroyed
+ * @see DestroyRegionOperation
+ */
+ Region getSubregion(String path, boolean destroyedRegionOk)
+ {
+ if (destroyedRegionOk) {
+ checkCacheClosed();
+ }
+ else if (isDestroyed()) {
+ // Assume if the owner of the subregion is destroyed, so are all of its
+ // subregions
+ return null;
+ }
+ if (path == null) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_PATH_SHOULD_NOT_BE_NULL.toLocalizedString());
+ }
+ if (path.length() == 0) {
+ waitOnInitialization(); // some internal methods rely on this
+ return this;
+ }
+
+ if (path.charAt(0) == SEPARATOR_CHAR)
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_PATH_SHOULD_NOT_START_WITH_A_SLASH.toLocalizedString());
+
+ int sep_idx; // the index of the next separator
+ // initialize the current region as this one
+ LocalRegion r = this;
+ // initialize the rest of the name to be regionName
+ String n = path;
+ String next; // the next part of the path
+ boolean last; // last: are we on the last part of the path?
+ do {
+ // if the rest of the name is empty, then we're done, return
+ // current region
+ if (n.length() == 0) {
+ break; // return r
+ }
+ sep_idx = n.indexOf(SEPARATOR_CHAR);
+ last = sep_idx < 0; // this is the last part if no separator
+ // try to get next region
+ next = last ? n : n.substring(0, sep_idx);
+ r = r.basicGetSubregion(next);
+ if (r == null) {
+ // not found
+ return null;
+ }
+ if (r.isDestroyed() && !destroyedRegionOk) {
+ return null;
+ }
+ if (!last) // if found but still more to do, get next rest of path
+ n = n.substring(sep_idx + 1);
+ } while (!last);
+
+ r.waitOnInitialization();
+
+ // if region has just been destroyed return null unless specified not to
+ if (r.isDestroyed()) {
+ if (!destroyedRegionOk) {
+ return null;
+ }
+ return r;
+ }
+
+ return r;
+ }
+
+ /**
+ * Called by a thread that is doing region initialization. Causes the
+ * initialization Latch to be bypassed by this thread.
+ *
+ * @return oldLevel
+ */
+ public static int setThreadInitLevelRequirement(int level)
+ {
+ int oldLevel = threadInitLevelRequirement();
+ if (level == AFTER_INITIAL_IMAGE) { // if setting to default, just reset
+ initializationThread.set(null);
+ }
+ else {
+ initializationThread.set(Integer.valueOf(level));
+ }
+ return oldLevel;
+ }
+
+ /**
+ * Return the access level this thread has for regions with respect to how
+ * initialized they need to be before this thread can have a reference to it.
+ * AFTER_INITIAL_IMAGE: Must be fully initialized (the default)
+ * BEFORE_INITIAL_IMAGE: Must have had first latch opened ANY_INIT: Thread
+ * uses region as soon as possible
+ */
+ public static int threadInitLevelRequirement()
+ {
+ Integer initLevel = (Integer)initializationThread.get();
+ if (initLevel == null) {
+ return AFTER_INITIAL_IMAGE;
+ }
+ return initLevel.intValue();
+ }
+
+ public boolean checkForInitialization() {
+ if (this.initialized) {
+ return true;
+ }
+ switch (threadInitLevelRequirement()) {
+ case AFTER_INITIAL_IMAGE:
+ return checkForInitialization(this.initializationLatchAfterGetInitialImage);
+ case BEFORE_INITIAL_IMAGE:
+ return checkForInitialization(this.initializationLatchBeforeGetInitialImage);
+ case ANY_INIT:
+ return true;
+ default:
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_UNEXPECTED_THREADINITLEVELREQUIREMENT.toLocalizedString());
+ }
+ }
+
+ private boolean checkForInitialization(
+ StoppableCountDownLatch latch) {
+ return latch.getCount() == 0;
+ }
+
+ /** wait on the initialization Latch based on thread requirements */
+ public void waitOnInitialization()
+ {
+ if (this.initialized) {
+ return;
+ }
+ switch (threadInitLevelRequirement()) {
+ case AFTER_INITIAL_IMAGE:
+ waitOnInitialization(this.initializationLatchAfterGetInitialImage);
+ break;
+ case BEFORE_INITIAL_IMAGE:
+ waitOnInitialization(this.initializationLatchBeforeGetInitialImage);
+ break;
+ case ANY_INIT:
+ return;
+ default:
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_UNEXPECTED_THREADINITLEVELREQUIREMENT.toLocalizedString());
+ }
+ }
+
+ protected void waitOnInitialization(StoppableCountDownLatch latch)
+ {
+ if (latch == null)
+ return; // latch resource has been freed
+
+ while (true) {
+ cache.getCancelCriterion().checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ latch.await();
+ break;
+ }
+ catch (InterruptedException e) {
+ interrupted = true;
+ cache.getCancelCriterion().checkCancelInProgress(e);
+ // continue waiting
+ }
+ finally {
+ if (interrupted) // set interrupted flag if was interrupted
+ Thread.currentThread().interrupt();
+ }
+ } // while
+ }
+
+ /**
+ * Wait until data is ready in this region
+ */
+ public void waitForData() {
+ if (this.initialized) {
+ return;
+ }
+ waitOnInitialization(this.initializationLatchAfterGetInitialImage);
+ }
+
+ /** return null if not found */
+ @Override
+ public RegionEntry basicGetEntry(Object key)
+ {
+ // ok to ignore tx state; all callers are non-transactional
+ RegionEntry re = this.entries.getEntry(key);
+ if (re != null && re.isRemoved()) {
+ re = null;
+ }
+ return re;
+ }
+
+ /**
+ * Return true if invalidation occurred; false if it did not, for example if
+ * it was already invalidated
+ *
+ * @see DistributedRegion#basicInvalidate(EntryEventImpl)
+ */
+ void basicInvalidate(EntryEventImpl event) throws EntryNotFoundException
+ {
+ basicInvalidate(event, isInitialized()/* for bug 35214 */);
+ }
+
+ /**
+ * Used by disk regions when recovering data from backup. Currently this "put"
+ * is done at a very low level to keep it from generating events or pushing
+ * updates to others.
+ */
+ public DiskEntry initializeRecoveredEntry(Object key, DiskEntry.RecoveredEntry value)
+ {
+ Assert.assertTrue(this.diskRegion != null);
+ // region operation so it is ok to ignore tx state
+ RegionEntry re = this.entries.initRecoveredEntry(key, value);
+ if (re == null) {
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_ENTRY_ALREADY_EXISTED_0.toLocalizedString(key));
+ }
+ return (DiskEntry)re;
+ }
+ /**
+ * Used by disk regions when recovering data from backup and
+ * initializedRecoveredEntry has already been called for the given key.
+ * Currently this "put"
+ * is done at a very low level to keep it from generating events or pushing
+ * updates to others.
+ */
+ public DiskEntry updateRecoveredEntry(Object key, DiskEntry.RecoveredEntry value)
+ {
+ Assert.assertTrue(this.diskRegion != null);
+ // region operation so it is ok to ignore tx state
+ RegionEntry re = this.entries.updateRecoveredEntry(key, value);
+ return (DiskEntry)re;
+ }
+
+ public void copyRecoveredEntries(RegionMap rm) {
+ this.entries.copyRecoveredEntries(rm);
+ }
+
+ public void recordRecoveredGCVersion(VersionSource member, long gcVersion) {
+ //TODO - RVV - I'm not sure about this recordGCVersion method. It seems
+ //like it's not doing the right thing if the current member is the member
+ //we just recovered.
+ //We need to update the RVV in memory
+ this.versionVector.recordGCVersion(member, gcVersion);
+
+ //We also need to update the RVV that represents what we have persisted on disk
+ DiskRegion dr = this.getDiskRegion();
+ if(dr != null) {
+ dr.recordRecoveredGCVersion(member, gcVersion);
+ }
+ }
+
+ public void recordRecoveredVersonHolder(VersionSource member,
+ RegionVersionHolder versionHolder, boolean latestOplog) {
+ if(this.concurrencyChecksEnabled) {
+ //We need to update the RVV in memory
+ this.versionVector.initRecoveredVersion(member, versionHolder, latestOplog);
+ DiskRegion dr = this.getDiskRegion();
+ //We also need to update the RVV that represents what we have persisted on disk
+ if(dr != null) {
+ dr.recordRecoveredVersonHolder(member, versionHolder, latestOplog);
+ }
+ }
+ }
+
+ @Override
+ public void recordRecoveredVersionTag(VersionTag tag) {
+ if(this.concurrencyChecksEnabled) {
+ this.versionVector.recordVersion(tag.getMemberID(), tag.getRegionVersion());
+ DiskRegion dr = this.getDiskRegion();
+ //We also need to update the RVV that represents what we have persisted on disk
+ if(dr != null) {
+ dr.recordRecoveredVersionTag(tag);
+ }
+ }
+ }
+
+
+
+ @Override
+ public void setRVVTrusted(boolean rvvTrusted) {
+ if(this.concurrencyChecksEnabled) {
+ DiskRegion dr = this.getDiskRegion();
+ //Update whether or not the RVV we have recovered is trusted (accurately
+ //represents what we have on disk).
+ if(dr != null) {
+ dr.setRVVTrusted(rvvTrusted);
+ }
+ }
+ }
+
+ /**
+ * Get the best iterator for the region entries.
+ *
+ * TODO there has been some work on this on the sqlfire branch that should
+ * be picked up here.
+ */
+ public Iterator getBestIterator(boolean includeValues) {
+ if(this instanceof DistributedRegion) {
+ return ((DistributedRegion)this).getBestIterator(includeValues);
+ }
+
+ return this.entries.regionEntries().iterator();
+ }
+
+ /**
+ * Fix up our RVV by iterating over the entries in the region
+ * and making sure they are applied to the RVV.
+ *
+ * If we failed to do a GII, we may have applied the RVV from a remote member.
+ * That RVV may not have seen some of the events in our local RVV. Those
+ * entries were supposed to be replaced with the results of the GII. However,
+ * if we failed the GII, those entries may still be in the cache, but are
+ * no longer reflected in the local RVV. This method iterates over those
+ * keys and makes sure their versions are applied to the local RVV.
+ *
+ * TODO - this method should probably rebuild the RVV from scratch, instead
+ * of starting with the existing RVV. By starting with the existing RVV, we
+ * may claim to have entries that we actually don't have. Unfortunately, we
+ * can't really rebuild the RVV from scratch because we will end up with
+ * huge exception lists.
+ *
+ * However, if we are in the state of recovering from disk with an untrusted
+ * RVV, we must be newer than any other surviving members. So they shouldn't
+ * have any entries in their cache that match entries that we failed to receive
+ * through the GII but are reflected in our current RVV. So it should be
+ * safe to start with the current RVV.
+ *
+ */
+ void repairRVV() {
+ RegionVersionVector rvv = this.getVersionVector();
+
+ if(rvv == null) {
+ //No need to do anything.
+ return;
+ }
+
+ Iterator it = getBestIterator(false);
+ int count = 0;
+ VersionSource> myId = this.getVersionMember();
+ //Iterate over the all of the entries
+ while (it.hasNext()) {
+ RegionEntry mapEntry = it.next();
+ VersionStamp> stamp = mapEntry.getVersionStamp();
+ VersionSource> id = stamp.getMemberID();
+ if (id == null) {
+ id = myId;
+ }
+ //Make sure the version is applied to the regions RVV
+ rvv.recordVersion(id, stamp.getRegionVersion());
+
+ }
+ }
+
+ /**
+ * Return true if invalidation occurred; false if it did not, for example if
+ * it was already invalidated
+ */
+ private void basicInvalidate(final EntryEventImpl event,
+ boolean invokeCallbacks) throws EntryNotFoundException
+ {
+ basicInvalidate(event, invokeCallbacks, false);
+ }
+
+ /**
+ * Asif:Made this function protected as this is over ridden in HARegion to
+ * abort expiry of Events which have key as Long , if it is not able to
+ * destroy from availableIDs
+ *
+ * @param forceNewEntry
+ * true if we are a mirror and still in the initialization phase.
+ * Called from InvalidateOperation.InvalidateMessage
+
+ */
+ void basicInvalidate(final EntryEventImpl event, boolean invokeCallbacks,
+ final boolean forceNewEntry) throws EntryNotFoundException
+ {
+ if (!event.isOriginRemote() && !event.isDistributed()
+ && getScope().isDistributed() && getDataPolicy().withReplication()
+ && invokeCallbacks /*
+ * catches case where being called by (distributed)
+ * invalidateRegion
+ */) {
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_CANNOT_DO_A_LOCAL_INVALIDATE_ON_A_REPLICATED_REGION.toLocalizedString());
+ }
+
+ if (hasSeenEvent(event)) {
+ if(logger.isTraceEnabled(LogMarker.DM)) {
+ logger.trace(LogMarker.DM, "LR.basicInvalidate: this cache has already seen this event {}", event);
+ }
+ if (this.concurrencyChecksEnabled && event.getVersionTag() != null && !event.getVersionTag().isRecorded()) {
+ getVersionVector().recordVersion((InternalDistributedMember) event.getDistributedMember(), event.getVersionTag());
+ }
+ return;
+ }
+
+ discoverJTA();
+ getDataView().invalidateExistingEntry(event, invokeCallbacks, forceNewEntry);
+ }
+
+ void basicInvalidatePart2(RegionEntry re, EntryEventImpl event,
+ boolean conflictwithClear, boolean invokeCallbacks)
+ {
+ updateStatsForInvalidate();
+
+ if (invokeCallbacks) {
+ try {
+ re.dispatchListenerEvents(event);
+ }
+ catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ stopper.checkCancelInProgress(null);
+ return;
+ }
+ }
+ else {
+ event.callbacksInvoked(true);
+ }
+ }
+
+ /**
+ * Update stats
+ */
+ private void updateStatsForInvalidate() {
+ getCachePerfStats().incInvalidates();
+ }
+
+ void basicInvalidatePart3(RegionEntry re, EntryEventImpl event,
+ boolean invokeCallbacks) {
+ // No op. overriden by sub classes.
+ // Dispatching listener events moved to basic*Part2.
+ }
+
+ /**
+ * invoke callbacks for an invalidation
+ */
+ public void invokeInvalidateCallbacks(final EnumListenerEvent eventType,
+ final EntryEventImpl event, final boolean callDispatchListenerEvent) {
+ // Notify bridge clients (if this is a BridgeServer)
+ event.setEventType(eventType);
+ notifyBridgeClients(event);
+ if (this.hdfsStoreName != null) {
+ notifyGatewaySender(eventType, event);
+ }
+ if(callDispatchListenerEvent){
+ dispatchListenerEvent(eventType, event);
+ }
+ }
+
+ /**
+ * @param key
+ * the key of the entry to invalidate
+ * @param newValue
+ * the new value of the entry
+ * @param didDestroy
+ * true if tx destroyed this entry at some point
+ * @param rmtOrigin
+ * true if transaction being applied had a remote origin
+ * @param event
+ * filled in if operation performed
+ * @param filterRoutingInfo
+ * @param bridgeContext
+ * @param txEntryState for passing up versionTag - only on near side
+ * @param versionTag tag generated by txCoordinator - only on far side
+ * @param tailKey tail (shadow) key generated by txCoordinator for WAN - only on farside
+ */
+ final void txApplyInvalidate(Object key, Object newValue, boolean didDestroy,
+ TransactionId rmtOrigin, TXRmtEvent event, boolean localOp,
+ EventID eventId, Object aCallbackArgument,List pendingCallbacks, FilterRoutingInfo filterRoutingInfo,
+ ClientProxyMembershipID bridgeContext, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+ this.entries.txApplyInvalidate(key, newValue, didDestroy, rmtOrigin,
+ event, localOp, eventId, aCallbackArgument,pendingCallbacks,filterRoutingInfo,bridgeContext, txEntryState, versionTag, tailKey);
+ }
+
+ /**
+ * Called by lower levels, while still holding the write sync lock, and the
+ * low level has completed its part of the basic destroy
+ */
+ final void txApplyInvalidatePart2(RegionEntry re, Object key,
+ boolean didDestroy, boolean didInvalidate, boolean clearConflict)
+ {
+ if (this.testCallable != null) {
+ this.testCallable.call(this, Operation.INVALIDATE, re);
+ }
+ if (didInvalidate) {
+ updateStatsForInvalidate();
+ // Bug 40842: clearing index of the old value
+ // performed in AbstractRegionMap
+ }
+ if (didDestroy) {
+ if (this.entryUserAttributes != null) {
+ this.entryUserAttributes.remove(key);
+ }
+ }
+ }
+
+ /**
+ * Allows null as new value to accomodate create with a null value. Assumes
+ * all key, value, and callback validations have been performed.
+ *
+ * @param event
+ * the event object for this operation, with the exception that the
+ * oldValue parameter is not yet filled in. The oldValue will be
+ * filled in by this operation.
+ *
+ * @param ifNew
+ * true if this operation must not overwrite an existing key
+ * @param ifOld
+ * true if this operation must not create a new key
+ * @param expectedOldValue
+ * only succeed if old value is equal to this value. If null,
+ * then doesn't matter what old value is. If INVALID token,
+ * must be INVALID.
+ * @param requireOldValue
+ * true if the oldValue should be set in event even if ifNew
+ * and entry exists
+ * @return false if ifNew is true and there is an existing key or
+ * if ifOld is true and expectedOldValue does not match the current
+ * value in the cache. Otherwise return true.
+ */
+ protected final boolean basicPut(EntryEventImpl event,
+ boolean ifNew,
+ boolean ifOld,
+ Object expectedOldValue,
+ boolean requireOldValue)
+ throws TimeoutException, CacheWriterException {
+ return getDataView().putEntry(event, ifNew, ifOld, expectedOldValue, requireOldValue, 0L, false);
+ }
+
+ /**
+ * @param putOp
+ * describes the operation that did the put
+ * @param key
+ * the key of the entry to put
+ * @param newValue
+ * the new value of the entry
+ * @param didDestroy
+ * true if tx destroyed this entry at some point
+ * @param rmtOrigin
+ * true if transaction being applied had a remote origin
+ * @param event
+ * filled in if operation performed
+ * @param aCallbackArgument
+ * argument passed in by user
+ * @param filterRoutingInfo
+ * @param bridgeContext
+ * @param txEntryState for passing up versionTag - only on near side
+ * @param versionTag tag generated by txCoordinator - only on far side
+ * @param tailKey tail (shadow) key generated by txCoordinator for WAN - only on farside
+ */
+ final void txApplyPut(Operation putOp, Object key, Object newValue,
+ boolean didDestroy, TransactionId rmtOrigin, TXRmtEvent event,
+ EventID eventId, Object aCallbackArgument,List pendingCallbacks, FilterRoutingInfo filterRoutingInfo,
+ ClientProxyMembershipID bridgeContext, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+ long startPut = CachePerfStats.getStatTime();
+ this.entries.txApplyPut(putOp, key, newValue, didDestroy, rmtOrigin,
+ event, eventId, aCallbackArgument,pendingCallbacks,filterRoutingInfo,bridgeContext, txEntryState, versionTag, tailKey);
+ updateStatsForPut(startPut);
+ //Fix for 47507 - make sure we throw an exception if we skip the TX put because
+ //the region is cleared (due to a destroy).
+ this.checkReadiness();
+ }
+
+ /**
+ * update stats
+ */
+ private void updateStatsForPut(long startPut) {
+ getCachePerfStats().endPut(startPut, false);
+ }
+
+ final void txApplyPutPart2(RegionEntry re, Object key, Object newValue,
+ long lastModified, boolean isCreate, boolean didDestroy, boolean clearConflict )
+ {
+ if (this.testCallable != null) {
+ Operation op = isCreate ? Operation.CREATE : Operation.UPDATE;
+ this.testCallable.call(this, op, re);
+ }
+ if (isCreate) {
+ updateStatsForCreate();
+ }
+ if (!isProxy() && !clearConflict) {
+ if (this.indexManager != null) {
+ try {
+ this.indexManager.updateIndexes(re,
+ isCreate ? IndexManager.ADD_ENTRY :
+ IndexManager.UPDATE_ENTRY,
+ isCreate ? IndexProtocol.OTHER_OP :
+ IndexProtocol.AFTER_UPDATE_OP);
+ }
+ catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }
+ if (didDestroy) {
+ if (this.entryUserAttributes != null) {
+ this.entryUserAttributes.remove(key);
+ }
+ }
+ if (this.statisticsEnabled && !clearConflict) {
+ addExpiryTaskIfAbsent(re);
+ }
+ setLastModifiedTime(lastModified);
+ }
+
+ public boolean basicBridgeCreate(final Object key, final byte[] value,
+ boolean isObject, Object p_callbackArg, final ClientProxyMembershipID client,
+ boolean fromClient, EntryEventImpl clientEvent, boolean throwEntryExists) throws TimeoutException,
+ EntryExistsException, CacheWriterException
+ {
+ EventID eventId = clientEvent.getEventId();
+ Object callbackArg = p_callbackArg;
+ long startPut = CachePerfStats.getStatTime();
+ if (fromClient) {
+ // If this region is also wan-enabled, then wrap that callback arg in a
+ // GatewayEventCallbackArgument to store the event id.
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+ }
+ //Asif: Modified the call to this constructor by passing the new value obtained from remote site
+ //instead of null .
+ //The need for this arose, because creation of EntryEvent, makes call to PartitionResolver,
+ //to get Hash. If the partitioning column is different from primary key,
+ //the resolver for Sqlfabric is not able to obtain the hash object used for creation of KeyInfo
+
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.CREATE, key,
+ value, callbackArg, false /* origin remote */, client.getDistributedMember(),
+ true /* generateCallbacks */,
+ eventId);
+ try {
+ event.setContext(client);
+
+ // if this is a replayed operation or WAN event we may already have a version tag
+ event.setVersionTag(clientEvent.getVersionTag());
+ //carry over the possibleDuplicate flag from clientEvent
+ event.setPossibleDuplicate(clientEvent.isPossibleDuplicate());
+
+ //Fix for 42448 - Only make create with null a local invalidate for
+ //normal regions. Otherwise, it will become a distributed invalidate.
+ if(getDataPolicy() == DataPolicy.NORMAL) {
+ event.setLocalInvalid(true);
+ }
+
+ // Set the new value to the input byte[] if it isn't null
+ /// For SqlFabric, if the new value happens to be an serialized object, then
+ //it needs to be converted into VMCachedDeserializable , or serializable delta
+ // as the case may be
+ if (value != null) {
+ // If the byte[] represents an object, then store it serialized
+ // in a CachedDeserializable; otherwise store it directly as a byte[]
+ if (isObject) {
+ // The value represents an object
+ event.setSerializedNewValue(value);
+ }
+ else {
+ // The value does not represent an object
+ event.setNewValue(value);
+ }
+ }
+
+ boolean ifNew = true; // cannot overwrite an existing key
+ boolean ifOld = false; // can create a new key
+ long lastModified = 0L; // use now
+ boolean overwriteDestroyed = false; // not okay to overwrite the DESTROYED
+ // token
+ boolean success = basicUpdate(event, ifNew, ifOld, lastModified,
+ overwriteDestroyed);
+ clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
+ if (success) {
+ clientEvent.setVersionTag(event.getVersionTag());
+ getCachePerfStats().endPut(startPut, event.isOriginRemote());
+ }
+ else {
+ this.stopper.checkCancelInProgress(null);
+ if (throwEntryExists) {
+ throw new EntryExistsException(""+key, event.getOldValue());
+ }
+ }
+ return success;
+ } finally {
+ event.release();
+ }
+ }
+
+ public boolean basicBridgePut(Object key, Object value, byte[] deltaBytes,
+ boolean isObject, Object p_callbackArg, ClientProxyMembershipID memberId,
+ boolean fromClient, EntryEventImpl clientEvent, boolean isSqlFabricSystem)
+ throws TimeoutException, CacheWriterException {
+ EventID eventID = clientEvent.getEventId();
+ Object callbackArg = p_callbackArg;
+ long startPut = CachePerfStats.getStatTime();
+ if (fromClient) {
+ // If this region is also wan-enabled, then wrap that callback arg in a
+ // GatewayEventCallbackArgument to store the event id.
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+ }
+
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.UPDATE, key,
+ null /* new value */, callbackArg,
+ false /* origin remote */, memberId.getDistributedMember(),
+ true /* generateCallbacks */,
+ eventID);
+ try {
+ event.setContext(memberId);
+ event.setDeltaBytes(deltaBytes);
+
+ // if this is a replayed operation we may already have a version tag
+ event.setVersionTag(clientEvent.getVersionTag());
+ //carry over the possibleDuplicate flag from clientEvent
+ event.setPossibleDuplicate(clientEvent.isPossibleDuplicate());
+
+ // Set the new value to the input byte[]
+ // If the byte[] represents an object, then store it
+ // serialized in a CachedDeserializable; otherwise store it directly
+ // as a byte[].
+ if (isObject && value instanceof byte[]) {
+ //Asif: If the system is SqlFabric, then the value byte[] corresponds to
+ //Delta , so we need to deserialize it & appropriately set the EntryEventImpl's.delta
+ //field to this value
+ if (isSqlFabricSystem) {
+ event.setNewValue(EntryEventImpl.deserialize((byte[])value));
+ } else {
+ event.setSerializedNewValue((byte[])value);
+ }
+ }
+ else {
+ event.setNewValue(value);
+ }
+
+ boolean ifNew = false; // can overwrite an existing key
+
+ //Asif: If the system is SqlFabric, then update will always have value of type
+ //SerializableDelta (i.e Delta) which requires that the old value should be present
+ boolean ifOld = isSqlFabricSystem ; //false; // can create a new key
+ long lastModified = 0L; // use now
+ boolean overwriteDestroyed = false; // not okay to overwrite the DESTROYED token
+ boolean success = false;
+ try {
+ success = basicUpdate(event, ifNew, ifOld, lastModified,
+ overwriteDestroyed);
+ } catch (ConcurrentCacheModificationException ex) { // thrown by WAN conflicts
+ event.isConcurrencyConflict(true);
+ }
+ clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
+ if (success) {
+ clientEvent.setVersionTag(event.getVersionTag());
+ getCachePerfStats().endPut(startPut, event.isOriginRemote());
+ }
+ else {
+ this.stopper.checkCancelInProgress(null);
+ }
+ return success;
+ } finally {
+ event.release();
+ }
+ }
+
+ /**
+ * issue a config message if the server and client have different
+ * concurrency checking expectations
+ * @param tag
+ */
+ private void concurrencyConfigurationCheck(VersionTag tag) {
+ if (!this.concurrencyMessageIssued && ((tag != null) != this.concurrencyChecksEnabled)) {
+ this.concurrencyMessageIssued = true;
+ logger.info(LocalizedMessage.create(LocalizedStrings.LocalRegion_SERVER_HAS_CONCURRENCY_CHECKS_ENABLED_0_BUT_CLIENT_HAS_1_FOR_REGION_2,
+ new Object[]{ !this.concurrencyChecksEnabled, this.concurrencyChecksEnabled, this}));
+ }
+ }
+
+ /**
+ * Perform an update in a bridge client. See CacheClientUpdater.handleUpdate()
+ * The op is from the bridge server and should not be distributed back to it.
+ *
+ * @throws CacheWriterException TODO-javadocs
+ */
+ public void basicBridgeClientUpdate(DistributedMember serverId, Object key,
+ Object value, byte[] deltaBytes, boolean isObject,
+ Object callbackArgument, boolean isCreate, boolean processedMarker,
+ EntryEventImpl event, EventID eventID) throws TimeoutException,
+ CacheWriterException {
+ if (isCacheContentProxy()) {
+ return;
+ }
+ concurrencyConfigurationCheck(event.getVersionTag());
+ long startPut = CachePerfStats.getStatTime();
+ // Generate EventID as it is possible that client is a cache server
+ // in hierarchical cache
+ if (generateEventID() && !this.cache.getCacheServers().isEmpty()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ } else {
+ event.setEventId(eventID);
+ }
+ event.setDeltaBytes(deltaBytes);
+
+ // Set the new value to the input byte[] if it isn't null
+ if (value != null) {
+ // If the byte[] represents an object, then store it
+ // serialized in a CachedDeserializable; otherwise store it directly
+ // as a byte[].
+ if (isObject && value instanceof byte[]) {
+ // The value represents an object
+ event.setSerializedNewValue((byte[])value);
+ }
+ else {
+ // The value does not represent an object
+ event.setNewValue(value);
+ }
+ }
+
+ // If the marker has been processed, process this put event normally;
+ // otherwise, this event occurred in the past and has been stored for a
+ // durable client. In this case, just invoke the put callbacks.
+ if (processedMarker) {
+ boolean ifNew = false; // can overwrite an existing key
+ boolean ifOld = false; // can create a new key
+ long lastModified = 0L; // use now
+ boolean overwriteDestroyed = true; //okay to overwrite the DESTROYED token
+ if (basicUpdate(event, ifNew, ifOld, lastModified, overwriteDestroyed)) {
+ getCachePerfStats().endPut(startPut, event.isOriginRemote());
+ }
+ }
+ else {
+ if (isInitialized()) {
+ invokePutCallbacks(isCreate ? EnumListenerEvent.AFTER_CREATE
+ : EnumListenerEvent.AFTER_UPDATE, event, true, true);
+ }
+ }
+ }
+
+ /**
+ * Perform an invalidate in a bridge client.
+ * The op is from the bridge server and should not be distributed back to it.
+ *
+ * @throws EntryNotFoundException TODO-javadocs
+ */
+ public void basicBridgeClientInvalidate(DistributedMember serverId, Object key,
+ Object callbackArgument, boolean processedMarker, EventID eventID,
+ VersionTag versionTag)
+ throws EntryNotFoundException {
+ if (!isCacheContentProxy()) {
+ concurrencyConfigurationCheck(versionTag);
+
+ // Create an event and put the entry
+ EntryEventImpl event =
+ EntryEventImpl.create(this,
+ Operation.INVALIDATE,
+ key, null /* newValue */,
+ callbackArgument /* callbackArg*/,
+ true /*originRemote*/,
+ serverId
+ );
+ try {
+
+ event.setVersionTag(versionTag);
+ event.setFromServer(true);
+ if (generateEventID() && !this.cache.getCacheServers().isEmpty()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ } else {
+ event.setEventId(eventID);
+ }
+
+ // If the marker has been processed, process this invalidate event
+ // normally; otherwise, this event occurred in the past and has been
+ // stored for a durable client. In this case, just invoke the invalidate
+ // callbacks.
+ if (processedMarker) {
+ // [bruce] changed to force new entry creation for consistency
+ final boolean forceNewEntry = this.concurrencyChecksEnabled;
+ basicInvalidate(event, true, forceNewEntry);
+ if (event.isConcurrencyConflict()) { // bug #45520 - we must throw this for the CacheClientUpdater
+ throw new ConcurrentCacheModificationException();
+ }
+ } else {
+ if (isInitialized()) {
+ invokeInvalidateCallbacks(EnumListenerEvent.AFTER_INVALIDATE, event,
+ true);
+ }
+ }
+ } finally {
+ event.release();
+ }
+ }
+ }
+
+ /**
+ * Perform a destroy in a bridge client.
+ * The op is from the bridge server and should not be distributed back to it.
+ *
+ * @throws EntryNotFoundException TODO-javadocs
+ */
+ public void basicBridgeClientDestroy(DistributedMember serverId, Object key,
+ Object callbackArgument, boolean processedMarker, EventID eventID,
+ VersionTag versionTag)
+ throws EntryNotFoundException {
+ if (!isCacheContentProxy()) {
+ concurrencyConfigurationCheck(versionTag);
+
+ // Create an event and destroy the entry
+ EntryEventImpl event =
+ EntryEventImpl.create(this,
+ Operation.DESTROY,
+ key, null /* newValue */,
+ callbackArgument /* callbackArg*/,
+ true /*originRemote*/,
+ serverId
+ );
+ try {
+ event.setFromServer(true);
+ event.setVersionTag(versionTag);
+
+ if (generateEventID() && !this.cache.getCacheServers().isEmpty()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ } else {
+ event.setEventId(eventID);
+ }
+ // If the marker has been processed, process this destroy event normally;
+ // otherwise, this event occurred in the past and has been stored for a
+ // durable client. In this case, just invoke the destroy callbacks.
+ if(logger.isDebugEnabled()) {
+ logger.debug("basicBridgeClientDestroy(processedMarker={})", processedMarker);
+ }
+ if (processedMarker) {
+ basicDestroy(event,
+ false, // cacheWrite
+ null); // expectedOldValue
+ if (event.isConcurrencyConflict()) { // bug #45520 - we must throw an exception for CacheClientUpdater
+ throw new ConcurrentCacheModificationException();
+ }
+ } else {
+ if (isInitialized()) {
+ invokeDestroyCallbacks(EnumListenerEvent.AFTER_DESTROY, event, true, true);
+ }
+ }
+ } finally {
+ event.release();
+ }
+ }
+ }
+
+ /**
+ * Clear the region from a server request.
+ * @param callbackArgument The callback argument. This is currently null
+ * since {@link java.util.Map#clear} supports no parameters.
+ * @param processedMarker Whether the marker has been processed (for durable
+ * clients)
+ */
+ public void basicBridgeClientClear(Object callbackArgument, boolean processedMarker) {
+ checkReadiness();
+ checkForNoAccess();
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.REGION_LOCAL_CLEAR, callbackArgument, true, getMyId(),
+ generateEventID()/* generate EventID */);
+ // If the marker has been processed, process this clear event normally;
+ // otherwise, this event occurred in the past and has been stored for a
+ // durable client. In this case, just invoke the clear callbacks.
+ if (processedMarker) {
+ basicLocalClear(event);
+ } else {
+ if (isInitialized()) {
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REGION_CLEAR, event);
+ }
+ }
+ }
+
+
+
+ public void basicBridgeDestroy(Object key, Object p_callbackArg,
+ ClientProxyMembershipID memberId, boolean fromClient, EntryEventImpl clientEvent)
+ throws TimeoutException, EntryNotFoundException, CacheWriterException
+ {
+ Object callbackArg = p_callbackArg;
+ if (fromClient) {
+ // If this region is also wan-enabled, then wrap that callback arg in a
+ // GatewayEventCallbackArgument to store the event id.
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+ }
+
+ // Create an event and put the entry
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.DESTROY, key,
+ null /* new value */, callbackArg,
+ false /* origin remote */, memberId.getDistributedMember(),
+ true /* generateCallbacks */,
+ clientEvent.getEventId());
+ try {
+ event.setContext(memberId);
+ // if this is a replayed or WAN operation we may already have a version tag
+ event.setVersionTag(clientEvent.getVersionTag());
+ try {
+ basicDestroy(event,
+ true, // cacheWrite
+ null); // expectedOldValue
+ } catch (ConcurrentCacheModificationException ex) { // thrown by WAN conflicts
+ event.isConcurrencyConflict(true);
+ } finally {
+ clientEvent.setVersionTag(event.getVersionTag());
+ clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
+ clientEvent.setIsRedestroyedEntry(event.getIsRedestroyedEntry());
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+
+ public void basicBridgeInvalidate(Object key, Object p_callbackArg,
+ ClientProxyMembershipID memberId, boolean fromClient, EntryEventImpl clientEvent)
+ throws TimeoutException, EntryNotFoundException, CacheWriterException
+ {
+ Object callbackArg = p_callbackArg;
+ if (fromClient) {
+ // If this region is also wan-enabled, then wrap that callback arg in a
+ // GatewayEventCallbackArgument to store the event id.
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+ }
+
+ // Create an event and put the entry
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.INVALIDATE, key,
+ null /* new value */, callbackArg,
+ false /* origin remote */, memberId.getDistributedMember(),
+ true /* generateCallbacks */,
+ clientEvent.getEventId());
+ try {
+ event.setContext(memberId);
+
+ // if this is a replayed operation we may already have a version tag
+ event.setVersionTag(clientEvent.getVersionTag());
+
+ try {
+ basicInvalidate(event);
+ } finally {
+ clientEvent.setVersionTag(event.getVersionTag());
+ clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+ public void basicBridgeUpdateVersionStamp(Object key, Object p_callbackArg,
+ ClientProxyMembershipID memberId, boolean fromClient, EntryEventImpl clientEvent) {
+
+ // Create an event and update version stamp of the entry
+ EntryEventImpl event = EntryEventImpl.create(this, Operation.UPDATE_VERSION_STAMP, key,
+ null /* new value */, null /*callbackArg*/,
+ false /* origin remote */, memberId.getDistributedMember(),
+ false /* generateCallbacks */,
+ clientEvent.getEventId());
+ event.setContext(memberId);
+
+ // if this is a replayed operation we may already have a version tag
+ event.setVersionTag(clientEvent.getVersionTag());
+
+ try {
+ basicUpdateEntryVersion(event);
+ } finally {
+ clientEvent.setVersionTag(event.getVersionTag());
+ clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
+ event.release();
+ }
+ }
+
+ void basicUpdateEntryVersion(EntryEventImpl event) throws EntryNotFoundException {
+ if (hasSeenEvent(event)) {
+ if(logger.isTraceEnabled(LogMarker.DM)) {
+ logger.trace(LogMarker.DM, "LR.basicDestroy: this cache has already seen this event {}", event);
+ }
+ if (this.concurrencyChecksEnabled && event.getVersionTag() != null && !event.getVersionTag().isRecorded()) {
+ getVersionVector().recordVersion((InternalDistributedMember) event.getDistributedMember(), event.getVersionTag());
+ }
+ return;
+ }
+
+ getDataView().updateEntryVersion(event);
+ }
+
+ /**
+ * Allows null as new value to accomodate create with a null value.
+ *
+ * @param event
+ * the event object for this operation, with the exception that the
+ * oldValue parameter is not yet filled in. The oldValue will be
+ * filled in by this operation.
+ *
+ * @param ifNew
+ * true if this operation must not overwrite an existing key
+ * @param ifOld
+ * true if this operation must not create a new entry
+ * @param lastModified
+ * the lastModified time to set with the value; if 0L, then the
+ * lastModified time will be set to now.
+ * @param overwriteDestroyed
+ * true if okay to overwrite the DESTROYED token: when this is true
+ * has the following effect: even when ifNew is true will write over
+ * DESTROYED token when overwriteDestroyed is false and ifNew or
+ * ifOld is true then if the put doesn't occur because there is a
+ * DESTROYED token present then the entry flag blockedDestroyed is
+ * set.
+ * @return false if ifNew is true and there is an existing key, or ifOld is
+ * true and there is no existing entry; otherwise return true.
+ */
+ final boolean basicUpdate(final EntryEventImpl event,
+ final boolean ifNew,
+ final boolean ifOld,
+ final long lastModified,
+ final boolean overwriteDestroyed)
+ throws TimeoutException,
+ CacheWriterException {
+ // check validity of key against keyConstraint
+ if (this.keyConstraint != null) {
+ if (!this.keyConstraint.isInstance(event.getKey()))
+ throw new ClassCastException(LocalizedStrings.LocalRegion_KEY_0_DOES_NOT_SATISFY_KEYCONSTRAINT_1.toLocalizedString(new Object[] {event.getKey().getClass().getName(), this.keyConstraint.getName()}));
+ }
+
+ validateValue(event.basicGetNewValue());
+
+ return getDataView().putEntry(event, ifNew, ifOld, null, false, lastModified, overwriteDestroyed);
+ }
+
+ /**
+ * Subclasses should reimplement if needed
+ *
+ * @see DistributedRegion#virtualPut(EntryEventImpl, boolean, boolean,
+ * Object, boolean, long, boolean)
+ */
+ boolean virtualPut(final EntryEventImpl event,
+ final boolean ifNew,
+ final boolean ifOld,
+ Object expectedOldValue,
+ boolean requireOldValue,
+ final long lastModified,
+ final boolean overwriteDestroyed)
+ throws TimeoutException,
+ CacheWriterException {
+ if (!MemoryThresholds.isLowMemoryExceptionDisabled()) {
+ checkIfAboveThreshold(event);
+ }
+ Operation originalOp = event.getOperation();
+ RegionEntry oldEntry = null;
+
+ try {
+ oldEntry = this.entries.basicPut(event,
+ lastModified,
+ ifNew,
+ ifOld,
+ expectedOldValue,
+ requireOldValue,
+ overwriteDestroyed);
+ } catch (ConcurrentCacheModificationException e) {
+ // this can happen in a client cache when another thread
+ // managed to slip in its version info to the region entry before this
+ // thread got around to doing so
+ if(logger.isDebugEnabled()) {
+ logger.debug("caught concurrent modification attempt when applying {}", event);
+ }
+ notifyBridgeClients(event);
+ return false;
+ }
+
+ // for EMPTY clients, see if a concurrent map operation had an entry on the server
+ ServerRegionProxy mySRP = getServerProxy();
+
+ if (mySRP != null && this.dataPolicy == DataPolicy.EMPTY) {
+ if (originalOp == Operation.PUT_IF_ABSENT) {
+ return !event.hasOldValue();
+ }
+ if (originalOp == Operation.REPLACE && !requireOldValue) {
+ // LocalRegion.serverPut throws an EntryNotFoundException if the operation failed
+ return true;
+ }
+ }
+
+ return oldEntry != null;
+ }
+
+ /**
+ * check to see if a LowMemoryException should be thrown for this event
+ * @param evi
+ * @throws LowMemoryException
+ */
+ public void checkIfAboveThreshold(final EntryEventImpl evi) throws LowMemoryException {
+ if (evi == null) {
+ checkIfAboveThreshold("UNKNOWN");
+ return;
+ }
+ // Threshold check is performed elsewhere for putAll when there is a server proxy
+ boolean alreadyCheckedThreshold = hasServerProxy() && evi.getOperation().isPutAll();
+ if (! alreadyCheckedThreshold && !evi.isOriginRemote()) {
+ checkIfAboveThreshold(evi.getKey());
+ }
+ }
+
+ /**
+ * Checks to see if the event should be rejected because of sick state either due to
+ * exceeding local critical threshold or a remote member exceeding critical threshold
+ * @param key the key for the operation
+ * @throws LowMemoryException if the target member for this operation is sick
+ */
+ private void checkIfAboveThreshold(final Object key) throws LowMemoryException{
+ if (memoryThresholdReached.get()) {
+ Set htrm = getMemoryThresholdReachedMembers();
+
+ // #45603: trigger a background eviction since we're above the the critical
+ // threshold
+ InternalResourceManager.getInternalResourceManager(cache).getHeapMonitor().updateStateAndSendEvent();
+
+ Object[] prms = new Object[] {getFullPath(), key, htrm};
+ throw new LowMemoryException(LocalizedStrings.ResourceManager_LOW_MEMORY_IN_0_FOR_PUT_1_MEMBER_2.toLocalizedString(prms),
+ htrm);
+ }
+ }
+
+ /**
+ * Perform a put without invoking callbacks or checking for transactions
+ */
+ /*public Object putNoCallbacks(Object key, Object value) {
+ EntryEventImpl event = new EntryEventImpl(
+ this, Operation.UPDATE, key,
+ value,
+ nullcallbackobj, false,
+ getMyId(),
+ true, true);
+ event.setNewEventId(getCache().getDistributedSystem());
+ boolean didPut = this.entries.basicPut(event, System.currentTimeMillis(),
+ false, false, true, false) != null;
+ if (didPut) {
+ return event.getOldValue();
+ }
+ else {
+ return null;
+ }
+ }*/
+
+ /**
+ * Allows null as new value to accomodate create with a null value.
+ *
+ * @param event
+ * the event object for this operation, with the exception that the
+ * oldValue parameter is not yet filled in. The oldValue will be
+ * filled in by this operation.
+ * @param lastModified
+ * the lastModified time to set with the value; if 0L then the
+ * lastModified time will be set to now.
+ * @return null if put not done; otherwise the put entry
+ */
+ protected RegionEntry basicPutEntry(final EntryEventImpl event,
+ final long lastModified)
+ throws TimeoutException, CacheWriterException {
+ discoverJTA();
+ TXStateInterface tx = getTXState();
+ // Note we are doing a load or netsearch result so it seems like
+ // we should set ifNew to true. The entry should not yet exist.
+ // However since the non-tx code sets ifNew to false this code will also.
+ final boolean ifNew = false;
+
+ if (isTX()) {
+ tx.txPutEntry(event, ifNew, false, false, null);
+ return null;
+ }
+ else {
+ if (GemFireCacheImpl.internalBeforeNonTXBasicPut != null) {
+ GemFireCacheImpl.internalBeforeNonTXBasicPut.run();
+ }
+
+ RegionEntry oldEntry = this.entries.basicPut(event,
+ lastModified,
+ ifNew,
+ false, // ifOld
+ null, // expectedOldValue
+ false, // requireOldValue
+ false); // overwriteDestroyed
+ return oldEntry;
+ }
+ }
+
+ protected long basicPutPart2(EntryEventImpl event, RegionEntry entry,
+ boolean isInitialized, long lastModified,
+ boolean clearConflict)
+ {
+ final boolean isNewKey = event.getOperation().isCreate();
+ final boolean invokeCallbacks = !entry.isTombstone(); // put() is creating a tombstone
+
+ if (isNewKey) {
+ updateStatsForCreate();
+ }
+ final boolean lruRecentUse = event.isNetSearch() || event.isLoad(); // fix for bug 31102
+ // the event may have a version timestamp that we need to use, so get the
+ // event time to store in the entry
+ long lastModifiedTime = event.getEventTime(lastModified);
+ updateStatsForPut(entry, lastModifiedTime, lruRecentUse);
+ if (!isProxy()) {
+ //if (this.isUsedForPartitionedRegionBucket) {
+ // if (this.sqlfIndexManager != null) {
+ // this.sqlfIndexManager.onEvent(this, event, entry);
+ // }
+ //}
+
+ if (!clearConflict && this.indexManager != null) {
+ try {
+ if (!entry.isInvalid()) {
+ this.indexManager.updateIndexes(entry,
+ isNewKey ? IndexManager.ADD_ENTRY :
+ IndexManager.UPDATE_ENTRY,
+ isNewKey ? IndexProtocol.OTHER_OP :
+ IndexProtocol.AFTER_UPDATE_OP);
+ }
+ }
+ catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }
+
+ if (invokeCallbacks) {
+ boolean doCallback = false;
+ if (isInitialized) {
+ // fix for #46662: skip wan notification during import
+ // newwan moves notification to here from invokePutCallbacks
+ if (event.isGenerateCallbacks()) {
+ doCallback = true;
+ }
+ }
+ else if (this.isUsedForPartitionedRegionBucket) {
+ // invokePutCallbacks in BucketRegion will be more discriminating
+ doCallback = true;
+ }
+ if (doCallback) {
+ notifyGatewaySender(event.getOperation().isUpdate()? EnumListenerEvent.AFTER_UPDATE
+ : EnumListenerEvent.AFTER_CREATE, event);
+ // Notify listeners
+ if (!event.isBulkOpInProgress()) {
+ try {
+ entry.dispatchListenerEvents(event);
+ }
+ catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ stopper.checkCancelInProgress(null);
+ }
+ }
+ }
+ }
+ return lastModifiedTime;
+ }
+
+ /**
+ * To lower latency, PRs generate the local filter rounting in
+ * DistributedCacheOperation after message distribution and before waiting for
+ * responses.
+ *
+ * Warning: Even if you comment out bucket condition in following method,
+ * getLocalRountingInfo() does NOT process CQs for bucket regions internally.
+ * Check
+ * {@link FilterProfile#getFilterRoutingInfoPart2(FilterRoutingInfo, CacheEvent)}
+ * .
+ *
+ * @param event
+ */
+ protected void generateLocalFilterRouting(InternalCacheEvent event) {
+ boolean isEntryEvent = event.getOperation().isEntry();
+ EntryEventImpl entryEvent = isEntryEvent? (EntryEventImpl)event : null;
+
+ FilterProfile fp = this.getFilterProfile();
+ FilterInfo routing = event.getLocalFilterInfo();
+ boolean lockForCQ = false;
+ Object re = null;
+ if (fp != null && routing == null) {
+ if (isEntryEvent && entryEvent.getRegionEntry() != null) {
+ // bug #45520 we should either have the lock on the region entry
+ // or the event was elided and CQ processing won't be done on it
+ re = entryEvent.getRegionEntry();
+ if (!entryEvent.isConcurrencyConflict()) {
+ Assert.assertTrue(re != null);
+ lockForCQ = true;
+ }
+ }
+ if (isEntryEvent) {
+ if(logger.isDebugEnabled()) {
+ logger.debug("getting local client routing.");
+ }
+ }
+ if (lockForCQ) {
+ synchronized(re) {
+ routing = fp.getLocalFilterRouting(event);
+ }
+ } else {
+ routing = fp.getLocalFilterRouting(event);
+ }
+ event.setLocalFilterInfo(routing);
+ }
+ // bug #45520 - do not send CQ events to clients out of order
+ if (routing != null && event.getOperation().isEntry()
+ && ((EntryEventImpl)event).isConcurrencyConflict()) {
+ if(logger.isDebugEnabled()) {
+ logger.debug("clearing CQ routing for event that's in conflict");
+ }
+ routing.clearCQRouting();
+ }
+ }
+
+ /**
+ * This notifies all WAN sites about updated timestamp on local site.
+ *
+ * @param event
+ */
+ protected void notifyTimestampsToGateways(EntryEventImpl event) {
+
+ // Create updateTimeStampEvent from event.
+ EntryEventImpl updateTimeStampEvent = EntryEventImpl.createVersionTagHolder(event.getVersionTag());
+ try {
+ updateTimeStampEvent.setOperation(Operation.UPDATE_VERSION_STAMP);
+ updateTimeStampEvent.setKeyInfo(event.getKeyInfo());
+ updateTimeStampEvent.setGenerateCallbacks(false);
+ updateTimeStampEvent.distributedMember = event.getDistributedMember();
+ updateTimeStampEvent.setNewEventId(getSystem());
+
+
+ if (event.getRegion() instanceof BucketRegion) {
+ BucketRegion br = (BucketRegion)event.getRegion();
+ PartitionedRegion pr = br.getPartitionedRegion();
+ updateTimeStampEvent.setRegion(pr);
+
+ // increment the tailKey for the event
+ if (pr.isParallelWanEnabled()) {
+ br.handleWANEvent(updateTimeStampEvent);
+ }
+
+ if (pr.isInitialized()) {
+ pr.notifyGatewaySender(EnumListenerEvent.TIMESTAMP_UPDATE, updateTimeStampEvent);
+ }
+ } else {
+ updateTimeStampEvent.setRegion(event.getRegion());
+ notifyGatewaySender(EnumListenerEvent.TIMESTAMP_UPDATE, updateTimeStampEvent);
+ }
+ } finally {
+ updateTimeStampEvent.release();
+ }
+ }
+
+
+ /**
+ * Update CachePerfStats
+ */
+ private void updateStatsForCreate() {
+ getCachePerfStats().incCreates();
+ }
+
+ public void basicPutPart3(EntryEventImpl event, RegionEntry entry,
+ boolean isInitialized, long lastModified, boolean invokeCallbacks,
+ boolean ifNew, boolean ifOld, Object expectedOldValue,
+ boolean requireOldValue)
+ {
+ // We used to dispatch listener events here which is moved to part2 to be in RE lock #45520.
+ if (invokeCallbacks) {
+ if (event.isBulkOpInProgress()) {
+ event.getPutAllOperation().addEntry(event);
+ }
+ }
+ }
+
+ public void invokePutCallbacks(final EnumListenerEvent eventType,
+ final EntryEventImpl event, final boolean callDispatchListenerEvent, boolean notifyGateways) {
+
+ // disallow callbacks on import
+ if (!event.isGenerateCallbacks()) {
+ return;
+ }
+
+ // Notify bridge clients (if this is a BridgeServer)
+ Operation op = event.getOperation();
+ // The spec for ConcurrentMap support requires that operations be mapped
+ // to non-CM counterparts
+ if (op == Operation.PUT_IF_ABSENT) {
+ event.setOperation(Operation.CREATE);
+ } else if (op == Operation.REPLACE) {
+ event.setOperation(Operation.UPDATE);
+ }
+ event.setEventType(eventType);
+ notifyBridgeClients(event);
+ if (notifyGateways) {
+ notifyGatewaySender(eventType, event);
+ }
+ if(callDispatchListenerEvent){
+ dispatchListenerEvent(eventType, event);
+ }
+ }
+
+ /**
+ * @see DistributedRegion#postUpdate(EntryEventImpl, long)
+ * @param event
+ * @param lastModifiedTime
+ */
+ protected void postUpdate(EntryEventImpl event, long lastModifiedTime)
+ {
+ }
+
+ /**
+ * retrieve a deep copy of the Region's event state. This is used
+ * for getInitialImage. The result is installed in the receiver of
+ * the image.
+ */
+ public Map extends DataSerializable, ? extends DataSerializable> getEventState() {
+ if (this.eventTracker != null) {
+ return this.eventTracker.getState();
+ }
+ else {
+ return null;
+ }
+ }
+
+ /**
+ * Record the event state encapsulated in the given Map.
+ * This is intended for state transfer during GII.
+ * @param provider the member that provided this state
+ * @param state a Map obtained from getEventState()
+ */
+ protected void recordEventState(InternalDistributedMember provider, Map state) {
+ if (this.eventTracker != null) {
+ this.eventTracker.recordState(provider, state);
+ }
+ }
+
+ /**
+ * generate version tag if it does not exist and set it into the event.
+ * @param event
+ * @param entry
+ */
+ public void generateAndSetVersionTag(InternalCacheEvent event, RegionEntry entry) {
+ if (entry != null && event.getOperation().isEntry()) {
+ EntryEventImpl entryEvent = (EntryEventImpl)event;
+ if (!entryEvent.isOriginRemote() && shouldGenerateVersionTag(entry, entryEvent)) {
+ boolean eventHasDelta = (getSystem().getConfig().getDeltaPropagation()
+ && !this.scope.isDistributedNoAck()
+ && entryEvent.getDeltaBytes() != null);
+ VersionTag v = entry.generateVersionTag(null, eventHasDelta, this, entryEvent);
+ if(logger.isDebugEnabled()) {
+ logger.debug("generated version tag {} for {}", v, entryEvent.getKey());
+ }
+ }
+ }
+ }
+
+ /**
+ * record the event's sequenceId in Region's event state to prevent replay.
+ * @param event
+ */
+ public void recordEvent(InternalCacheEvent event) {
+ if (this.eventTracker != null) {
+ this.eventTracker.recordEvent(event);
+ }
+ }
+
+ /**
+ * has the Region's event state seen this event?
+ * @param event
+ * @return true if the Region's event state has seen the event
+ */
+ public boolean hasSeenEvent(EntryEventImpl event) {
+ boolean isDup = false;
+ if (this.eventTracker != null) {
+ // bug 41289 - wait for event tracker to be initialized before checkin
+ // so that an operation inteded for a previous version of a bucket
+ // is not prematurely applied to a new version of the bucket
+ if (this.isUsedForPartitionedRegionBucket()) {
+ try {
+ this.eventTracker.waitOnInitialization();
+ } catch (InterruptedException ie) {
+ this.stopper.checkCancelInProgress(ie);
+ Thread.currentThread().interrupt();
+ }
+ }
+ isDup = this.eventTracker.hasSeenEvent(event);
+ // don't clobber existing posDup flag e.g. set from SQLFabric client
+ if (isDup) {
+ event.setPossibleDuplicate(true);
+ if (this.concurrencyChecksEnabled && event.getVersionTag() == null) {
+ event.setVersionTag(findVersionTagForClientEvent(event.getEventId()));
+ }
+ } else {
+ // bug #48205 - a retried PR operation may already have a version assigned to it
+ // in another VM
+ if (event.isPossibleDuplicate()
+ && event.getRegion().concurrencyChecksEnabled
+ && (event.getVersionTag() == null)
+ && (event.getEventId() != null)) {
+ boolean isBulkOp = event.getOperation().isPutAll() || event.getOperation().isRemoveAll();
+ VersionTag tag = FindVersionTagOperation.findVersionTag(event.getRegion(), event.getEventId(), isBulkOp);
+ event.setVersionTag(tag);
+ }
+ }
+ }
+ return isDup;
+ }
+
+ /**
+ * tries to find the version tag for a replayed client event
+ * @param eventId
+ * @return the version tag, if known. Null if not
+ */
+ public VersionTag findVersionTagForClientEvent(EventID eventId) {
+ if (this.eventTracker != null) {
+ return this.eventTracker.findVersionTag(eventId);
+ }
+ return null;
+ }
+
+ public VersionTag findVersionTagForGatewayEvent(EventID eventId) {
+ if (this.eventTracker != null) {
+ return this.eventTracker.findVersionTagForGateway(eventId);
+ }
+ return null;
+ }
+
+ /**
+ * tries to find the version tag for a replayed client event
+ * @param eventId
+ * @return the version tag, if known. Null if not
+ */
+ public VersionTag findVersionTagForClientBulkOp(EventID eventId) {
+ if (eventId == null) {
+ return null;
+ }
+ if (this.eventTracker != null) {
+ return this.eventTracker.findVersionTagForBulkOp(eventId);
+ }
+ return null;
+ }
+
+ /**
+ * has the Region's event state seen this event? Most checks should use
+ * the method that takes an Event, not an ID, but with transactions we
+ * do not have an event at the time the check needs to be made. Consequently,
+ * this method may cause events to be recorded that would otherwise be
+ * ignored.
+ * @param eventID the identifier of the event
+ * @return true if the Region's event state has seen the event
+ */
+ public boolean hasSeenEvent(EventID eventID) {
+ if (eventID == null) {
+ return false;
+ }
+ boolean isDup = false;
+ if (this.eventTracker != null) {
+ // bug 41289 - wait for event tracker to be initialized before checkin
+ // so that an operation intended for a previous version of a bucket
+ // is not prematurely applied to a new version of the bucket
+ if (this.isUsedForPartitionedRegionBucket()) {
+ try {
+ this.eventTracker.waitOnInitialization();
+ } catch (InterruptedException ie) {
+ this.stopper.checkCancelInProgress(ie);
+ Thread.currentThread().interrupt();
+ }
+ }
+ isDup = this.eventTracker.hasSeenEvent(eventID, null);
+ }
+ return isDup;
+ }
+
+ /**
+ * A routine to provide synchronization running based on
+ * of the requesting client for the region's event state
+ * @param r - a Runnable to wrap the processing of the bulk op
+ * @param eventID - the base event ID of the bulk op
+ *
+ * @since 5.7
+ */
+ public void syncBulkOp(Runnable r, EventID eventID) {
+ if (this.eventTracker != null && !isTX()) {
+ this.eventTracker.syncBulkOp(r, eventID);
+ }
+ else {
+ r.run();
+ }
+ }
+
+ public void recordBulkOpStart(ThreadIdentifier membershipID) {
+ if (this.eventTracker != null && !isTX()) {
+ this.eventTracker.recordBulkOpStart(membershipID);
+ }
+ }
+
+ final protected void notifyBridgeClients(CacheEvent event)
+ {
+ int numBS = getCache().getCacheServers().size();
+
+ // #Bugfix 37518: In case of localOperations no need to notify clients.
+ if (event.getOperation().isLocal() || numBS == 0) {
+ return;
+ }
+ // Return if the inhibit all notifications flag is set
+ if (event instanceof EntryEventImpl) {
+ if (((EntryEventImpl)event).inhibitAllNotifications()) {
+ if(logger.isDebugEnabled()) {
+ logger.debug("Notification inhibited for key {}", event);
+ }
+ return;
+ }
+ }
+
+ if (shouldNotifyBridgeClients()) {
+ if (numBS > 0) {
+ if(logger.isDebugEnabled()) {
+ logger.debug("{}: notifying {} bridge servers of event: {}", this.getName(), numBS, event);
+ }
+ }
+
+ Operation op = event.getOperation();
+ if (event.getOperation().isEntry()) {
+ EntryEventImpl e = (EntryEventImpl)event;
+ if (e.getEventType() == null) {
+ if (op.isCreate()) {
+ e.setEventType(EnumListenerEvent.AFTER_CREATE);
+ } else if (op.isUpdate()) {
+ e.setEventType(EnumListenerEvent.AFTER_UPDATE);
+ } else if (op.isDestroy()) {
+ e.setEventType(EnumListenerEvent.AFTER_DESTROY);
+ } else if (op.isInvalidate()) {
+ e.setEventType(EnumListenerEvent.AFTER_INVALIDATE);
+ } else {
+ throw new IllegalStateException("event is missing client notification eventType: " + e);
+ }
+ }
+ }
+
+ InternalCacheEvent ice = (InternalCacheEvent)event;
+ if (!this.isUsedForPartitionedRegionBucket()) {
+ generateLocalFilterRouting(ice);
+ }
+
+ CacheClientNotifier.notifyClients((InternalCacheEvent)event);
+ }
+ }
+
+ /**
+ * Returns true if this region notifies multiple serial gateways.
+ */
+ public boolean notifiesMultipleSerialGateways() {
+ if (isPdxTypesRegion()) {
+ return false;
+ }
+ int serialGatewayCount = 0;
+ Set allGatewaySenderIds = getAllGatewaySenderIds();
+ if (!allGatewaySenderIds.isEmpty()) {
+ List allRemoteDSIds = getRemoteDsIds(allGatewaySenderIds);
+ if (allRemoteDSIds != null) {
+ for (GatewaySender sender : getCache().getAllGatewaySenders()) {
+ if (allGatewaySenderIds.contains(sender.getId())) {
+ if (!sender.isParallel()) {
+ serialGatewayCount++;
+ if (serialGatewayCount > 1) {
+ return true;
+ }
+ }
+ }
+ }
+ }
+ }
+ return false;
+ }
+
+ protected void notifyGatewaySender(EnumListenerEvent operation,
+ EntryEventImpl event) {
+
+ if (event.isConcurrencyConflict()) { // usually concurrent cache modification problem
+ return;
+ }
+
+ // Return if the inhibit all notifications flag is set
+ if (event.inhibitAllNotifications()){
+ if(logger.isDebugEnabled()) {
+ logger.debug("Notification inhibited for key {}", event);
+ }
+ return;
+ }
+
+ if (!event.getOperation().isLocal()) {
+ Set allGatewaySenderIds = null;
+ checkSameSenderIdsAvailableOnAllNodes();
+ if (event.getOperation() == Operation.UPDATE_VERSION_STAMP) {
+ allGatewaySenderIds = getGatewaySenderIds();
+ } else {
+ allGatewaySenderIds = getAllGatewaySenderIds();
+ }
+
+ List allRemoteDSIds = getRemoteDsIds(allGatewaySenderIds);
+
+ if (allRemoteDSIds != null) {
+ for (GatewaySender sender : getCache().getAllGatewaySenders()) {
+ if (!isPdxTypesRegion()) {
+ if (allGatewaySenderIds.contains(sender.getId())) {
+ //TODO: This is a BUG. Why return and not continue?
+ if((!this.getDataPolicy().withStorage()) && sender.isParallel()){
+ return;
+ }
+ if(logger.isDebugEnabled()) {
+ logger.debug("Notifying the GatewaySender : {}", sender.getId());
+ }
+ ((AbstractGatewaySender)sender).distribute(operation, event,
+ allRemoteDSIds);
+ }
+ }
+ }
+ }
+
+// if (shouldNotifyGatewaySender()) {
+// // Get All WAN site DSID's to be sent to each WAN site so that they
+// // don't redistribute it to same WAN sites again again (infinite loop).
+// if (!this.getName().equals(PeerTypeRegistration.REGION_NAME)) { //This is for all region except pdx Region
+// for (GatewaySender sender : getCache().getAllGatewaySenders()) {
+// if (allGatewaySenderIds.contains(sender.getId())) { //Make sure we are distributing to only those senders whose id is avaialble on this region
+// if(this.partitionAttributes == null && sender.isParallel()){ //ParalleGatewaySender with DR is not allowed
+// throw new IllegalStateException(
+// LocalizedStrings.AttributesFactory_PARALLELGATEWAYSENDER_0_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION
+// .toLocalizedString(sender.getId()));
+// }
+// if (allRemoteDSIds == null) {
+// allRemoteDSIds = new ArrayList();
+// }
+// allRemoteDSIds.add(sender.getRemoteDSId());
+// }
+// else { //this else is for PDX region
+// if (allRemoteDSIds == null) {
+// allRemoteDSIds = new ArrayList();
+// }
+// allRemoteDSIds.add(sender.getRemoteDSId());
+// }
+// }
+// }
+// if (allRemoteDSIds == null) {
+// allRemoteDSIds = Collections.emptyList();
+// }
+//
+// // Now distribute the event to each WAN site.
+// if (!this.getName().equals(PeerTypeRegistration.REGION_NAME)) {
+// for (GatewaySender sender : getCache().getAllGatewaySenders()) {
+// if (allGatewaySenderIds.contains(sender.getId())) {
+// if((!this.getDataPolicy().withStorage()) && sender.isParallel()){
+// return;
+// }
+// ((AbstractGatewaySender)sender).distribute(operation, event,
+// allRemoteDSIds);
+// }
+// }
+// }
+// }
+ }
+ }
+
+ public void checkSameSenderIdsAvailableOnAllNodes() {
+ }
+ /**
+ * @param cacheWrite
+ * if true, then we're just cleaning up the local cache and calling
+ * listeners,
+ * @see DistributedRegion#basicDestroyRegion(RegionEventImpl, boolean,
+ * boolean, boolean)
+ */
+ void basicDestroyRegion(RegionEventImpl event, boolean cacheWrite)
+ throws CacheWriterException, TimeoutException
+ {
+ basicDestroyRegion(event, cacheWrite, true, true);
+ }
+
+ void basicDestroyRegion(RegionEventImpl event, boolean cacheWrite,
+ boolean lock, boolean callbackEvents) throws CacheWriterException,
+ TimeoutException
+ {
+ preDestroyChecks();
+ HashSet eventSet = null;
+ final TXStateProxy tx = this.cache.getTXMgr().internalSuspend();
+ try {
+ boolean acquiredLock = false;
+ if (lock) {
+ try {
+ acquireDestroyLock();
+ acquiredLock = true;
+ }
+ catch (CancelException e) {
+ // ignore
+ if(logger.isDebugEnabled()) {
+ logger.debug("basicDestroyRegion: acquireDestroyLock failed due to cache closure, region = {}", getFullPath());
+ }
+ }
+ }
+ try { // maintain destroy lock and TXStateInterface
+ // I moved checkRegionDestroyed up out of the following
+ // try block because it does not seem correct to deliver
+ // a destroy event to the clients of the region was already
+ // destroyed on the server.
+ checkRegionDestroyed(false);
+ boolean cancelledByCacheWriterException = false; // see bug 47736
+ try { // ensure that destroy events are dispatched
+
+ if (this instanceof PartitionedRegion
+ && !((PartitionedRegion)this).getParallelGatewaySenderIds()
+ .isEmpty()) {
+ ((PartitionedRegion)this).destroyParallelGatewaySenderRegion(event.getOperation(),
+ cacheWrite, lock, callbackEvents);
+ }
+
+ if (this.parentRegion != null) {
+ // "Bubble up" the cache statistics to parent if this regions are more
+ // recent
+ this.parentRegion.updateStats();
+ }
+
+
+ try {
+ eventSet = callbackEvents ? new HashSet() : null;
+ this.destroyedSubregionSerialNumbers = collectSubregionSerialNumbers();
+ recursiveDestroyRegion(eventSet, event, cacheWrite);
+ }
+ catch (CancelException e) {
+ // This should be properly caught and ignored; if we see this there is
+ // a serious problem.
+ if (!cache.forcedDisconnect()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_RECURSIVEDESTROYREGION_RECURSION_FAILED_DUE_TO_CACHE_CLOSURE_REGION_0,
+ getFullPath()), e);
+ }
+ } catch (CacheWriterException cwe) {
+ cancelledByCacheWriterException = true;
+ throw cwe;
+ }
+
+ // at this point all subregions are destroyed and this region
+ // has been marked as destroyed and postDestroyRegion has been
+ // called for each region. The only detail left is
+ // unhooking this region from the parent subregion map, and
+ // sending listener events
+ Assert.assertTrue(this.isDestroyed);
+
+ /**
+ * Added for M&M : At this point we can safely call ResourceEvent
+ * to remove the region artifacts From Management Layer
+ **/
+ if (!isInternalRegion()) {
+ InternalDistributedSystem system = this.cache
+ .getDistributedSystem();
+ system.handleResourceEvent(ResourceEvent.REGION_REMOVE, this);
+ }
+
+ try {
+ LocalRegion parent = this.parentRegion;
+ if (parent == null) {
+ this.cache.removeRoot(this);
+ }
+ else {
+ parent.subregions.remove(this.regionName, this);
+ }
+ }
+ catch (CancelException e) {
+ // I don't think this should ever happens: bulletproofing for bug 39454
+ if (!cache.forcedDisconnect()) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_BASICDESTROYREGION_PARENT_REMOVAL_FAILED_DUE_TO_CACHE_CLOSURE_REGION_0,
+ getFullPath()), e);
+ }
+ }
+ } // ensure that destroy events are dispatched
+ finally {
+ if (!cancelledByCacheWriterException) {
+ // We only need to notify bridgeClients of the top level region destroy
+ // which it will take and do a localRegionDestroy.
+ // So we pass it event and NOT eventSet
+ event.setEventType(EnumListenerEvent.AFTER_REGION_DESTROY);
+ notifyBridgeClients(event);
+ }
+ // call sendPendingRegionDestroyEvents even if cancelledByCacheWriterException
+ // since some of the destroys happened.
+ if (eventSet != null && callbackEvents) {
+ try {
+ sendPendingRegionDestroyEvents(eventSet);
+ }
+ catch (CancelException e) {
+ // ignore, we're mute.
+ }
+ }
+ }
+ } // maintain destroy lock and TXStateInterface
+ finally {
+ if (acquiredLock) {
+ try {
+ releaseDestroyLock();
+ }
+ catch (CancelException e) {
+ // ignore
+ }
+ }
+ }
+ } finally {
+ this.cache.getTXMgr().resume(tx);
+ }
+ }
+
+ protected void preDestroyChecks() {
+
+ }
+
+ protected void distributeDestroyRegion(RegionEventImpl event, boolean notifyOfRegionDeparture) {
+ }
+
+ public static final float DEFAULT_HEAPLRU_EVICTION_HEAP_PERCENTAGE = 80.0f;
+ /**
+ * Called after this region has been completely created
+ *
+ * @since 5.0
+ *
+ * @see DistributedRegion#postDestroyRegion(boolean, RegionEventImpl)
+ */
+ protected void postCreateRegion()
+ {
+ if (getEvictionAttributes().getAlgorithm().isLRUHeap()) {
+ final LogWriter logWriter = cache.getLogger();
+ float evictionPercentage = DEFAULT_HEAPLRU_EVICTION_HEAP_PERCENTAGE;
+ // This is new to 6.5. If a heap lru region is created
+ // we make sure that the eviction percentage is enabled.
+ InternalResourceManager rm = this.cache.getResourceManager();
+ if (!getOffHeap()) {
+ if (!rm.getHeapMonitor().hasEvictionThreshold()) {
+ float criticalPercentage = rm.getCriticalHeapPercentage();
+ if (criticalPercentage > 0.0f) {
+ if (criticalPercentage >= 10.f) {
+ evictionPercentage = criticalPercentage - 5.0f;
+ } else {
+ evictionPercentage = criticalPercentage;
+ }
+ }
+ rm.setEvictionHeapPercentage(evictionPercentage);
+ if (logWriter.fineEnabled()) {
+ logWriter.fine("Enabled heap eviction at " + evictionPercentage + " percent for LRU region");
+ }
+ }
+ } else {
+ if (!rm.getOffHeapMonitor().hasEvictionThreshold()) {
+ float criticalPercentage = rm.getCriticalOffHeapPercentage();
+ if (criticalPercentage > 0.0f) {
+ if (criticalPercentage >= 10.f) {
+ evictionPercentage = criticalPercentage - 5.0f;
+ } else {
+ evictionPercentage = criticalPercentage;
+ }
+ }
+ rm.setEvictionOffHeapPercentage(evictionPercentage);
+ if (logWriter.fineEnabled()) {
+ logWriter.fine("Enabled off-heap eviction at " + evictionPercentage + " percent for LRU region");
+ }
+ }
+ }
+ }
+
+ if (!isInternalRegion()) {
+ getCachePerfStats().incRegions(1);
+ if (getMembershipAttributes().hasRequiredRoles()) {
+ getCachePerfStats().incReliableRegions(1);
+ }
+ }
+
+ if (hasListener()) {
+ RegionEventImpl event = new RegionEventImpl(this,
+ Operation.REGION_CREATE, null, false, getMyId());
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REGION_CREATE, event);
+ }
+ releaseAfterRegionCreateEventLatch();
+ SystemMemberCacheEventProcessor.send(getCache(), this,
+ Operation.REGION_CREATE);
+ initializingRegion.set(null);
+ }
+
+ /**
+ * notify region membership listeners of the initial membership
+ * @param listeners an array of listeners to notify
+ */
+ public void notifyOfInitialMembers(CacheListener[] listeners, Set others) {
+ if (listeners != null) {
+ for (int i = 0; i < listeners.length; i++) {
+ if (listeners[i] instanceof RegionMembershipListener) {
+ RegionMembershipListener rml = (RegionMembershipListener)listeners[i];
+ try {
+ DistributedMember[] otherDms = new DistributedMember[others
+ .size()];
+ others.toArray(otherDms);
+ rml.initialMembers(this, otherDms);
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.error(LocalizedMessage.create(LocalizedStrings.DistributedRegion_EXCEPTION_OCCURRED_IN_REGIONMEMBERSHIPLISTENER), t);
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * This method is invoked after isDestroyed has been set to true
+ */
+ protected void postDestroyRegion(boolean destroyDiskRegion,
+ RegionEventImpl event)
+ {
+ if (this.diskRegion != null) {
+ if (destroyDiskRegion) {
+ this.diskRegion.endDestroy(this);
+ }
+ else {
+ this.diskRegion.close(this);
+ }
+ }
+ if (this.versionVector != null) {
+ try {
+ this.cache.getDistributionManager().removeMembershipListener(this.versionVector);
+ } catch (CancelException e) {
+ // ignore: cache close will remove the membership listener
+ }
+ }
+ }
+
+ /**
+ * @param cacheWrite
+ * true if cacheWrite should be performed or false if cacheWrite
+ * should not be performed
+ * @see DistributedRegion#basicDestroy(EntryEventImpl, boolean, Object)
+ */
+ void basicDestroy(final EntryEventImpl event,
+ final boolean cacheWrite,
+ Object expectedOldValue)
+ throws EntryNotFoundException, CacheWriterException, TimeoutException {
+
+ if (!event.isOriginRemote()) {
+ checkIfReplicatedAndLocalDestroy(event);
+ }
+
+ if (hasSeenEvent(event)) {
+ assert getJTAEnlistedTX() == null;
+ if(logger.isTraceEnabled(LogMarker.DM)) {
+ logger.trace(LogMarker.DM, "LR.basicDestroy: this cache has already seen this event {}", event);
+ }
+ if (this.concurrencyChecksEnabled && event.getVersionTag() != null && !event.getVersionTag().isRecorded()) {
+ getVersionVector().recordVersion((InternalDistributedMember) event.getDistributedMember(), event.getVersionTag());
+ }
+ // Bug 49449: When client retried and returned with hasSeenEvent for both LR and DR, the server should still
+ // notifyGatewayHubs even the event could be duplicated in gateway queues1
+ notifyGatewaySender(EnumListenerEvent.AFTER_DESTROY, event);
+ return;
+ }
+
+ discoverJTA();
+ getDataView().destroyExistingEntry(event, cacheWrite, expectedOldValue);
+ }
+
+ /**
+ * Do the expensive work of discovering an existing JTA transaction
+ * Only needs to be called at Region.Entry entry points e.g. Region.put, Region.invalidate, etc.
+ * @since tx
+ */
+ final public void discoverJTA() {
+ if (!isSecret() && !isUsedForPartitionedRegionAdmin()
+ && !isUsedForMetaRegion()) { // prevent internal regions from participating in a TX
+ getJTAEnlistedTX();
+ }
+ }
+
+ /**
+ * @return true if a transaction is in process
+ * @since tx
+ */
+ public final boolean isTX() {
+ return getTXState() != null;
+ }
+
+ /**
+ * @param expectedOldValue if this is non-null, only destroy if key exists
+ * and old value is equal to expectedOldValue
+ * @return true if a the destroy was done; false if it was not needed
+ */
+ final boolean mapDestroy(final EntryEventImpl event,
+ final boolean cacheWrite,
+ final boolean isEviction,
+ Object expectedOldValue)
+ throws CacheWriterException, EntryNotFoundException, TimeoutException {
+ final boolean inGII = lockGII();
+ try { // make sure unlockGII is called for bug 40001
+ return mapDestroy(event, cacheWrite, isEviction, expectedOldValue, inGII, false);
+ } finally {
+ if (inGII) {
+ unlockGII();
+ }
+ }
+ }
+
+ final boolean mapDestroy(final EntryEventImpl event,
+ final boolean cacheWrite,
+ final boolean isEviction,
+ Object expectedOldValue,
+ boolean needTokensForGII,
+ boolean removeRecoveredEntry) {
+ //When register interest is in progress ,
+ // We should not remove the key from the
+ // region and instead replace the value
+ // in the map with a DESTROYED token
+ final boolean inRI = !needTokensForGII
+ && !event.isFromRILocalDestroy()
+ && lockRIReadLock();
+ // at this point riCnt is guaranteed to be correct and we know for sure
+ // whether a RI is in progress and that riCnt will not change during this
+ // destroy operation
+ try {
+ final boolean needRIDestroyToken = inRI && (this.riCnt > 0);
+ final boolean inTokenMode = needTokensForGII || needRIDestroyToken;
+ // the following will call basicDestroyPart2 at the correct moment
+ boolean result = this.entries.destroy(event,
+ inTokenMode,
+ needRIDestroyToken,
+ cacheWrite,
+ isEviction,
+ expectedOldValue,
+ removeRecoveredEntry);
+ // AbstractRegionMap.destroy now calls cancelExpiryTask
+ return result;
+ } catch (ConcurrentCacheModificationException e) {
+ // this can happen in a client/server cache when another thread
+ // managed to slip in its version info to the region entry before this
+ // thread got around to doing so
+ if(logger.isDebugEnabled()) {
+ logger.debug("caught concurrent modification attempt when applying {}", event);
+ }
+ // Notify clients only if its NOT a gateway event.
+ if (event.getVersionTag() != null && !event.getVersionTag().isGatewayTag()) {
+ notifyBridgeClients(event);
+ }
+ return true; // event was elided
+ } catch(DiskAccessException dae) {
+ handleDiskAccessException(dae);
+ throw dae;
+ }
+ finally {
+ if (inRI) {
+ unlockRIReadLock();
+ }
+ }
+ }
+
+ /**
+ * Return true if dae was caused by a RegionDestroyedException.
+ * This was added for bug 39603.
+ */
+ static boolean causedByRDE(DiskAccessException dae) {
+ boolean result = false;
+ if (dae != null) {
+ Throwable cause = dae.getCause();
+ while (cause != null) {
+ if (cause instanceof RegionDestroyedException) {
+ result = true;
+ break;
+ }
+ cause = cause.getCause();
+ }
+ }
+ return result;
+ }
+
+ final public void handleDiskAccessException(DiskAccessException dae) {
+ handleDiskAccessException(dae, false);
+ }
+ //Asif:To Fix bug 39079, we are locally destroying the region, the
+ //destruction takes place here & not at DiskRegion or AbstractOplogDiskRegionEntry level
+ //is to eliminate any possibility of deadlocks ,as it is an entry operation thread
+ //which is implictly closing the region & stopping the Servers
+ /**
+ * @param dae DiskAccessException encountered by the thread
+ * @param duringInitialization indicates that this exception occurred during
+ * region initialization. Instead of closing the cache here, we rely on the
+ * region initialization to clean things up.
+ * @see DistributedRegion#initialize(InputStream, InternalDistributedMember, InternalRegionArguments)
+ * @see LocalRegion#initialize(InputStream, InternalDistributedMember, InternalRegionArguments)
+ * @see InitialImageOperation#processChunk
+ */
+ final public void handleDiskAccessException(DiskAccessException dae, boolean duringInitialization) {
+ // these will rethrow the originating exception
+ if (duringInitialization || causedByRDE(dae)) {
+ return;
+ }
+
+ // log the error
+ StringId sid = LocalizedStrings.LocalRegion_A_DISKACCESSEXCEPTION_HAS_OCCURED_WHILE_WRITING_TO_THE_DISK_FOR_REGION_0_THE_CACHE_WILL_BE_CLOSED;
+ logger.error(LocalizedMessage.create(sid, this.fullPath), dae);
+
+ // forward the error to the disk store
+ getDiskStore().handleDiskAccessException(dae);
+ }
+
+ void expireDestroy(final EntryEventImpl event,
+ final boolean cacheWrite) {
+ basicDestroy(event, cacheWrite, null);
+ }
+
+ void expireInvalidate(final EntryEventImpl event) {
+ basicInvalidate(event);
+ }
+
+ /**
+ * Creates an event for EVICT_DESTROY operations.
+ * It is intended that this method be overridden to allow for special
+ * handling of Partitioned Regions.
+ * @param key - the key that this event is related to
+ * @return an event for EVICT_DESTROY
+ */
+ protected EntryEventImpl generateEvictDestroyEvent(final Object key) {
+ EntryEventImpl event = EntryEventImpl.create(
+ this, Operation.EVICT_DESTROY, key, null/* newValue */,
+ null, false, getMyId());
+ // Fix for bug#36963
+ if (generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ event.setFetchFromHDFS(false);
+ return event;
+ }
+ protected EntryEventImpl generateCustomEvictDestroyEvent(final Object key) {
+ EntryEventImpl event = EntryEventImpl.create(
+ this, Operation.CUSTOM_EVICT_DESTROY, key, null/* newValue */,
+ null, false, getMyId());
+
+ // Fix for bug#36963
+ if (generateEventID()) {
+ event.setNewEventId(cache.getDistributedSystem());
+ }
+ event.setFetchFromHDFS(false);
+ return event;
+ }
+
+ /**
+ * @return true if the evict destroy was done; false if it was not needed
+ */
+ boolean evictDestroy(LRUEntry entry)
+ {
+
+ checkReadiness();
+ final EntryEventImpl event =
+ generateEvictDestroyEvent(entry.getKey());
+ try {
+ return mapDestroy(event,
+ false, // cacheWrite
+ true, // isEviction
+ null); // expectedOldValue
+ }
+ catch (CacheWriterException error) {
+ throw new Error(LocalizedStrings.LocalRegion_CACHE_WRITER_SHOULD_NOT_HAVE_BEEN_CALLED_FOR_EVICTDESTROY.toLocalizedString(), error);
+ }
+ catch (TimeoutException anotherError) {
+ throw new Error(LocalizedStrings.LocalRegion_NO_DISTRIBUTED_LOCK_SHOULD_HAVE_BEEN_ATTEMPTED_FOR_EVICTDESTROY.toLocalizedString(), anotherError);
+ }
+ catch (EntryNotFoundException yetAnotherError) {
+ throw new Error(LocalizedStrings.LocalRegion_ENTRYNOTFOUNDEXCEPTION_SHOULD_BE_MASKED_FOR_EVICTDESTROY.toLocalizedString(), yetAnotherError);
+ } finally {
+ event.release();
+ }
+ }
+
+ /**
+ * Called by lower levels {@link AbstractRegionMap} while holding the entry
+ * synchronization and while the entry remains in the map. Once
+ * the entry is removed from the map, then other operations synchronize on a
+ * new entry, allow for ordering problems between
+ * {@link #create(Object, Object, Object)} and
+ * {@link #destroy(Object, Object)} operations.
+ *
+ * @param entry the Region entry being destroyed
+ * @param event
+ * the event describing the destroy operation
+ * @since 5.1
+ */
+ protected void basicDestroyBeforeRemoval(RegionEntry entry, EntryEventImpl event)
+ {
+ }
+
+ /**
+ * Called by lower levels, while still holding the write sync lock, and the
+ * low level has completed its part of the basic destroy
+ */
+ void basicDestroyPart2(RegionEntry re, EntryEventImpl event,
+ boolean inTokenMode, boolean conflictWithClear, boolean duringRI, boolean invokeCallbacks)
+ {
+ if (!(this instanceof HARegion)) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("basicDestroyPart2(inTokenMode={},conflictWithClear={},duringRI={}) event={}",
+ inTokenMode, conflictWithClear, duringRI, event);
+ }
+ }
+ VersionTag v = event.getVersionTag();
+
+ /**
+ * destroys that are not part of the cleaning out of keys prior to a register-interest
+ * are marked with Tombstones instead of Destroyed tokens so that they are not
+ * reaped after the RI completes. RI does not create Tombstones because it
+ * would flood the TombstoneService with unnecessary work.
+ */
+ if (inTokenMode && !(this.concurrencyChecksEnabled || event.isFromRILocalDestroy())) {
+ if (re.isDestroyed()) {
+ getImageState().addDestroyedEntry(event.getKey());
+ if (!(this instanceof HARegion)) {
+ if (logger.isTraceEnabled()) {
+ logger.trace("basicDestroy: {}--> Token.DESTROYED", event.getKey());
+ }
+ }
+ }
+ }
+ else {
+ if (this.concurrencyChecksEnabled && !(this instanceof HARegion)) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("basicDestroyPart2: {}, version={}", event.getKey(), v);
+ }
+ }
+ }
+ /* this is too late to do index maintenance with a CompactRangeIndex
+ because we need to have the old value still intact. At this point
+ the old value has already be replaced with a destroyed token.
+ if (!isProxy() && !conflictWithClear) {
+ if (this.indexManager != null) {
+ try {
+ this.indexManager.updateIndexes(re, IndexManager.REMOVE_ENTRY);
+ }
+ catch (QueryException e) {
+ throw new IndexMaintenanceException(e);
+ }
+ }
+ }*/
+
+ notifyGatewaySender(EnumListenerEvent.AFTER_DESTROY, event);
+
+ // invoke callbacks if initialized and told to do so, or if this
+ // is a bucket in a partitioned region
+ if (invokeCallbacks && !event.isBulkOpInProgress()) {
+ if ((isInitialized() && (!inTokenMode || duringRI))
+ || this.isUsedForPartitionedRegionBucket) {
+ try {
+ re.dispatchListenerEvents(event);
+ }
+ catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ stopper.checkCancelInProgress(null);
+ return;
+ }
+ } else {
+ event.callbacksInvoked(true);
+ }
+ }
+ }
+
+ /**
+ * distribution and callback notification are done in part2 inside
+ * entry lock for maintaining the order of events.
+ */
+ void basicDestroyPart3(RegionEntry re, EntryEventImpl event,
+ boolean inTokenMode, boolean duringRI, boolean invokeCallbacks,
+ Object expectedOldValue) {
+ if (invokeCallbacks) {
+ if (event.isBulkOpInProgress()) {
+ event.getRemoveAllOperation().addEntry(event);
+ }
+ }
+ if (!inTokenMode || duringRI) {
+ updateStatsForDestroy();
+ }
+
+ if (this.entryUserAttributes != null) {
+ this.entryUserAttributes.remove(event.getKey());
+ }
+ }
+
+ /**
+ * Update stats
+ */
+ private void updateStatsForDestroy() {
+ getCachePerfStats().incDestroys();
+ }
+
+ //Asif : This method will clear the tranxnl entries
+ final void txClearRegion()
+ {
+ TXStateInterface tx = getJTAEnlistedTX();
+ if (tx != null) {
+ tx.rmRegion(this);
+ }
+ }
+
+ public void invokeDestroyCallbacks(final EnumListenerEvent eventType,
+ final EntryEventImpl event , final boolean callDispatchListenerEvent, boolean notifyGateways)
+ {
+ // The spec for ConcurrentMap support requires that operations be mapped
+ // to non-CM counterparts
+ if (event.getOperation() == Operation.REMOVE) {
+ event.setOperation(Operation.DESTROY);
+ }
+ event.setEventType(eventType);
+ notifyBridgeClients(event);
+ if (notifyGateways) {
+ notifyGatewaySender(eventType, event);
+ }
+ if(callDispatchListenerEvent){
+ dispatchListenerEvent(eventType, event);
+ }
+ }
+
+
+ public void invokeTXCallbacks(final EnumListenerEvent eventType,
+ final EntryEventImpl event , final boolean callDispatchListenerEvent)
+ {
+ // The spec for ConcurrentMap support requires that operations be mapped
+ // to non-CM counterparts
+
+ Operation op = event.getOperation();
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("invokeTXCallbacks for event {}", event);
+ }
+
+ if (op == Operation.REMOVE) {
+ event.setOperation(Operation.DESTROY);
+ } else if (op == Operation.PUT_IF_ABSENT) {
+ event.setOperation(Operation.CREATE);
+ } else if (op == Operation.REPLACE) {
+ event.setOperation(Operation.UPDATE);
+ }
+ event.setEventType(eventType);
+ notifyBridgeClients(event);
+ notifyGatewaySender(eventType, event);
+ if (callDispatchListenerEvent){
+ if (event.getInvokePRCallbacks() || (!(event.getRegion() instanceof PartitionedRegion) && !(event.getRegion().isUsedForPartitionedRegionBucket()))) {
+ dispatchListenerEvent(eventType, event);
+ }
+ }
+ }
+
+
+ /**
+ * @param key
+ * the key of the entry to destroy
+ * @param rmtOrigin
+ * true if transaction being applied had a remote origin
+ * @param event
+ * filled in if operation performed
+ * @param needTokensForGII
+ * true if caller has determined we are in destroy token mode and
+ * will keep us in that mode while this call is executing.
+ * @param filterRoutingInfo
+ * @param bridgeContext
+ * @param isOriginRemote whether the event originated in a peer or in this vm
+ * @param txEntryState for passing up versionTag - only on near side
+ * @param versionTag tag generated by txCoordinator - only on far side
+ * @param tailKey tail (shadow) key generated by txCoordinator for WAN - only on farside
+ */
+ final void txApplyDestroy(Object key, TransactionId rmtOrigin,
+ TXRmtEvent event, boolean needTokensForGII, Operation op,
+ EventID eventId, Object aCallbackArgument,List pendingCallbacks,
+ FilterRoutingInfo filterRoutingInfo, ClientProxyMembershipID bridgeContext,
+ boolean isOriginRemote, TXEntryState txEntryState, VersionTag versionTag, long tailKey)
+ {
+ final boolean inRI = !needTokensForGII && lockRIReadLock();
+ final boolean needRIDestroyToken = inRI && (this.riCnt > 0);
+ final boolean inTokenMode = needTokensForGII || needRIDestroyToken;
+
+ try {
+ this.entries.txApplyDestroy(key, rmtOrigin, event, inTokenMode, needRIDestroyToken,
+ op, eventId, aCallbackArgument,pendingCallbacks,filterRoutingInfo,bridgeContext, isOriginRemote, txEntryState, versionTag, tailKey);
+ } finally {
+ if (inRI) {
+ unlockRIReadLock();
+ }
+ }
+ }
+
+ /**
+ * Called by lower levels, while still holding the write sync lock, and the
+ * low level has completed its part of the basic destroy
+ */
+ void txApplyDestroyPart2(RegionEntry re, Object key, boolean inTokenMode, boolean clearConflict)
+ {
+ if (this.testCallable != null) {
+ this.testCallable.call(this, Operation.DESTROY, re);
+ }
+ if (inTokenMode) {
+ getImageState().addDestroyedEntry(key);
+ }
+ else {
+ updateStatsForDestroy();
+ }
+ if (this.entryUserAttributes != null) {
+ this.entryUserAttributes.remove(key);
+ }
+ }
+
+ /**
+ * @see DistributedRegion#basicInvalidateRegion(RegionEventImpl)
+ * @param event
+ */
+ void basicInvalidateRegion(RegionEventImpl event)
+ {
+ final TXStateProxy tx = this.cache.getTXMgr().internalSuspend();
+ try {
+ this.regionInvalid = true;
+ getImageState().setRegionInvalidated(true);
+ invalidateAllEntries(event);
+ Set allSubregions = subregions(true);
+ for (Iterator itr = allSubregions.iterator(); itr.hasNext();) {
+ LocalRegion rgn = (LocalRegion)itr.next();
+ rgn.regionInvalid = true;
+ try {
+ rgn.getImageState().setRegionInvalidated(true);
+ rgn.invalidateAllEntries(event);
+
+ if (!rgn.isInitialized())
+ continue; // don't invoke callbacks if not initialized yet
+
+ if (rgn.hasListener()) {
+ RegionEventImpl event2 = (RegionEventImpl)event.clone();
+ event2.region = rgn;
+ rgn.dispatchListenerEvent(
+ EnumListenerEvent.AFTER_REGION_INVALIDATE, event2);
+ }
+ }
+ catch (RegionDestroyedException ignore) {
+ // ignore subregions that have been destroyed to fix bug 33276
+ }
+ }
+
+ if (!isInitialized())
+ return;
+
+ event.setEventType(EnumListenerEvent.AFTER_REGION_INVALIDATE);
+ notifyBridgeClients(event);
+
+ boolean hasListener = hasListener();
+ if (logger.isDebugEnabled()) {
+ logger.debug("basicInvalidateRegion: hasListener = {}", hasListener);
+ }
+ if (hasListener) {
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REGION_INVALIDATE, event);
+ }
+ }
+ finally {
+ this.cache.getTXMgr().resume(tx);
+ }
+ }
+
+ /**
+ * Determines whether the receiver is unexpired with regard to the given
+ * timeToLive and idleTime attributes, which may different from this entry's
+ * actual attributes. Used for validation of objects during netSearch(), which
+ * must validate remote entries against local timeout attributes.
+ */
+ boolean isExpiredWithRegardTo(Object key, int ttl, int idleTime)
+ {
+
+ if (!getAttributes().getStatisticsEnabled())
+ return false;
+
+ long expTime;
+ try {
+ expTime = (new NetSearchExpirationCalculator(this, key, ttl, idleTime))
+ .getExpirationTime();
+ }
+ catch (EntryNotFoundException ex) {
+ return true;
+ }
+ if (expTime == 0)
+ return false;
+ return expTime <= cacheTimeMillis();
+ }
+
+ void dispatchListenerEvent(EnumListenerEvent op, InternalCacheEvent event)
+ {
+ // Return if the inhibit all notifications flag is set
+ boolean isEntryEvent = event instanceof EntryEventImpl;
+ if (isEntryEvent) {
+ if (((EntryEventImpl)event).inhibitAllNotifications()){
+ if (logger.isDebugEnabled()) {
+ logger.debug("Notification inhibited for key {}", event);
+ }
+ return;
+ }
+ }
+
+ if (shouldDispatchListenerEvent()) {
+ //Assert.assertTrue(event.getRegion() == this);
+ if (logger.isTraceEnabled()) {
+ logger.trace("dispatchListenerEvent event={}", event);
+ }
+ final long start = getCachePerfStats().startCacheListenerCall();
+
+ boolean origOriginRemote = false;
+ boolean isOriginRemoteSet = false;
+
+ try {
+ if ((isEntryEvent)) {
+ if (((EntryEventImpl)event).isSingleHop()) {
+ origOriginRemote = event.isOriginRemote();
+ ((EntryEventImpl)event).setOriginRemote(true);
+ isOriginRemoteSet = true;
+ }
+ RegionEntry re = ((EntryEventImpl) event).getRegionEntry();
+ if (re != null) {
+ ((EntryEventImpl) event).getRegionEntry()
+ .setCacheListenerInvocationInProgress(true);
+ }
+ }
+
+ if (!GemFireCacheImpl.ASYNC_EVENT_LISTENERS) {
+ dispatchEvent(this, event, op);
+ }
+ else {
+ final EventDispatcher ed = new EventDispatcher(event, op);
+ try {
+ this.cache.getEventThreadPool().execute(ed);
+ }
+ catch (RejectedExecutionException rex) {
+ ed.release();
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_0_EVENT_NOT_DISPATCHED_DUE_TO_REJECTED_EXECUTION), rex);
+ }
+ }
+ }
+ finally {
+ getCachePerfStats().endCacheListenerCall(start);
+ if (isOriginRemoteSet) {
+ ((EntryEventImpl)event).setOriginRemote(origOriginRemote);
+ }
+ if (isEntryEvent) {
+ RegionEntry re = ((EntryEventImpl) event).getRegionEntry();
+ if (re != null) {
+ re.setCacheListenerInvocationInProgress(false);
+ }
+ }
+ }
+ }
+ }
+
+ /** @return true if initialization is complete */
+ public boolean isInitialized()
+ {
+ if (this.initialized) {
+ return true;
+ }
+ else {
+ long count;
+ StoppableCountDownLatch latch = this.initializationLatchAfterGetInitialImage;
+ if (latch == null) {
+ return true;
+ }
+ count = latch.getCount();
+ if (count == 0) {
+ this.initialized = true;
+ return true;
+ }
+ else {
+ return false;
+ }
+ }
+ }
+
+ /**
+ * @return true if event state has been transfered to this region
+ * from another cache
+ */
+ public boolean isEventTrackerInitialized() {
+ if (this.eventTracker != null) {
+ return this.eventTracker.isInitialized();
+ }
+ return false;
+ }
+
+ /**
+ * @return true if this region has an event tracker
+ */
+ public boolean hasEventTracker() {
+ return (this.eventTracker != null);
+ }
+
+ public void acquireDestroyLock()
+ {
+ LocalRegion root = getRoot();
+ boolean acquired = false;
+ do {
+ this.cache.getCancelCriterion().checkCancelInProgress(null);
+ boolean interrupted = Thread.interrupted();
+ try {
+ root.destroyLock.acquire();
+ acquired = true;
+ }
+ catch (InterruptedException ie) {
+ interrupted = true;
+ this.cache.getCancelCriterion().checkCancelInProgress(ie);
+ }
+ finally {
+ if (interrupted) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ } while (!acquired);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Acquired Destroy Lock: {}", root);
+ }
+ }
+
+ public void releaseDestroyLock()
+ {
+ LocalRegion root = getRoot();
+ if (logger.isDebugEnabled()) {
+ logger.debug("Releasing Destroy Lock: {}", root.getName());
+ }
+ root.destroyLock.release();
+ }
+
+ /**
+ * Cleans up any resources that may have been allocated for this region during
+ * its initialization.
+ */
+ void cleanupFailedInitialization()
+ {
+ // mark as destroyed
+ // TODO OFFHEAP MERGE: to fix 49905 asif commented out isDestroyed being set.
+ // But in xd it was set after closeEntries was called.
+ // Here it is set before and it fixed 49555.
+ this.isDestroyed = true;
+ // after isDestroyed is set to true call removeResourceListener to fix bug 49555
+ this.cache.getResourceManager(false).removeResourceListener(this);
+ closeEntries(); //fixes bug 41333
+ this.destroyedSubregionSerialNumbers = collectSubregionSerialNumbers();
+ try {
+ if (this.eventTracker != null) {
+ this.eventTracker.stop();
+ }
+ if (this.diskRegion != null) {
+ // This was needed to fix bug 30937
+ try {
+ diskRegion.cleanupFailedInitialization(this);
+ }
+ catch (IllegalStateException ex) {
+ // just ignore this exception since whoever called us is going
+ // to report the exception that caused initialization to fail.
+ }
+ }
+ }
+ finally {
+ // make sure any waiters on initializing Latch are released
+ this.releaseLatches();
+ }
+ }
+
+ //////////////////// Private Methods ////////////////////////////////////////
+
+ LocalRegion getRoot()
+ {
+ LocalRegion r = this;
+ while (r.parentRegion != null) {
+ r = r.parentRegion;
+ }
+ return r;
+ }
+
+ private void initializationFailed(LocalRegion subregion)
+ {
+ synchronized (this.subregionsLock) { // bugfix for bug#34883 (tushar)
+ this.subregions.remove(subregion.getName());
+ }
+ subregion.cleanupFailedInitialization();
+ }
+
+ /**
+ * PRECONDITIONS: Synchronized on updateMonitor for this key in order to
+ * guarantee write-through to map entry, and key must be in map
+ *
+ * @param p_lastModified
+ * time, may be 0 in which case uses now instead
+ *
+ * @return the actual lastModifiedTime used.
+ */
+ long updateStatsForPut(RegionEntry entry, long p_lastModified,
+ boolean lruRecentUse)
+ {
+ long lastModified = p_lastModified;
+ if (lruRecentUse) {
+ entry.setRecentlyUsed(); // fix for bug 31102
+ }
+ if (lastModified == 0L) {
+ lastModified = cacheTimeMillis();
+ }
+ entry.updateStatsForPut(lastModified);
+ IndexManager.setIndexBufferTime(lastModified, cacheTimeMillis());
+ if (this.statisticsEnabled && !isProxy()) {
+ // do not reschedule if there is already a task in the queue.
+ // this prevents bloat in the TimerTask since cancelled tasks
+ // do not actually get removed from the TimerQueue.
+ // When the already existing task gets fired it checks to see
+ // if it is premature and if so reschedules a task at that time.
+ addExpiryTaskIfAbsent(entry);
+ }
+ // propagate to region
+ setLastModifiedTime(lastModified);
+ return lastModified;
+ }
+
+ /**
+ * Returns a region in the subregion map first, then looks in the
+ * reinitializing region registry.
+ *
+ * @return the region or null if not found, may be destroyed
+ */
+ private LocalRegion basicGetSubregion(String name)
+ {
+ LocalRegion r = toRegion(this.subregions.get(name));
+ // don't wait for reinitialization if the init_level for this thread is
+ // ANY_INIT: We don't want CreateRegion messages to wait on a future
+ // because it would cause a deadlock. If the region is ready for a
+ // CreateRegion message, it would have been in the subregions map.
+ if (r == null && threadInitLevelRequirement() != ANY_INIT) {
+ // try future
+ // Region p = this.parentRegion;
+ String thePath = getFullPath() + SEPARATOR + name;
+ if (logger.isDebugEnabled()) {
+ logger.debug("Trying reinitializing region, fullPath={}", thePath);
+ }
+ r = this.cache.getReinitializingRegion(thePath);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Reinitialized region is {}", r);
+ }
+ }
+ return r;
+ }
+
+ /**
+ * Make a LocalRegion from an element in the subregion map Sent to parent
+ * region.
+ *
+ * @return This method may return null or a destroyed region if the region was
+ * just destroyed
+ */
+ private LocalRegion toRegion(Object element)
+ {
+ LocalRegion rgn = (LocalRegion)element;
+ if (rgn != null) {
+ // do not return until done initializing (unless this is an initializing
+ // thread)
+ rgn.waitOnInitialization();
+ }
+ return rgn;
+ }
+
+ /**
+ * Update the API statistics appropriately for returning this value from get.
+ *
+ * @param re
+ * the entry whose value was accessed
+ */
+ public void updateStatsForGet(final RegionEntry re, final boolean hit) {
+ if (!this.statisticsEnabled) {
+ return;
+ }
+
+ final long now = cacheTimeMillis();
+ if (re != null) {
+ re.updateStatsForGet(hit, now);
+ if (isEntryIdleExpiryPossible()) {
+ addExpiryTaskIfAbsent(re);
+ }
+ }
+
+ // update region stats
+ setLastAccessedTime(now, hit);
+ }
+
+ private void sendPendingRegionDestroyEvents(HashSet set)
+ {
+ Iterator iterator = set.iterator();
+ while (iterator.hasNext()) {
+ RegionEventImpl event = (RegionEventImpl)iterator.next();
+ event.region.dispatchListenerEvent(
+ EnumListenerEvent.AFTER_REGION_DESTROY, event);
+ if (!cache.forcedDisconnect()) {
+ SystemMemberCacheEventProcessor.send(getCache(), event.getRegion(), event
+ .getOperation());
+ }
+ }
+ }
+
+ /** The listener is not closed until after the afterRegionDestroy event */
+ protected void closeCallbacksExceptListener()
+ {
+ closeCacheCallback(getCacheLoader());
+ closeCacheCallback(getCacheWriter());
+ closeCacheCallback(getEvictionController());
+ }
+
+ /** This is only done when the cache is closed. */
+ private void closeAllCallbacks()
+ {
+ closeCallbacksExceptListener();
+ CacheListener[] listeners = fetchCacheListenersField();
+ if (listeners != null) {
+ for (int i = 0; i < listeners.length; i++) {
+ closeCacheCallback(listeners[i]);
+ }
+ }
+ }
+
+ /**
+ * Release the client connection pool if we have one
+ * @since 5.7
+ */
+ private void detachPool() {
+ ServerRegionProxy mySRP = getServerProxy();
+ if (mySRP != null) {
+ GemFireCacheImpl gc = getCache();
+ String poolname =this.getPoolName();
+ PoolImpl dpool= (PoolImpl)PoolManager.find(this.getPoolName());
+ if(poolname!=null && dpool!=null){
+ mySRP.detach(gc.keepDurableSubscriptionsAlive() || dpool.getKeepAlive());
+ }else{
+ mySRP.detach(gc.keepDurableSubscriptionsAlive());
+ }
+ }
+ }
+
+ /**
+ * Closes the cqs created based on this region (Cache Client/writer/loader).
+ */
+ private void closeCqs() {
+ CqService cqService = getCache().getCqService();
+ if (cqService != null) {
+ try {
+ cqService.closeCqs(getFullPath());
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_EXCEPTION_OCCURRED_WHILE_CLOSING_CQS_ON_REGION_DESTORY), t);
+ }
+ }
+ }
+
+ /**
+ * Called when the cache is closed. Behaves just like a Region.close except
+ * the operation is CACHE_CLOSE
+ */
+ void handleCacheClose(Operation op)
+ {
+ RegionEventImpl ev = new RegionEventImpl(this, op, null, false, getMyId(),
+ generateEventID());
+ if (!this.isDestroyed) { // bruce: don't destroy if already destroyed
+ try {
+ basicDestroyRegion(ev, false, true, true);
+ }
+ catch (CancelException ignore) {
+ // If the region was destroyed we see this because the cache is closing.
+ // Since we are trying to close the cache don't get upset if
+ // a region was destroyed out from under us
+ if (logger.isDebugEnabled()) {
+ logger.debug("handleCacheClose: Encountered cache closure while closing region {}", getFullPath());
+ }
+ }
+ catch (RegionDestroyedException ignore) {
+ // Since we are trying to close the cache don't get upset if
+ // a region was destroyed out from under us
+ }
+ catch (CacheWriterException e) {
+ // not possible with local operation, CacheWriter not called
+ throw new Error(LocalizedStrings.LocalRegion_CACHEWRITEREXCEPTION_SHOULD_NOT_BE_THROWN_HERE.toLocalizedString(), e);
+ }
+ catch (TimeoutException e) {
+ // not possible with local operation, no distributed locks possible
+ InternalDistributedSystem ids = (this.getCache().getDistributedSystem());
+ if (!ids.isDisconnecting()) {
+ throw new InternalGemFireError(LocalizedStrings.LocalRegion_TIMEOUTEXCEPTION_SHOULD_NOT_BE_THROWN_HERE.toLocalizedString(), e);
+ }
+ }
+ }
+ }
+ void cleanUpOnIncompleteOp(EntryEventImpl event, RegionEntry re,
+ boolean eventRecorded, boolean updateStats, boolean isReplace) {
+ //TODO:Asif: This is incorrect implementation for replicated region in case of
+ //sql fabric, as sqlf index would already be updated, if eventRecorded
+ //flag is true.So if entry is being removed ,
+ //then the sqlfindex also needs to be corrected
+ IndexUpdater iu = this.getIndexUpdater(); // sqlf system
+ if(!eventRecorded || iu ==null || isReplace) {
+ //Ok to remove entry whether sqlfabric or gfe as index has not been modified yet by the operation
+ this.entries.removeEntry(event.getKey(), re, updateStats) ;
+ }else {
+ // a sqlf system, with event recorded as true. we need to update index.
+ //Use the current event to indicate destroy.should be ok
+ Operation oldOp = event.getOperation();
+ event.setOperation(Operation.DESTROY);
+ this.entries.removeEntry(event.getKey(), re, updateStats, event, this, iu);
+ event.setOperation(oldOp);
+ }
+
+ }
+
+ static void validateRegionName(String name)
+ {
+ if (name == null) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_NAME_CANNOT_BE_NULL.toLocalizedString());
+ }
+ if (name.length() == 0) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_NAME_CANNOT_BE_EMPTY.toLocalizedString());
+ }
+ if (name.indexOf(SEPARATOR) >= 0) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_NAME_CANNOT_CONTAIN_THE_SEPARATOR_0.toLocalizedString(SEPARATOR));
+ }
+ }
+
+ private void checkCacheClosed()
+ {
+ if (this.cache.isClosed()) {
+ throw cache.getCacheClosedException(null, null);
+ }
+ }
+
+ private void checkRegionDestroyed(boolean checkCancel)
+ {
+ if (checkCancel) {
+ this.cache.getCancelCriterion().checkCancelInProgress(null);
+ }
+ if (this.isDestroyed) {
+ RegionDestroyedException ex;
+ if (this.reinitialized_old) {
+ ex = new RegionReinitializedException(toString(), getFullPath());
+ } else if (this.cache.isCacheAtShutdownAll()) {
+ throw new CacheClosedException("Cache is being closed by ShutdownAll");
+ }
+ else {
+ ex = new RegionDestroyedException(toString(), getFullPath());
+ }
+ // Race condition could cause the cache to be destroyed after the
+ // cache close check above, so we need to re-check before throwing.
+ if (checkCancel) {
+ this.cache.getCancelCriterion().checkCancelInProgress(null);
+ }
+ throw ex;
+ }
+
+ if (this.isDestroyedForParallelWAN) {
+ throw new RegionDestroyedException(
+ LocalizedStrings.LocalRegion_REGION_IS_BEING_DESTROYED_WAITING_FOR_PARALLEL_QUEUE_TO_DRAIN
+ .toLocalizedString(), getFullPath());
+ }
+ }
+
+ /**
+ * For each region entry in this region call the callback
+ * @since prPersistSprint2
+ */
+ public void foreachRegionEntry(RegionEntryCallback callback) {
+ Iterator it = this.entries.regionEntriesInVM().iterator();
+ while (it.hasNext()) {
+ callback.handleRegionEntry((RegionEntry)it.next());
+ }
+ }
+
+ /**
+ * Used by {@link #foreachRegionEntry}.
+ * @since prPersistSprint2
+ */
+ public interface RegionEntryCallback {
+ public void handleRegionEntry(RegionEntry re);
+ }
+
+ protected void checkIfReplicatedAndLocalDestroy(EntryEventImpl event) {
+ // Actiual: disallow local invalidation for replicated regions
+ if (getScope().isDistributed() && getDataPolicy().withReplication()
+ && (!event.isDistributed()) && !isUsedForSerialGatewaySenderQueue()) {
+ throw new IllegalStateException(LocalizedStrings.LocalRegion_NOT_ALLOWED_TO_DO_A_LOCAL_DESTROY_ON_A_REPLICATED_REGION.toLocalizedString());
+ }
+ }
+
+ /**
+ * Return the number of subregions, including this region. Used for recursive
+ * size calculation in SubregionsSet.size
+ */
+ protected int allSubregionsSize()
+ {
+ int sz = 1; /* 1 for this region */
+ for (Iterator itr = this.subregions.values().iterator(); itr.hasNext();) {
+ LocalRegion r = (LocalRegion)itr.next();
+ if (r != null && r.isInitialized() && !r.isDestroyed()) {
+ sz += r.allSubregionsSize();
+ }
+ }
+ return sz;
+ }
+
+ /**
+ * Return the number of entries including in subregions. Used for recursive
+ * size calculation in EntriesSet.size. This does not include tombstone
+ * entries stored in the region.
+ */
+ protected int allEntriesSize()
+ {
+ int sz = entryCount();
+ for (Iterator itr = this.subregions.values().iterator(); itr.hasNext();) {
+ LocalRegion r = toRegion(itr.next());
+ if (r != null && !r.isDestroyed()) {
+ sz += r.allEntriesSize();
+ }
+ }
+ return sz;
+ }
+
+ /**
+ * @param rgnEvent
+ * the RegionEvent for region invalidation
+ */
+ protected void invalidateAllEntries(RegionEvent rgnEvent)
+ {
+ Operation op = Operation.LOCAL_INVALIDATE;
+ if (rgnEvent.getOperation().isDistributed()) {
+ op = Operation.INVALIDATE;
+ }
+
+
+ // if this is a local invalidation, then set local invalid flag on event
+ // so LOCAL_INVALID tokens is used (even though each individual entry
+ // invalidation is not distributed).
+
+
+ // region operation so it is ok to ignore tx state
+ for (Iterator itr = keySet().iterator(); itr.hasNext();) {
+ try {
+ //EventID will not be generated by this constructor
+ EntryEventImpl event = EntryEventImpl.create(
+ this, op, itr.next() /*key*/,
+ null/* newValue */, null/* callbackArg */, rgnEvent.isOriginRemote(),
+ rgnEvent.getDistributedMember());
+ try {
+ event.setLocalInvalid(!rgnEvent.getOperation().isDistributed());
+ basicInvalidate(event, false);
+ } finally {
+ event.release();
+ }
+ }
+ catch (EntryNotFoundException e) {
+ // ignore
+ }
+ }
+ }
+
+ boolean hasListener()
+ {
+ CacheListener[] listeners = fetchCacheListenersField();
+ return listeners != null && listeners.length > 0;
+ }
+
+ private final DiskStoreImpl dsi;
+ public DiskStoreImpl getDiskStore() {
+ return this.dsi;
+ }
+
+ /**
+ * Return true if all disk attributes are defaults.
+ * DWA.isSynchronous can be true or false.
+ */
+ private boolean useDefaultDiskStore() {
+ assert(getDiskStoreName()== null);
+ if (!Arrays.equals(getDiskDirs(), DiskStoreFactory.DEFAULT_DISK_DIRS)) {
+ return false;
+ }
+ if (!Arrays.equals(getDiskDirSizes(), DiskStoreFactory.DEFAULT_DISK_DIR_SIZES)) {
+ return false;
+ }
+ DiskWriteAttributesFactory dwf = new DiskWriteAttributesFactory();
+ dwf.setSynchronous(false);
+ if (dwf.create().equals(getDiskWriteAttributes())) {
+ return true;
+ }
+ dwf.setSynchronous(true);
+ if (dwf.create().equals(getDiskWriteAttributes())) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Returns true if this region's config indicates that it will use a disk store.
+ * Added for bug 42055.
+ */
+ protected boolean usesDiskStore(RegionAttributes ra) {
+ return !isProxy()
+ && (getAttributes().getDataPolicy().withPersistence()
+ || isOverflowEnabled());
+ }
+
+ protected DiskStoreImpl findDiskStore(RegionAttributes ra, InternalRegionArguments internalRegionArgs) {
+ //validate that persistent type registry is persistent
+ if(getAttributes().getDataPolicy().withPersistence()) {
+ getCache().getPdxRegistry().creatingPersistentRegion();
+ }
+
+ if (usesDiskStore(ra)) {
+ if (getDiskStoreName() != null) {
+ DiskStoreImpl diskStore = (DiskStoreImpl)getGemFireCache().findDiskStore(getDiskStoreName());
+ if (diskStore == null) {
+ throw new IllegalStateException(LocalizedStrings.CacheCreation_DISKSTORE_NOTFOUND_0.toLocalizedString(getDiskStoreName()));
+ }
+ return diskStore;
+ }
+ else if (useDefaultDiskStore()){
+ return getGemFireCache().getOrCreateDefaultDiskStore();
+ } else /* backwards compat mode */{
+ DiskStoreFactory dsf = getGemFireCache().createDiskStoreFactory();
+ dsf.setDiskDirsAndSizes(getDiskDirs(), getDiskDirSizes());
+ DiskWriteAttributes dwa = getDiskWriteAttributes();
+ dsf.setAutoCompact(dwa.isRollOplogs());
+ dsf.setMaxOplogSize(dwa.getMaxOplogSize());
+ dsf.setTimeInterval(dwa.getTimeInterval());
+ if (dwa.getBytesThreshold() > 0) {
+ dsf.setQueueSize(1);
+ } else {
+ dsf.setQueueSize(0);
+ }
+ DiskStoreFactoryImpl dsfi = (DiskStoreFactoryImpl)dsf;
+ return dsfi.createOwnedByRegion(getFullPath().replace('/', '_'),
+ this instanceof PartitionedRegion, internalRegionArgs);
+ }
+ }
+
+ return null;
+ }
+ /**
+ * Creates a new DiskRegion
for this region. We assume that the
+ * attributes and the name of the region have been set.
+ *
+ * @return null
is a disk region is not desired
+ *
+ * @since 3.2
+ */
+ protected DiskRegion createDiskRegion(InternalRegionArguments internalRegionArgs)
+ throws DiskAccessException {
+ if (internalRegionArgs.getDiskRegion() != null) {
+ DiskRegion dr = internalRegionArgs.getDiskRegion();
+ dr.createDataStorage();
+ return dr;
+ }
+ // A Proxy inherently has no storage.
+ if (dsi != null) {
+ DiskRegionStats stats;
+ if (this instanceof BucketRegion) {
+ stats = internalRegionArgs.getPartitionedRegion().getDiskRegionStats();
+ } else {
+ stats = new DiskRegionStats(getCache().getDistributedSystem(), getFullPath());
+ }
+
+ EnumSet diskFlags = EnumSet.noneOf(DiskRegionFlag.class);
+ // Add flag if this region has versioning enabled
+ if(this.getAttributes().getConcurrencyChecksEnabled()) {
+ diskFlags.add(DiskRegionFlag.IS_WITH_VERSIONING);
+ }
+ return DiskRegion.create(dsi, getFullPath(), false,
+ getDataPolicy().withPersistence(),
+ isOverflowEnabled(), isDiskSynchronous(),
+ stats, getCancelCriterion(), this, getAttributes(),
+ diskFlags, "NO_PARTITITON", -1,
+ getCompressor(), getOffHeap());
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Returns the object sizer on this region or null if it has no sizer.
+ * @since 6.1.2.9
+ */
+ public ObjectSizer getObjectSizer() {
+ ObjectSizer result = null;
+ EvictionAttributes ea = getEvictionAttributes();
+ if (ea != null) {
+ result = ea.getObjectSizer();
+ }
+ return result;
+ }
+
+ /** ************************ Expiration methods ******************************* */
+
+ /**
+ * Add the Region TTL expiry task to the scheduler
+ */
+ void addTTLExpiryTask()
+ {
+ synchronized (this.regionExpiryLock) {
+ RegionTTLExpiryTask task = this.regionTTLExpiryTask;
+ if (task != null) {
+ task.cancel();
+ }
+ if (this.regionTimeToLive > 0) {
+ this.regionTTLExpiryTask = (RegionTTLExpiryTask)
+ this.cache.getExpirationScheduler().addExpiryTask(
+ new RegionTTLExpiryTask(this));
+ if (this.regionTTLExpiryTask != null) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Initialized Region TTL Expiry Task {}", this.regionTTLExpiryTask);
+ }
+ }
+ } else {
+ this.regionTTLExpiryTask = null;
+ }
+ }
+ }
+
+ void addTTLExpiryTask(RegionTTLExpiryTask callingTask)
+ {
+ synchronized (this.regionExpiryLock) {
+ if (this.regionTTLExpiryTask != null && this.regionTTLExpiryTask != callingTask) {
+ return;
+ }
+ if (this.regionTimeToLive <= 0) {
+ this.regionTTLExpiryTask = null;
+ return;
+ }
+ RegionTTLExpiryTask task = new RegionTTLExpiryTask(this);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Scheduling Region TTL Expiry Task {} which replaces {}", task, this.regionTTLExpiryTask);
+ }
+ this.regionTTLExpiryTask = (RegionTTLExpiryTask)
+ this.cache.getExpirationScheduler().addExpiryTask(task);
+ }
+ }
+
+ /**
+ * Add the Region Idle expiry task to the scheduler
+ */
+ final void addIdleExpiryTask()
+ {
+ synchronized (this.regionExpiryLock) {
+ RegionIdleExpiryTask task = this.regionIdleExpiryTask;
+ if (task != null) {
+ task.cancel();
+ }
+ if (this.regionIdleTimeout > 0) {
+ this.regionIdleExpiryTask = (RegionIdleExpiryTask)
+ this.cache.getExpirationScheduler().addExpiryTask(
+ new RegionIdleExpiryTask(this));
+ if (this.regionIdleExpiryTask != null) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Initialized Region Idle Expiry Task {}", this.regionIdleExpiryTask);
+ }
+ }
+ } else {
+ this.regionIdleExpiryTask = null;
+ }
+ }
+ }
+
+ void addIdleExpiryTask(RegionIdleExpiryTask callingTask)
+ {
+ synchronized (this.regionExpiryLock) {
+ if (this.regionIdleExpiryTask != null && this.regionIdleExpiryTask != callingTask) {
+ return;
+ }
+ if (this.regionIdleTimeout <= 0) {
+ this.regionIdleExpiryTask = null;
+ return;
+ }
+ RegionIdleExpiryTask task = new RegionIdleExpiryTask(this);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Scheduling Region Idle Expiry Task {} which replaces ", task, this.regionIdleExpiryTask);
+ }
+ this.regionIdleExpiryTask = (RegionIdleExpiryTask)
+ this.cache.getExpirationScheduler().addExpiryTask(task);
+ }
+ }
+
+ /**
+ * Added to fix bug 31204
+ */
+ protected boolean isEntryIdleExpiryPossible()
+ {
+ return this.entryIdleTimeout > 0 || this.customEntryIdleTimeout != null;
+ }
+
+ private void cancelTTLExpiryTask()
+ {
+ RegionTTLExpiryTask task;
+ synchronized (this.regionExpiryLock) {
+ task = this.regionTTLExpiryTask;
+ if (task != null) {
+ this.regionTTLExpiryTask = null;
+ }
+ }
+ if (task != null) {
+ task.cancel();
+ }
+ }
+
+ private void cancelIdleExpiryTask()
+ {
+ RegionIdleExpiryTask task;
+ synchronized (this.regionExpiryLock) {
+ task = this.regionIdleExpiryTask;
+ if (task != null) {
+ this.regionIdleExpiryTask = null;
+ }
+ }
+ if (task != null) {
+ task.cancel();
+ }
+ }
+
+ @Override
+ protected void regionTimeToLiveChanged(ExpirationAttributes oldTimeToLive)
+ {
+ addTTLExpiryTask();
+ }
+
+ @Override
+ protected void regionIdleTimeoutChanged(ExpirationAttributes oldIdleTimeout)
+ {
+ addIdleExpiryTask();
+ }
+
+ @Override
+ protected void timeToLiveChanged(ExpirationAttributes oldTimeToLive)
+ {
+ int oldTimeout = oldTimeToLive.getTimeout();
+ if (customEntryTimeToLive != null) {
+ rescheduleEntryExpiryTasks();
+ }
+ else
+ if (entryTimeToLive > 0
+ && (oldTimeout == 0 || entryTimeToLive < oldTimeout)) {
+ rescheduleEntryExpiryTasks();
+ }
+ else {
+ // It's safe to let them get rescheduled lazily, as the old expiration
+ // time will cause the tasks to fire sooner than the new ones.
+ }
+ }
+
+ @Override
+ protected void idleTimeoutChanged(ExpirationAttributes oldIdleTimeout)
+ {
+ int oldTimeout = oldIdleTimeout.getTimeout();
+ if (customEntryIdleTimeout != null) {
+ rescheduleEntryExpiryTasks();
+ }
+ else
+ if (entryIdleTimeout > 0
+ && (oldTimeout == 0 || entryIdleTimeout < oldTimeout)) {
+ rescheduleEntryExpiryTasks();
+ }
+ else {
+ // It's safe to let them get rescheduled lazily, as the old expiration
+ // time will cause the tasks to fire sooner than the new ones.
+ }
+ }
+
+ protected void rescheduleEntryExpiryTasks()
+ {
+ if (isProxy()) {
+ return;
+ }
+ if (!isInitialized()) {
+ return; // don't schedule expiration until region is initialized (bug
+ }
+ // OK to ignore transaction since Expiry only done non-tran
+ Iterator it = this.entries.regionEntries().iterator();
+ if (it.hasNext()) {
+ try {
+ if (isEntryExpiryPossible()) {
+ ExpiryTask.setNow();
+ }
+ while (it.hasNext()) {
+ addExpiryTask(it.next());
+ }
+ } finally {
+ ExpiryTask.clearNow();
+ }
+ }
+ }
+
+ void addExpiryTaskIfAbsent(RegionEntry re)
+ {
+ addExpiryTask(re, true);
+ }
+
+ void addExpiryTask(RegionEntry re)
+ {
+ addExpiryTask(re, false);
+ }
+
+ /**
+ * Used to create a cheap Region.Entry that can be passed to the CustomExpiry callback
+ *
+ */
+ private static class ExpiryRegionEntry implements Region.Entry {
+ private final LocalRegion region;
+ private final RegionEntry re;
+
+ public ExpiryRegionEntry(LocalRegion lr, RegionEntry re) {
+ this.region = lr;
+ this.re = re;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((re == null) ? 0 : re.hashCode());
+ result = prime * result + ((region == null) ? 0 : region.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ExpiryRegionEntry other = (ExpiryRegionEntry) obj;
+ if (re == null) {
+ if (other.re != null)
+ return false;
+ } else if (!re.equals(other.re))
+ return false;
+ if (region == null) {
+ if (other.region != null)
+ return false;
+ } else if (!region.equals(other.region))
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "region=" + region.getFullPath() + ", key=" + getKey() + " value=" + getValue();
+ }
+
+ @Override
+ public Region getRegion() {
+ return this.region;
+ }
+
+ /**
+ * Returns the entry's RegionEntry if it "checks" out. The check is to
+ * see if the region entry still exists.
+ * @throws EntryNotFoundException if the RegionEntry has been removed.
+ */
+ private RegionEntry getCheckedRegionEntry() throws EntryNotFoundException {
+ RegionEntry result = this.re;
+ if (re.isDestroyedOrRemoved()) {
+ throw new EntryNotFoundException("Entry for key " + re.getKey() + " no longer exists");
+ }
+ return result;
+ }
+
+ @Override
+ public Object getValue() {
+ Object value = this.region.getDeserialized(getCheckedRegionEntry(), false, false, false, false);
+ if (value == null) {
+ throw new EntryDestroyedException(getKey().toString());
+ }
+ else if(Token.isInvalid(value)) {
+ return null;
+ }
+ return value;
+ }
+
+ @Override
+ public boolean isLocal() {
+ return true; // we only create expiry tasks for local entries
+ }
+
+ @Override
+ public CacheStatistics getStatistics() {
+ LocalRegion lr = this.region;
+ if (!lr.statisticsEnabled) {
+ throw new StatisticsDisabledException(LocalizedStrings.LocalRegion_STATISTICS_DISABLED_FOR_REGION_0.toLocalizedString(lr.getFullPath()));
+ }
+ return new CacheStatisticsImpl(getCheckedRegionEntry(), lr);
+ }
+
+ @Override
+ public Object getUserAttribute() {
+ Map userAttr = this.region.entryUserAttributes;
+ if (userAttr == null) {
+ return null;
+ }
+ return userAttr.get(getKey());
+ }
+
+ @Override
+ public Object setUserAttribute(Object userAttribute) {
+ LocalRegion lr = this.region;
+ if (lr.entryUserAttributes == null) {
+ lr.entryUserAttributes = new Hashtable();
+ }
+ return lr.entryUserAttributes.put(getKey(), userAttribute);
+ }
+
+ @Override
+ public boolean isDestroyed() {
+ if (this.re.isDestroyedOrRemoved()) return true;
+ return false;
+ }
+
+ @Override
+ public Object setValue(Object value) {
+ return this.region.put(getKey(), value);
+ }
+
+ @Override
+ public Object getKey() {
+ return this.re.getKey();
+ }
+ }
+ /**
+ * If custom expiration returns non-null expiration attributes
+ * then create a CustomEntryExpiryTask for this region and the given entry and return it.
+ * Otherwise if the region is configured for expiration
+ * then create an EntryExpiryTask for this region and the given entry and return it.
+ * Null is returned if the expiration attributes indicate that expiration is disabled.
+ */
+ private EntryExpiryTask createExpiryTask(RegionEntry re) {
+ if (re == null || re.isDestroyedOrRemoved()) {
+ return null;
+ }
+ if (this.customEntryIdleTimeout != null || this.customEntryTimeToLive != null) {
+ ExpiryRegionEntry ere = new ExpiryRegionEntry(this, re);
+ ExpirationAttributes ttlAtts = null;
+ ExpirationAttributes idleAtts = null;
+ final RegionAttributes,?> ra = this.getAttributes();
+ {
+ final CustomExpiry,?> customTTL = ra.getCustomEntryTimeToLive();
+ if (customTTL != null) {
+ try {
+ ttlAtts = customTTL.getExpiry(ere);
+ if (ttlAtts != null) {
+ this.checkEntryTimeoutAction("timeToLive", ttlAtts.getAction());
+ }
+ }
+ catch (RegionDestroyedException rde) {
+ // Ignore - #42273
+ }
+ catch (EntryNotFoundException enfe){
+ //Ignore - #51933
+ }
+ catch (EntryDestroyedException edf){
+ //Ignore - #51933
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.EntryExpiryTask_ERROR_CALCULATING_EXPIRATION_0,
+ e.getMessage()), e);
+ }
+ }
+ if (ttlAtts == null) {
+ ttlAtts = ra.getEntryTimeToLive();
+ }
+ }
+ {
+ CustomExpiry,?> customIdle = ra.getCustomEntryIdleTimeout();
+ if (customIdle != null) {
+ try {
+ idleAtts = customIdle.getExpiry(ere);
+ if (idleAtts != null) {
+ this.checkEntryTimeoutAction("idleTimeout", idleAtts.getAction());
+ }
+ }
+ catch (RegionDestroyedException rde) {
+ // Ignore - #42273
+ }
+ catch (EntryNotFoundException enfe){
+ //Ignore - #51933
+ }
+ catch (EntryDestroyedException edf){
+ //Ignore - #51933
+ }
+ catch (Exception e) {
+ logger.fatal(LocalizedMessage.create(LocalizedStrings.EntryExpiryTask_ERROR_CALCULATING_EXPIRATION_0,
+ e.getMessage()), e);
+ }
+ }
+ if (idleAtts == null) {
+ idleAtts = ra.getEntryIdleTimeout();
+ }
+ }
+ final boolean ttlDisabled = ttlAtts == null || ttlAtts.getTimeout() == 0;
+ final boolean idleDisabled = idleAtts == null || idleAtts.getTimeout() == 0;
+ if (ttlDisabled && idleDisabled) {
+ return null;
+ } else if ((ttlDisabled || ttlAtts.equals(ra.getEntryTimeToLive()))
+ && (idleDisabled || idleAtts.equals(ra.getEntryIdleTimeout()))) {
+ // no need for custom since we can just use the region's expiration attributes.
+ return new EntryExpiryTask(this, re);
+ } else {
+ return new CustomEntryExpiryTask(this, re, ttlAtts, idleAtts);
+ }
+ } else if (isEntryExpiryPossible()) {
+ return new EntryExpiryTask(this, re);
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Used by unit tests to get access to the EntryExpiryTask
+ * of the given key. Returns null if the entry exists but
+ * does not have an expiry task.
+ * @throws EntryNotFoundException if no entry exists key.
+ */
+ public EntryExpiryTask getEntryExpiryTask(Object key) {
+ RegionEntry re = this.getRegionEntry(key);
+ if (re == null) {
+ throw new EntryNotFoundException("Entry for key " + key + " does not exist.");
+ }
+ return this.entryExpiryTasks.get(re);
+ }
+ /**
+ * Used by unit tests to get access to the RegionIdleExpiryTask
+ * of this region. Returns null if no task exists.
+ */
+ public RegionIdleExpiryTask getRegionIdleExpiryTask() {
+ return this.regionIdleExpiryTask;
+ }
+ /**
+ * Used by unit tests to get access to the RegionTTLExpiryTask
+ * of this region. Returns null if no task exists.
+ */
+ public RegionTTLExpiryTask getRegionTTLExpiryTask() {
+ return this.regionTTLExpiryTask;
+ }
+
+ private void addExpiryTask(RegionEntry re, boolean ifAbsent)
+ {
+ if (isProxy()) {
+ return;
+ }
+ if (!isInitialized()) {
+ return; // don't schedule expiration until region is initialized (bug
+ // 35214)
+ }
+ if (isEntryExpiryPossible()) {
+ EntryExpiryTask newTask = null;
+ EntryExpiryTask oldTask = null;
+ if (ifAbsent) {
+ oldTask = (EntryExpiryTask)this.entryExpiryTasks.get(re);
+ if (oldTask != null) {
+ boolean keepOldTask = true;
+ if (this.customEntryIdleTimeout != null || this.customEntryTimeToLive != null) {
+ newTask = createExpiryTask(re);
+ if (newTask == null) {
+ cancelExpiryTask(re); // cancel any old task
+ return;
+ }
+ // to fix bug 44418 see if the new tasks expiration would be earlier than
+ // the scheduled task.
+ long ntTime = newTask.getExpirationTime();
+ try{
+ if (ntTime != 0 && ntTime < oldTask.getExpirationTime()) {
+ // it is so get rid of the old task and schedule the new one.
+ keepOldTask = false;
+ }
+ }catch(EntryNotFoundException ex){
+ keepOldTask=false;
+ }
+ }
+ if (keepOldTask) {
+ // if an oldTask is present leave it be
+ if (logger.isTraceEnabled()) {
+ logger.trace("Expiry Task not added because one already present. Key={}" + re.getKey());
+ }
+ return;
+ }
+ }
+ }
+ if (newTask == null) {
+ newTask = createExpiryTask(re);
+ if (newTask == null) {
+ cancelExpiryTask(re); // cancel any old task
+ return;
+ }
+ }
+ oldTask = this.entryExpiryTasks.put(re, newTask);
+ ExpirationScheduler es = this.cache.getExpirationScheduler();
+ if (oldTask != null) {
+ if (oldTask.cancel()) {
+ es.incCancels();
+ }
+ }
+ if (!es.addEntryExpiryTask(newTask)) {
+ this.entryExpiryTasks.remove(re);
+ }
+ // @todo darrel: merge question: should we catch EntryNotFoundException
+ // if addExpiryTask throws it?
+ // } catch (EntryNotFoundException e) {
+ // // ignore - there are unsynchronized paths that allow an entry to
+ // // be destroyed out from under us.
+ // return;
+ // }
+ }
+ else {
+ cancelExpiryTask(re);
+ if (logger.isTraceEnabled()) {
+ logger.trace("addExpiryTask(key) ignored");
+ }
+ }
+ }
+
+ void cancelExpiryTask(RegionEntry re)
+ {
+ EntryExpiryTask oldTask = this.entryExpiryTasks.remove(re);
+ if (oldTask != null) {
+ if (oldTask.cancel()) {
+ this.cache.getExpirationScheduler().incCancels();
+ }
+ }
+ }
+
+ public void cancelAllEntryExpiryTasks()
+ {
+ // This method gets called during LocalRegion construction
+ // in which case the final entryExpiryTasks field can still be null
+ if (this.entryExpiryTasks == null) return;
+ if (this.entryExpiryTasks.isEmpty()) return;
+ boolean doPurge = false;
+ Iterator tasksIter = this.entryExpiryTasks.values().iterator();
+ while (tasksIter.hasNext()) {
+ EntryExpiryTask task = tasksIter.next();
+ task.cancel(); // no need to call incCancels since we will call forcePurge
+ doPurge = true;
+ }
+ if (doPurge) {
+ // do a force to not leave any refs to this region
+ this.cache.getExpirationScheduler().forcePurge();
+ }
+ }
+
+ /**
+ * Used internally by EntryExpiryTask. Ok for it to ignore transaction.
+ *
+ * @return 0 if statistics not available
+ */
+ long getLastAccessedTime(Object key) throws EntryNotFoundException
+ {
+ RegionEntry entry = this.entries.getEntry(key);
+ if (entry == null)
+ throw new EntryNotFoundException(key.toString());
+ try {
+ return entry.getLastAccessed();
+ }
+ catch (InternalStatisticsDisabledException e) {
+ return 0;
+ }
+ }
+
+ /**
+ * Used internally by EntryExpiryTask. Ok for it to ignore transaction.
+ */
+ long getLastModifiedTime(Object key) throws EntryNotFoundException
+ {
+ RegionEntry entry = this.entries.getEntry(key);
+ if (entry == null)
+ throw new EntryNotFoundException(key.toString());
+ return entry.getLastModified();
+ }
+
+ /**
+ * get the ImageState for this region
+ */
+ protected final ImageState getImageState()
+ {
+ return this.imageState;
+ }
+
+ /**
+ * Callers of this method should always follow the call with: if (lockGII()) {
+ * try { } finally { unlockGII(); } }
+ *
+ * @return true if lock obtained and unlock needs to be called
+ */
+ boolean lockGII() {
+ ImageState is = getImageState();
+ if (is.isReplicate() && !isInitialized()) {
+ is.lockGII();
+ // recheck initialized while holding lock
+ if (isInitialized()) {
+ // we didn't need to lock after all so clear and return false
+ is.unlockGII();
+ } else {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ void unlockGII() {
+ ImageState is = getImageState();
+ assert is.isReplicate();
+ is.unlockGII();
+ }
+
+ /**
+ * Callers of this method should always follow the call with: if (lockRIReadLock()) {
+ * try { } finally { unlockRIReadLock(); } }
+ *
+ * @return true if lock obtained and unlock needs to be called
+ */
+ private boolean lockRIReadLock() {
+ if (getImageState().isClient()) {
+ getImageState().readLockRI();
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ private void unlockRIReadLock() {
+ assert getImageState().isClient();
+ getImageState().readUnlockRI();
+ }
+
+ /** doesn't throw RegionDestroyedException, used by CacheDistributionAdvisor */
+ public LocalRegion basicGetParentRegion()
+ {
+ return this.parentRegion;
+ }
+
+ Object basicGetEntryUserAttribute(Object entryKey)
+ {
+ Map userAttr = this.entryUserAttributes;
+ if (userAttr == null) {
+ return null;
+ }
+ return userAttr.get(entryKey);
+ }
+
+ /////////////////////// Transaction Helper Methods ////////////////////
+
+ public final TXStateProxy getTXState() {
+ if (this.supportsTX) {
+ return TXManagerImpl.getCurrentTXState();
+ }
+ else {
+ return null;
+ }
+ }
+
+ final TXId getTXId() {
+ final TXStateInterface tx = getTXState();
+ if (tx == null) {
+ return null;
+ }
+ return (TXId)tx.getTransactionId();
+ }
+
+ protected final TXRegionState txReadRegion()
+ {
+ final TXStateInterface tx = getTXState();
+ if (tx != null) {
+ return tx.txReadRegion(this);
+ }
+ else {
+ return null;
+ }
+ }
+
+ public TXEntryState createReadEntry(TXRegionState txr, KeyInfo keyInfo, boolean createIfAbsent) {
+ TXEntryState result = null;
+ final RegionEntry re = this.basicGetTXEntry(keyInfo);
+ if (re != null) {
+ boolean needsLRUCleanup = false;
+ try {
+ synchronized (re) {
+ if (!re.isRemoved()) {
+ if (re instanceof DiskEntry && re instanceof LRUEntry) {
+ LRUEntry le = (LRUEntry)re;
+ if (le.testEvicted()) {
+ // Handle the case where we fault in a disk entry
+ txLRUStart();
+ needsLRUCleanup = true;
+
+ // Fault in the value from disk
+ re.getValue(this);
+ }
+ }
+
+ Object value = re.getValueInVM(this);
+ /*
+ * The tx will need the raw value for identity comparison.
+ * Please see TXEntryState#checkForConflict(LocalRegion,Object)
+ */
+ Object id = re.getTransformedValue();
+
+ result = txr.createReadEntry(this, keyInfo.getKey(), re, id, value);
+ }
+ }
+ } catch (DiskAccessException dae) {
+ handleDiskAccessException(dae);
+ needsLRUCleanup = false;
+ throw dae;
+ } finally {
+ if (needsLRUCleanup) {
+ // do this after releasing sync
+ txLRUEnd();
+ }
+ }
+ }
+ if (result == null && createIfAbsent) {
+ result = txr.createReadEntry(this, keyInfo.getKey(), null, null, null);
+ }
+ return result;
+ }
+
+ protected static final TXEntryState NOOP_INVALIDATE = new TXEntryState();
+
+ protected TXStateInterface getJTAEnlistedTX()
+ {
+ if (ignoreJTA) {
+ // fixes bug 45541
+ return null;
+ }
+ TXStateInterface tx = getTXState();
+ if (tx != null) {
+ return tx;
+ }
+ else {
+ javax.transaction.Transaction jtaTx;
+ try {
+ if (!ignoreJTA && this.cache.getJTATransactionManager() != null) {
+
+ jtaTx = this.cache.getJTATransactionManager().getTransaction();
+ if (jtaTx == null
+ || jtaTx.getStatus() == javax.transaction.Status.STATUS_NO_TRANSACTION) {
+ return null;
+ }
+ tx = this.cache.getTXMgr().beginJTA();
+ jtaTx.registerSynchronization(tx);
+ return tx;
+ }
+ else {
+ return null;
+ }
+ }
+ catch (javax.transaction.SystemException se) {
+ // this can be thrown when the system is shutting down (see bug #39728)
+ stopper.checkCancelInProgress(se);
+ jtaEnlistmentFailureCleanup(tx, se);
+ return null;
+ }
+ catch (javax.transaction.RollbackException re) {
+ jtaEnlistmentFailureCleanup(tx, re);
+ return null;
+ }
+ catch (IllegalStateException ie) {
+ jtaEnlistmentFailureCleanup(tx, ie);
+ return null;
+ }
+ }
+ }
+
+ private final void jtaEnlistmentFailureCleanup(TXStateInterface tx, Exception reason) {
+ if (cache == null) {
+ return;
+ }
+ cache.getTXMgr().setTXState(null);
+ if (tx != null) {
+ tx.rollback();
+ }
+ String jtaTransName = null;
+ try {
+ jtaTransName = cache.getJTATransactionManager().getTransaction()
+ .toString();
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable ignore) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ }
+
+ throw new FailedSynchronizationException(LocalizedStrings.LocalRegion_FAILED_ENLISTEMENT_WITH_TRANSACTION_0.toLocalizedString(jtaTransName), reason);
+ }
+
+ final void txLRUStart()
+ {
+ this.entries.disableLruUpdateCallback();
+ }
+
+ final void txLRUEnd()
+ {
+ this.entries.enableLruUpdateCallback();
+ try {
+ this.entries.lruUpdateCallback();
+ }catch(DiskAccessException dae) {
+ handleDiskAccessException(dae);
+ throw dae;
+ }
+ }
+
+ final void txDecRefCount(RegionEntry re)
+ {
+ this.entries.decTxRefCount(re);
+ }
+
+ /** ******* DEBUG Methods */
+ /** Does not throw RegionDestroyedException even if destroyed */
+ List debugGetSubregionNames()
+ {
+ List names = new ArrayList();
+ for (Iterator itr = this.subregions.keySet().iterator(); itr.hasNext();)
+ names.add(itr.next());
+ return names;
+ }
+
+ /*****************************************************************************
+ * INNER CLASSES
+ ****************************************************************************/
+
+ protected final static void dispatchEvent(LocalRegion region,
+ InternalCacheEvent event, EnumListenerEvent op)
+ {
+
+ CacheListener[] listeners = region.fetchCacheListenersField();
+ if (event.getOperation().isCreate()) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("invoking listeners: " + Arrays.toString(listeners));
+ }
+ }
+ if (listeners == null || listeners.length == 0) {
+ return;
+ }
+ if (op != EnumListenerEvent.AFTER_REGION_CREATE) {
+ try {
+ region.waitForRegionCreateEvent();
+ }
+ catch (CancelException e) {
+ // ignore and keep going
+ if (logger.isTraceEnabled()) {
+ logger.trace("Dispatching events after cache closure for region {}", region.getFullPath());
+ }
+ }
+ }
+ //Assert.assertTrue(event.getRegion() == region);
+ if (!event.isGenerateCallbacks()) {
+ return;
+ }
+// this check moved earlier for bug 36983
+// CacheListener[] listeners = region.fetchCacheListenersField();
+// if (listeners == null || listeners.length == 0)
+// return;
+ for (int i = 0; i < listeners.length; i++) {
+ CacheListener listener = listeners[i];
+ if (listener != null) {
+ try {
+ op.dispatchEvent(event, listener);
+ }
+ catch (CancelException ignore) {
+ // ignore for bug 37105
+ }
+ catch (VirtualMachineError err) {
+ SystemFailure.initiateFailure(err);
+ // If this ever returns, rethrow the error. We're poisoned
+ // now, so don't let this thread continue.
+ throw err;
+ }
+ catch (Throwable t) {
+ // Whenever you catch Error or Throwable, you must also
+ // catch VirtualMachineError (see above). However, there is
+ // _still_ a possibility that you are dealing with a cascading
+ // error condition, so you also need to check to see if the JVM
+ // is still usable:
+ SystemFailure.checkFailure();
+ logger.error(LocalizedMessage.create(LocalizedStrings.LocalRegion_EXCEPTION_OCCURRED_IN_CACHELISTENER), t);
+ }
+ }
+ }
+ }
+
+ /** ********************* Class EventDispatcher ***************************** */
+
+ class EventDispatcher implements Runnable
+ {
+ InternalCacheEvent event;
+
+ EnumListenerEvent op;
+
+ EventDispatcher(InternalCacheEvent event, EnumListenerEvent op) {
+
+ if (LocalRegion.this.offHeap && event instanceof EntryEventImpl) {
+ // Make a copy that has its own off-heap refcount so fix bug 48837
+ event = new EntryEventImpl( (EntryEventImpl)event);
+ }
+ this.event = event;
+ this.op = op;
+ }
+
+ public void run()
+ {
+ try {
+ dispatchEvent(LocalRegion.this, this.event, this.op);
+ }finally {
+ this.release();
+ }
+ }
+
+ public void release() {
+ if (LocalRegion.this.offHeap && this.event instanceof EntryEventImpl) {
+ ((EntryEventImpl)this.event).release();
+ }
+ }
+ }
+
+
+
+ /** ******************* Class SubregionsSet ********************************* */
+
+ /** Set view of subregions */
+ private class SubregionsSet extends AbstractSet
+ {
+ final boolean recursive;
+
+ SubregionsSet(boolean recursive) {
+ this.recursive = recursive;
+ }
+
+ @Override
+ public Iterator iterator()
+ {
+
+ // iterates breadth-first (if recursive)
+ return new Iterator() {
+ Iterator currItr = LocalRegion.this.subregions.values().iterator();
+
+ List itrQ; // FIFO queue of iterators
+
+ Object nextElem = null;
+
+ public void remove()
+ {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_THIS_ITERATOR_DOES_NOT_SUPPORT_MODIFICATION.toLocalizedString());
+ }
+
+ public boolean hasNext()
+ {
+ if (nextElem != null) {
+ return true;
+ }
+ else {
+ Object el = next(true);
+ if (el != null) {
+ nextElem = el;
+ return true;
+ }
+ else {
+ return false;
+ }
+ }
+ }
+
+ private boolean _hasNext()
+ {
+ return this.currItr != null && this.currItr.hasNext();
+ }
+
+ public Object next() {
+ return next(false);
+ }
+
+ /**
+ * @param nullOK if true, return null instead of throwing NoSuchElementException
+ * @return the next element
+ */
+
+ private Object next(boolean nullOK)
+ {
+ if (nextElem != null) {
+ Object next = nextElem;
+ nextElem = null;
+ return next;
+ }
+
+ LocalRegion rgn;
+ do {
+ rgn = null;
+ if (!_hasNext()) {
+ if (itrQ == null || itrQ.isEmpty()) {
+ if (nullOK) {
+ return null;
+ }
+ else {
+ throw new NoSuchElementException();
+ }
+ }
+ else {
+ this.currItr = (Iterator)itrQ.remove(0);
+ continue;
+ }
+ }
+ rgn = (LocalRegion)currItr.next();
+ } while (rgn == null || !rgn.isInitialized() || rgn.isDestroyed());
+
+ if (recursive) {
+ Iterator nextIterator = rgn.subregions.values().iterator();
+ if (nextIterator.hasNext()) {
+ if (itrQ == null) {
+ itrQ = new ArrayList();
+ }
+ itrQ.add(nextIterator);
+ }
+ }
+ if (!_hasNext()) {
+ if (itrQ == null || itrQ.isEmpty()) {
+ this.currItr = null;
+ }
+ else {
+ this.currItr = (Iterator)itrQ.remove(0);
+ }
+ }
+ return rgn;
+ }
+ };
+ }
+
+ @Override
+ public int size()
+ {
+ if (this.recursive) {
+ return allSubregionsSize() - 1 /* don't count this region */;
+ }
+ else {
+ return LocalRegion.this.subregions.size();
+ }
+ }
+
+ @Override
+ public Object[] toArray()
+ {
+ List temp = new ArrayList(this.size());
+ for (Iterator iter = this.iterator(); iter.hasNext();) {
+ temp.add(iter.next());
+ }
+ return temp.toArray();
+ }
+
+ @Override
+ public Object[] toArray(Object[] array)
+ {
+ List temp = new ArrayList(this.size());
+ for (Iterator iter = this.iterator(); iter.hasNext();) {
+ temp.add(iter.next());
+ }
+ return temp.toArray(array);
+ }
+ }
+
+ /** ******************* Class EntriesSet ************************************ */
+
+ public class NonTXEntry implements Region.Entry
+ {
+
+ private final Object key;
+
+ private boolean entryIsDestroyed = false;
+
+ public boolean isLocal() {
+ return true;
+ }
+
+ /**
+ * Create an Entry given a key. The returned Entry may or may not be
+ * destroyed
+ */
+ public NonTXEntry(RegionEntry regionEntry) {
+ if (regionEntry == null) {
+ throw new IllegalArgumentException(LocalizedStrings.LocalRegion_REGIONENTRY_SHOULD_NOT_BE_NULL.toLocalizedString());
+ }
+ // for a soplog region, since the entry may not be in memory,
+ // we will have to fetch it from soplog, if the entry is in
+ // memory this is a quick lookup, so rather than RegionEntry
+ // we keep reference to key
+ this.key = regionEntry.getKey();
+ }
+
+ /** Internal method for getting the underlying RegionEntry */
+ public RegionEntry getRegionEntry() {
+ RegionEntry re = LocalRegion.this.getRegionMap().getEntry(key);
+ if (re == null) {
+ throw new EntryDestroyedException(this.key.toString());
+ }
+ return re;
+ }
+
+ private RegionEntry basicGetEntry() {
+ RegionEntry re = LocalRegion.this.basicGetEntry(key);
+ if (re == null) {
+ throw new EntryDestroyedException(this.key.toString());
+ }
+ return re;
+ }
+
+ public boolean isDestroyed()
+ {
+ if (this.entryIsDestroyed) {
+ return true;
+ }
+ if (LocalRegion.this.isDestroyed) {
+ this.entryIsDestroyed = true;
+ } else if (LocalRegion.this.basicGetEntry(key) == null) {
+ this.entryIsDestroyed = true;
+ }
+ return this.entryIsDestroyed;
+ }
+
+ public Object getKey()
+ {
+ return this.basicGetEntry().getKey();
+ }
+
+ public Object getValue()
+ {
+ return getValue(false);
+ }
+
+ public Object getValue(boolean ignoreCopyOnRead)
+ {
+ Object value = getDeserialized(this.basicGetEntry(), false, ignoreCopyOnRead, false, false);
+ if (value == null) {
+ throw new EntryDestroyedException(getKey().toString());
+ }
+ else if(Token.isInvalid(value)) {
+ return null;
+ }
+
+ return value;
+ }
+
+ /**
+ * To get the value from region in serialized form
+ * @return {@link VMCachedDeserializable}
+ */
+ public Object getRawValue()
+ {
+ Object value = this.basicGetEntry().getValue((LocalRegion) getRegion());
+ if (value == null) {
+ throw new EntryDestroyedException(this.getRegionEntry().getKey().toString());
+ }
+ else if(Token.isInvalid(value)) {
+ return null;
+ }
+
+ return value;
+ }
+
+ public Region getRegion()
+ {
+ this.basicGetEntry();
+ return LocalRegion.this;
+ }
+
+ public CacheStatistics getStatistics()
+ {
+ // prefer entry destroyed exception over statistics disabled exception
+ this.basicGetEntry();
+ if (!LocalRegion.this.statisticsEnabled) {
+ throw new StatisticsDisabledException(LocalizedStrings.LocalRegion_STATISTICS_DISABLED_FOR_REGION_0.toLocalizedString(getFullPath()));
+ }
+ return new CacheStatisticsImpl(this.basicGetEntry(), LocalRegion.this);
+ }
+
+ public Object getUserAttribute()
+ {
+ this.basicGetEntry();
+ Map userAttr = LocalRegion.this.entryUserAttributes;
+ if (userAttr == null) {
+ return null;
+ }
+ return userAttr.get(this.basicGetEntry().getKey());
+ }
+
+ public Object setUserAttribute(Object value)
+ {
+ if (LocalRegion.this.entryUserAttributes == null) {
+ LocalRegion.this.entryUserAttributes = new Hashtable();
+ }
+ return LocalRegion.this.entryUserAttributes.put(
+ this.basicGetEntry().getKey(), value);
+ }
+
+ @Override
+ public boolean equals(Object obj)
+ {
+ if (!(obj instanceof LocalRegion.NonTXEntry)) {
+ return false;
+ }
+ LocalRegion.NonTXEntry lre = (LocalRegion.NonTXEntry)obj;
+ return this.basicGetEntry().equals(lre.getRegionEntry())
+ && this.getRegion() == lre.getRegion();
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return this.basicGetEntry().hashCode() ^ this.getRegion().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return new StringBuilder("NonTXEntry@").append(
+ Integer.toHexString(System.identityHashCode(this))).append(' ')
+ .append(this.getRegionEntry()).toString();
+ }
+
+ ////////////////// Private Methods
+ // /////////////////////////////////////////
+
+ /**
+ * @since 5.0
+ */
+ public Object setValue(Object arg0)
+ {
+ return put(this.getKey(), arg0);
+ }
+ }
+
+ /**
+ * Checks to make sure current thread does not have a transaction. If it does
+ * then throws UnsupportedOperationException
+ */
+ public void checkForTransaction(String opName)
+ {
+ if (isTX()) {
+ throw new UnsupportedOperationException(LocalizedStrings.LocalRegion_0_OPERATIONS_ARE_NOT_ALLOWED_BECAUSE_THIS_THREAD_HAS_AN_ACTIVE_TRANSACTION
+ .toLocalizedString(opName));
+ }
+ }
+
+ /**
+ * For internal use only.
+ */
+ public RegionMap getRegionMap()
+ {
+ // OK to ignore tx state
+ return this.entries;
+ }
+
+ /**
+ * Methods for java.util.Map compliance
+ *
+ * @since 5.0
+ */
+
+ /**
+ * (description copied from entryCount() Returns the number of entries in this
+ * region. Note that because of the concurrency properties of the
+ * {@link RegionMap}, the number of entries is only an approximate. That is,
+ * other threads may change the number of entries in this region while this
+ * method is being invoked.
+ *
+ * @see LocalRegion#entryCount()
+ */
+ public int size()
+ {
+ checkReadiness();
+ checkForNoAccess();
+ discoverJTA();
+ boolean isClient = this.imageState.isClient();
+ if (isClient) {
+ lockRIReadLock(); // bug #40871 - test sees wrong size for region during RI
+ }
+ try {
+ return entryCount();
+ } finally {
+ if (isClient) {
+ unlockRIReadLock();
+ }
+ }
+ }
+
+ /**
+ * returns an estimate of the number of entries in this region. This method
+ * should be prefered over size() for hdfs regions where an accurate size is
+ * not needed. This method is not supported on a client
+ *
+ * @return the estimated size of this region
+ */
+ public int sizeEstimate() {
+ boolean isClient = this.imageState.isClient();
+ if (isClient) {
+ throw new UnsupportedOperationException(
+ "Method not supported on a client");
+ }
+ return entryCount(null, true);
+ }
+
+ /**
+ * This method returns true if Region is Empty.
+ */
+ public boolean isEmpty()
+ {
+ //checkForNoAccess(); // size does this check
+ return this.size() > 0 ? false : true;
+ }
+
+ /**
+ * Returns true if the value is present in the Map
+ */
+ public boolean containsValue(final Object value)
+ {
+ if (value == null) {
+ throw new NullPointerException(LocalizedStrings.LocalRegion_VALUE_FOR_CONTAINSVALUEVALUE_CANNOT_BE_NULL.toLocalizedString());
+ }
+ checkReadiness();
+ checkForNoAccess();
+ boolean result = false;
+ Iterator iterator = new EntriesSet(this, false, IteratorType.VALUES, false).iterator();
+ Object val = null;
+ while (iterator.hasNext()) {
+ val = iterator.next();
+ if (val != null) {
+ if (value.equals(val)) {
+ result = true;
+ break;
+ }
+ }
+ }
+ return result;
+ }
+
+ /**
+ * Returns a set of the entries present in the Map. This set is Not
+ * Modifiable. If changes are made to this set, they will be not reflected in
+ * the map
+ */
+ public Set entrySet()
+ {
+ //entries(false) takes care of open transactions
+ return entries(false);
+ }
+
+
+
+ /**
+ * Returns a set of the keys present in the Map. This set is Not Modifiable.
+ * If changes are made to this set, they will be not reflected in the map
+ */
+
+ public Set keySet()
+ {
+ //keys() takes care of open transactions
+ return keys();
+ }
+
+ /**
+ * removes the object from the Map and returns the object removed. The object
+ * is returned only if present in the localMap. If the value is present in
+ * another Node, null is returned
+ */
+ public Object remove(Object obj)
+ {
+ // no validations needed here since destroy does it for us
+ Object returnObject = null;
+ try {
+ returnObject = destroy(obj);
+ }
+ catch (EntryNotFoundException e) {
+ // No need to log this exception; caller can test for null;
+ }
+ return returnObject;
+ }
+
+ public void basicBridgeDestroyRegion( Object p_callbackArg, final ClientProxyMembershipID client,
+ boolean fromClient, EventID eventId) throws TimeoutException,
+ EntryExistsException, CacheWriterException
+ {
+ Object callbackArg = p_callbackArg;
+ //long startPut = CachePerfStats.getStatTime();
+ if (fromClient) {
+ // If this region is also wan-enabled, then wrap that callback arg in a
+ // GatewayEventCallbackArgument to store the event id.
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+ }
+
+ RegionEventImpl event = new ClientRegionEventImpl(this, Operation.REGION_DESTROY,
+ callbackArg,false, client.getDistributedMember(), client/* context */, eventId);
+
+ basicDestroyRegion(event, true);
+
+ }
+
+
+
+
+ public void basicBridgeClear( Object p_callbackArg,
+ final ClientProxyMembershipID client,
+ boolean fromClient, EventID eventId) throws TimeoutException,
+ EntryExistsException, CacheWriterException
+ {
+ Object callbackArg = p_callbackArg;
+ //long startPut = CachePerfStats.getStatTime();
+ if (fromClient) {
+ // If this region is also wan-enabled, then wrap that callback arg in a
+ // GatewayEventCallbackArgument to store the event id.
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+ }
+
+ RegionEventImpl event = new ClientRegionEventImpl(this, Operation.REGION_CLEAR,
+ callbackArg,false, client.getDistributedMember(), client/* context */, eventId);
+
+ basicClear(event, true);
+ }
+
+
+
+ @Override
+ void basicClear(RegionEventImpl regionEvent)
+ {
+ getDataView().checkSupportsRegionClear();
+ basicClear(regionEvent, true/* cacheWrite */);
+ }
+
+ void basicClear(RegionEventImpl regionEvent, boolean cacheWrite) {
+ cmnClearRegion(regionEvent, cacheWrite, true);
+ }
+
+
+ void cmnClearRegion(RegionEventImpl regionEvent, boolean cacheWrite, boolean useRVV) {
+ RegionVersionVector rvv = null;
+ if (useRVV && this.dataPolicy.withReplication() && this.concurrencyChecksEnabled) {
+ rvv = this.versionVector.getCloneForTransmission();
+ }
+ clearRegionLocally(regionEvent, cacheWrite, rvv);
+ }
+
+ /**
+ * Common code used by both clear and localClear. Asif : On the lines of
+ * destroyRegion, this method will be invoked for clearing the local cache.The
+ * cmnClearRegion will be overridden in the derived class DistributedRegion
+ * too. For clear operation , no CacheWriter will be invoked . It will only
+ * have afterClear callback. Also like destroyRegion & invalidateRegion , the
+ * clear operation will not take distributedLock. The clear operation will
+ * also clear the local tranxnl entries . The clear operation will have
+ * immediate committed state.
+ */
+ void clearRegionLocally(RegionEventImpl regionEvent, boolean cacheWrite, RegionVersionVector vector)
+ {
+ final boolean isRvvDebugEnabled = logger.isTraceEnabled(LogMarker.RVV);
+
+ RegionVersionVector rvv = vector;
+ if (this.srp != null) {
+ // clients and local regions do not maintain a full RVV. can't use it with clear()
+ rvv = null;
+ }
+ if (rvv != null && this.dataPolicy.withStorage()) {
+ if (isRvvDebugEnabled) {
+ logger.trace(LogMarker.RVV, "waiting for my version vector to dominate\nmine={}\nother=", this.versionVector.fullToString(), rvv);
+ }
+ boolean result = this.versionVector.waitToDominate(rvv, this);
+ if (!result) {
+ if (isRvvDebugEnabled) {
+ logger.trace(LogMarker.RVV, "incrementing clearTimeouts for {} rvv={}", this.getName(), this.versionVector.fullToString());
+ }
+ getCachePerfStats().incClearTimeouts();
+ }
+ }
+
+ //If the initial image operation is still in progress
+ // then we need will have to do the clear operation at the
+ // end of the GII.For this we try to acquire the lock of GII
+ // the boolean returned is true that means lock was obtained which
+ // also means that GII is still in progress.
+ boolean isGIIInProg = lockGII();
+ if (isGIIInProg) {
+ //Set a flag which will indicate that the Clear was invoked.
+ // Also we should try & abort the GII
+ try {
+ getImageState().setClearRegionFlag(true /* Clear region */, rvv);
+ }
+ finally {
+ unlockGII();
+ }
+ }
+
+ if (cacheWrite && !isGIIInProg) {
+ this.cacheWriteBeforeRegionClear(regionEvent);
+ }
+
+ RegionVersionVector myVector = getVersionVector();
+ if (myVector != null) {
+ if (isRvvDebugEnabled) {
+ logger.trace(LogMarker.RVV, "processing version information for {}", regionEvent);
+ }
+ if (!regionEvent.isOriginRemote() && !regionEvent.getOperation().isLocal()) {
+ // generate a new version for the operation
+ VersionTag tag = VersionTag.create(getVersionMember());
+ tag.setVersionTimeStamp(cacheTimeMillis());
+ tag.setRegionVersion(myVector.getNextVersionWhileLocked());
+ if (isRvvDebugEnabled) {
+ logger.trace(LogMarker.RVV, "generated version tag for clear: {}", tag);
+ }
+ regionEvent.setVersionTag(tag);
+ } else {
+ VersionTag tag = regionEvent.getVersionTag();
+ if (tag != null) {
+ if (isRvvDebugEnabled) {
+ logger.trace(LogMarker.RVV, "recording version tag for clear: {}", tag);
+ }
+ myVector.recordVersion(tag.getMemberID(), tag); // clear() events always have the ID in the tag
+ }
+ }
+ }
+
+ // Asif:Clear the expirational task for all the entries. It is possible that
+ //after clearing it some new entries may get added befoe issuing clear
+ //on the map , but that should be OK, as the expirational thread will
+ //silently move ahead if the entry to be expired no longer existed
+ this.cancelAllEntryExpiryTasks();
+ if (this.entryUserAttributes != null) {
+ this.entryUserAttributes.clear();
+ }
+
+ // if all current content has been removed then the version vector
+ // does not need to retain any exceptions and the GC versions can
+ // be set to the current vector versions
+ if (rvv == null && myVector != null) {
+ myVector.removeOldVersions();
+ }
+
+ /*
+ * Asif : First we need to clear the Tranxl state for the current region for
+ * the thread. The operation will not take global lock similar to
+ * regionInvalidateor regionDestroy behaviour.
+ */
+
+ // mbid : clear the disk region if present
+ if (diskRegion != null) {
+ // persist current rvv and rvvgc which contained version for clear() itself
+ if (this.getDataPolicy().withPersistence()) {
+ // null means not to change dr.rvvTrust
+ if (isRvvDebugEnabled) {
+ logger.trace(LogMarker.RVV, "Clear: Saved current rvv: {}", diskRegion.getRegionVersionVector());
+ }
+ diskRegion.writeRVV(this, null);
+ diskRegion.writeRVVGC(this);
+ }
+
+ // clear the entries in disk
+ diskRegion.clear(this, rvv);
+ }
+ // this will be done in diskRegion.clear if it is not null else it has to be
+ // done here
+ else {
+ // Now remove the tranxnl entries for this region
+ this.txClearRegion();
+ // Now clear the map of committed entries
+ Set remainingIDs = clearEntries(rvv);
+ if (!this.dataPolicy.withPersistence()) { // persistent regions do not reap IDs
+ if (myVector != null) {
+ myVector.removeOldMembers(remainingIDs);
+ }
+ }
+ }
+
+ clearHDFSData();
+
+ if (!isProxy()) {
+ // Now we need to recreate all the indexes.
+ //If the indexManager is null we don't have to worry
+ //for any other thread creating index at that instant
+ // because the region has already been cleared
+ //of entries.
+ //TODO Asif:Have made indexManager variable is made volatile. Is it
+ // necessary?
+ if (this.indexManager != null) {
+ try {
+ this.indexManager.rerunIndexCreationQuery();
+ }
+ catch (QueryException qe) {
+ // Asif : Create an anonymous inner class of CacheRuntimeException so
+ // that a RuntimeException is thrown
+ throw new CacheRuntimeException(LocalizedStrings.LocalRegion_EXCEPTION_OCCURED_WHILE_RE_CREATING_INDEX_DATA_ON_CLEARED_REGION.toLocalizedString(), qe) {
+ private static final long serialVersionUID = 0L;
+ };
+ }
+ }
+ }
+
+ if (ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
+ CacheObserverHolder.getInstance().afterRegionClear(regionEvent);
+ }
+
+ if (isGIIInProg) {
+ return;
+ }
+ regionEvent.setEventType(EnumListenerEvent.AFTER_REGION_CLEAR);
+ // notifyBridgeClients(EnumListenerEvent.AFTER_REGION_CLEAR, regionEvent);
+ //Issue a callback to afterClear if the region is initialized
+ boolean hasListener = hasListener();
+ if (hasListener) {
+ dispatchListenerEvent(EnumListenerEvent.AFTER_REGION_CLEAR, regionEvent);
+ }
+ }
+
+ /**Clear HDFS data, if present */
+ protected void clearHDFSData() {
+ //do nothing, clear is implemented for subclasses like BucketRegion.
+ }
+
+ @Override
+ void basicLocalClear(RegionEventImpl rEvent)
+ {
+ getDataView().checkSupportsRegionClear();
+ cmnClearRegion(rEvent, false/* cacheWrite */, false/*useRVV*/);
+ }
+
+ public void handleInterestEvent(InterestRegistrationEvent event) {
+ throw new UnsupportedOperationException(LocalizedStrings.
+ LocalRegion_REGION_INTEREST_REGISTRATION_IS_ONLY_SUPPORTED_FOR_PARTITIONEDREGIONS
+ .toLocalizedString());
+ }
+
+ @Override
+ Map basicGetAll(Collection keys, Object callback) {
+ final boolean isDebugEnabled = logger.isDebugEnabled();
+ final boolean isTraceEnabled = logger.isTraceEnabled();
+
+ if (isDebugEnabled) {
+ logger.debug("Processing getAll request for: {}", keys);
+ }
+ discoverJTA();
+ Map allResults = new HashMap();
+ if (hasServerProxy()) {
+ // Some of our implementation expects a list of keys so make sure it is a list
+ List keysList;
+ if (keys instanceof List) {
+ keysList = (List) keys;
+ } else {
+ keysList = new ArrayList(keys);
+ }
+
+ // Gather any local values
+ // We only need to do this if this region has local storage
+ if(getTXState() == null && hasStorage()) {
+ if (keysList == keys) {
+ // Create a copy of the collection of keys
+ // to keep the original collection intact
+ keysList = new ArrayList(keys);
+ }
+ for (Iterator i = keysList.iterator(); i.hasNext();) {
+ Object key = i.next();
+ Object value;
+ Region.Entry entry = accessEntry(key, true);
+ if (entry != null && (value = entry.getValue()) != null) {
+ allResults.put(key, value);
+ i.remove();
+ }
+ }
+ if (isDebugEnabled) {
+ logger.debug("Added local results for getAll request: {}", allResults);
+ }
+ }
+
+ // Send the rest of the keys to the server (if necessary)
+ if (!keysList.isEmpty()) {
+ VersionedObjectList remoteResults = getServerProxy().getAll(keysList, callback);
+ if (isDebugEnabled) {
+ logger.debug("remote getAll results are {}", remoteResults);
+ }
+
+ // Add remote results to local cache and all results if successful
+ for (VersionedObjectList.Iterator it = remoteResults.iterator(); it.hasNext();) {
+ VersionedObjectList.Entry entry = it.next();
+ Object key = entry.getKey();
+ boolean notOnServer = entry.isKeyNotOnServer();
+ // in 8.0 we added transfer of tombstones with RI/getAll results for bug #40791
+ boolean createTombstone = false;
+ if (notOnServer) {
+ createTombstone = (entry.getVersionTag() != null && this.concurrencyChecksEnabled);
+ allResults.put(key, null);
+ if (isDebugEnabled) {
+ logger.debug("Added remote result for missing key: {}", key);
+ }
+ if (!createTombstone) {
+ continue;
+ }
+ }
+
+ Object value;
+ if (createTombstone) {
+ // the value is null in this case, so use TOKEN_TOMBSTONE
+ value = Token.TOMBSTONE;
+ } else {
+ value = entry.getObject();
+ }
+
+ if (value instanceof Throwable){
+ continue;
+ }
+ // The following basicPutEntry needs to be done
+ // even if we do not have storage so that the
+ // correct events will be delivered to any callbacks we have.
+ long startPut = CachePerfStats.getStatTime();
+ validateKey(key);
+ EntryEventImpl event = EntryEventImpl.create(
+ this, Operation.LOCAL_LOAD_CREATE, key, value,
+ callback, false, getMyId(), true);
+ try {
+ event.setFromServer(true);
+ event.setVersionTag(entry.getVersionTag());
+
+ if (!alreadyInvalid(key, event)) { // bug #47716 - don't update if it's already here & invalid
+ TXStateProxy tx = this.cache.getTXMgr().internalSuspend();
+ try {
+ basicPutEntry(event, 0L);
+ } catch (ConcurrentCacheModificationException e) {
+ if (isDebugEnabled) {
+ logger.debug("getAll result for {} not stored in cache due to concurrent modification", key);
+ }
+ } finally {
+ this.cache.getTXMgr().resume(tx);
+ }
+ getCachePerfStats().endPut(startPut, event.isOriginRemote());
+ }
+
+ if (!createTombstone) {
+ allResults.put(key, value);
+ if (isTraceEnabled) {
+ logger.trace("Added remote result for getAll request: {}, {}", key, value);
+ }
+ }
+ } finally {
+ event.release();
+ }
+ }
+ }
+ } else {
+ // This implementation for a P2P VM is a stop-gap to provide the
+ // functionality. It needs to be rewritten more efficiently.
+ for (Iterator i = keys.iterator(); i.hasNext();) {
+ Object key = i.next();
+ try {
+ allResults.put(key, get(key, callback));
+ } catch (Exception e) {
+ logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_THE_FOLLOWING_EXCEPTION_OCCURRED_ATTEMPTING_TO_GET_KEY_0, key), e);
+ }
+ }
+ }
+ return allResults;
+ }
+
+ /**
+ * Return false if it will never store entry ekys and values locally; otherwise return true.
+ */
+ protected boolean hasStorage() {
+ return this.getDataPolicy().withStorage();
+ }
+
+ private void verifyPutAllMap(Map map) {
+ Map.Entry mapEntry = null;
+ Collection theEntries = map.entrySet();
+ Iterator iterator = theEntries.iterator();
+ while (iterator.hasNext()) {
+ mapEntry = (Map.Entry)iterator.next();
+ Object key = mapEntry.getKey();
+ if (mapEntry.getValue() == null || key == null) {
+ throw new NullPointerException("Any key or value in putAll should not be null");
+ }
+ if (!MemoryThresholds.isLowMemoryExceptionDisabled()) {
+ checkIfAboveThreshold(key);
+ }
+ // Threshold check should not be performed again
+ }
+ }
+ private void verifyRemoveAllKeys(Collection keys) {
+ for (Object key: keys) {
+ if (key == null) {
+ throw new NullPointerException("Any key in removeAll must not be null");
+ }
+ }
+ }
+
+
+ /**
+ * Called on a bridge server when it has a received a putAll command from a client.
+ * @param map a map of key->value for the entries we are putting
+ * @param retryVersions a map of key->version tag. If any of the entries
+ * are the result of a retried client event, we need to make sure we send
+ * the original version tag along with the event.
+ * @param callbackArg callback argument from client
+ */
+ public VersionedObjectList basicBridgePutAll(Map map, Map retryVersions, ClientProxyMembershipID memberId,
+ EventID eventId, boolean skipCallbacks, Object callbackArg) throws TimeoutException, CacheWriterException
+ {
+ long startPut = CachePerfStats.getStatTime();
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.PUTALL_CREATE, null,
+ null /* new value */, callbackArg,
+ false /* origin remote */, memberId.getDistributedMember(),
+ !skipCallbacks /* generateCallbacks */,
+ eventId);
+ try {
+ event.setContext(memberId);
+ DistributedPutAllOperation putAllOp = new DistributedPutAllOperation(event,
+ map.size(), true);
+ try {
+ VersionedObjectList result = basicPutAll(map, putAllOp, retryVersions);
+ getCachePerfStats().endPutAll(startPut);
+ return result;
+ } finally {
+ putAllOp.freeOffHeapResources();
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+ /**
+ * Called on a bridge server when it has a received a removeAll command from a client.
+ * @param keys a collection of the keys we are putting
+ * @param retryVersions a collection of version tags. If the client is retrying a key
+ * then that keys slot will be non-null in this collection. Note that keys and retryVersions are parallel lists.
+ * @param callbackArg callback argument from client
+ */
+ public VersionedObjectList basicBridgeRemoveAll(ArrayList keys, ArrayList retryVersions, ClientProxyMembershipID memberId,
+ EventID eventId, Object callbackArg) throws TimeoutException, CacheWriterException
+ {
+ long startOp = CachePerfStats.getStatTime();
+ if(isGatewaySenderEnabled()) {
+ callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
+ }
+
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.REMOVEALL_DESTROY, null,
+ null /* new value */, callbackArg,
+ false /* origin remote */, memberId.getDistributedMember(),
+ true /* generateCallbacks */,
+ eventId);
+ try {
+ event.setContext(memberId);
+ DistributedRemoveAllOperation removeAllOp = new DistributedRemoveAllOperation(event, keys.size(), true);
+ try {
+ VersionedObjectList result = basicRemoveAll(keys, removeAllOp, retryVersions);
+ getCachePerfStats().endRemoveAll(startOp);
+ return result;
+ } finally {
+ removeAllOp.freeOffHeapResources();
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+ public VersionedObjectList basicImportPutAll(Map map, boolean skipCallbacks) {
+ long startPut = CachePerfStats.getStatTime();
+
+ // generateCallbacks == false
+ EntryEventImpl event = EntryEventImpl.create(this, Operation.PUTALL_CREATE,
+ null, null, null, true, getMyId(), !skipCallbacks);
+ try {
+ DistributedPutAllOperation putAllOp = new DistributedPutAllOperation(event, map.size(), false);
+ try {
+ VersionedObjectList result = basicPutAll(map, putAllOp, null);
+ getCachePerfStats().endPutAll(startPut);
+ return result;
+ } finally {
+ putAllOp.freeOffHeapResources();
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+ @Override
+ public final void putAll(Map map, Object callbackArg) {
+ long startPut = CachePerfStats.getStatTime();
+ final DistributedPutAllOperation putAllOp = newPutAllOperation(map, callbackArg);
+ if (putAllOp != null) {
+ try {
+ basicPutAll(map, putAllOp, null);
+ } finally {
+ putAllOp.getBaseEvent().release();
+ putAllOp.freeOffHeapResources();
+ }
+ }
+
+ getCachePerfStats().endPutAll(startPut);
+ }
+
+ @Override
+ public final void putAll(Map map) {
+ putAll(map, null);
+ }
+
+ @Override
+ public final void removeAll(Collection keys) {
+ removeAll(keys, null);
+ }
+
+ @Override
+ public final void removeAll(Collection keys, Object callbackArg) {
+ final long startOp = CachePerfStats.getStatTime();
+ final DistributedRemoveAllOperation op = newRemoveAllOperation(keys, callbackArg);
+ if (op != null) {
+ try {
+ basicRemoveAll(keys, op, null);
+ } finally {
+ op.getBaseEvent().release();
+ op.freeOffHeapResources();
+ }
+ }
+ getCachePerfStats().endRemoveAll(startOp);
+ }
+
+ /**
+ * Returns true if a one-hop (RemoteOperationMessage) should be used when applying the change
+ * to the system.
+ */
+ public boolean requiresOneHopForMissingEntry(EntryEventImpl event) {
+ return false;
+ }
+
+ public VersionedObjectList basicPutAll(final Map, ?> map,
+ final DistributedPutAllOperation putAllOp, final Map retryVersions) {
+ final boolean isDebugEnabled = logger.isDebugEnabled();
+ final EntryEventImpl event = putAllOp.getBaseEvent();
+ EventID eventId = event.getEventId();
+ if (eventId == null && generateEventID()) {
+ // Gester: We need to "reserve" the eventIds for the entries in map here
+ event.reserveNewEventId(cache.getDistributedSystem(), map.size());
+ eventId = event.getEventId();
+ }
+
+ RuntimeException e = null;
+ verifyPutAllMap(map);
+ VersionedObjectList proxyResult = null;
+ boolean partialResult = false;
+ if (hasServerProxy()) {
+ // send message to bridge server
+ if (isTX()) {
+ TXStateProxyImpl tx = (TXStateProxyImpl) cache.getTxManager().getTXState();
+ tx.getRealDeal(null, this);
+ }
+ try {
+ proxyResult = getServerProxy().putAll(map, eventId, !event.isGenerateCallbacks(), event.getCallbackArgument());
+ if (isDebugEnabled) {
+ logger.debug("PutAll received response from server: {}", proxyResult);
+ }
+ } catch (PutAllPartialResultException e1) {
+ // adjust the map to only add succeeded entries, then apply the adjustedMap
+ proxyResult = e1.getSucceededKeysAndVersions();
+ partialResult = true;
+ if (isDebugEnabled) {
+ logger.debug("putAll in client encountered a PutAllPartialResultException:{}\n. Adjusted keys are: ",
+ e1.getMessage(), proxyResult.getKeys());
+ }
+ Throwable txException = e1.getFailure();
+ while (txException != null) {
+ if (txException instanceof TransactionException) {
+ e = (TransactionException) txException;
+ break;
+ }
+ txException = txException.getCause();
+ }
+ if (e == null) {
+ e = new ServerOperationException(LocalizedStrings.Region_PutAll_Applied_PartialKeys_At_Server_0.toLocalizedString(getFullPath()), e1.getFailure());
+ }
+ }
+ }
+
+ final VersionedObjectList succeeded = new VersionedObjectList(map.size(), true, this.concurrencyChecksEnabled);
+ // if this is a transactional putAll, we will not have version information as it is only generated at commit
+ // so treat transactional putAll as if the server is not versioned
+ final boolean serverIsVersioned = proxyResult != null && proxyResult.regionIsVersioned()
+ && !isTX() && this.dataPolicy != DataPolicy.EMPTY;
+ if (!serverIsVersioned && !partialResult) {
+ // we don't need server information if it isn't versioned or if the region is empty
+ proxyResult = null;
+ }
+ lockRVVForBulkOp();
+ try {
+
+ final DistributedPutAllOperation dpao = putAllOp;
+ int size = (proxyResult == null)? map.size() : proxyResult.size();
+
+ if (isDebugEnabled) {
+ logger.debug( "size of put result is {} maps is {} proxyResult is {}", size, map, proxyResult);
+ }
+
+ final PutAllPartialResult partialKeys = new PutAllPartialResult(size);
+ final Iterator iterator;
+ final boolean isVersionedResults;
+ if (proxyResult != null) {
+ iterator = proxyResult.iterator();
+ isVersionedResults = true;
+ } else {
+ iterator = map.entrySet().iterator();
+ isVersionedResults = false;
+ }
+ Runnable r = new Runnable() {
+ public void run() {
+ int offset = 0;
+ EntryEventImpl tagHolder = EntryEventImpl.createVersionTagHolder();
+ while (iterator.hasNext()) {
+ stopper.checkCancelInProgress(null);
+ Map.Entry mapEntry = (Map.Entry)iterator.next();
+ Object key = mapEntry.getKey();
+ VersionTag versionTag = null;
+ tagHolder.setVersionTag(null);
+ final Object value;
+ boolean overwritten = false;
+ if (isVersionedResults) {
+ versionTag = ((VersionedObjectList.Entry)mapEntry).getVersionTag();
+ value = map.get(key);
+ if (isDebugEnabled) {
+ logger.debug("putAll key {} -> {} version={}", key, value, versionTag);
+ }
+ if (versionTag == null && serverIsVersioned && concurrencyChecksEnabled && dataPolicy.withStorage()) {
+ // server was unable to determine the version for this operation.
+ // I'm not sure this can still happen as described below on a pr.
+ // But it can happen on the server if NORMAL or PRELOADED. See bug 51644.
+ // This can happen in a PR with redundancy if there is a bucket
+ // failure or migration during the operation. We destroy the
+ // entry since we don't know what its state should be (but the server should)
+ if (isDebugEnabled) {
+ logger.debug("server returned no version information for {}", key);
+ }
+ localDestroyNoCallbacks(key);
+ // to be consistent we need to fetch the current entry
+ get(key, event.getCallbackArgument(), false, null);
+ overwritten = true;
+ }
+ } else {
+ value = mapEntry.getValue();
+ if (isDebugEnabled) {
+ logger.debug("putAll {} -> {}", key, value);
+ }
+ }
+ try {
+ if (serverIsVersioned) {
+ if (isDebugEnabled) {
+ logger.debug("associating version tag with {} version={}", key, versionTag);
+ }
+ //If we have received a version tag from a server, add it to the event
+ tagHolder.setVersionTag(versionTag);
+ tagHolder.setFromServer(true);
+ } else if (retryVersions != null && retryVersions.containsKey(key)) {
+ //If this is a retried event, and we have a version tag for the retry,
+ //add it to the event.
+ tagHolder.setVersionTag(retryVersions.get(key));
+ }
+
+ if (!overwritten) {
+ try {
+ basicEntryPutAll(key, value, dpao, offset, tagHolder);
+ } finally {
+ tagHolder.release();
+ }
+ }
+ // now we must check again since the cache may have closed during
+ // distribution (causing this process to not receive and queue the
+ // event for clients
+ stopper.checkCancelInProgress(null);
+ succeeded.addKeyAndVersion(key, tagHolder.getVersionTag());
+ }
+ catch (Exception ex) {
+ // TODO ask Gester if this debug logging can be removed
+ if (isDebugEnabled) {
+ logger.debug("PutAll operation encountered exception for key {}", key, ex);
+ }
+ partialKeys.saveFailedKey(key, ex);
+ }
+ offset++;
+ }
+ }
+ };
+ this.syncBulkOp(r, eventId);
+ if (partialKeys.hasFailure()) {
+ // Bug 51725: Now succeeded contains an order key list, may be missing the version tags.
+ // Save reference of succeeded into partialKeys. The succeeded may be modified by
+ // postPutAll() to fill in the version tags.
+ partialKeys.setSucceededKeysAndVersions(succeeded);
+ logger.info(LocalizedMessage.create(LocalizedStrings.Region_PutAll_Applied_PartialKeys_0_1,
+ new Object[] {getFullPath(), partialKeys}));
+ if (isDebugEnabled) {
+ logger.debug(partialKeys.detailString());
+ }
+ if (e == null) {
+ // if received exception from server first, ignore local exception
+ if (dpao.isBridgeOperation()) {
+ if (partialKeys.getFailure() instanceof CancelException) {
+ e = (CancelException)partialKeys.getFailure();
+ } else if (partialKeys.getFailure() instanceof LowMemoryException) {
+ throw partialKeys.getFailure(); // fix for #43589
+ } else {
+ e = new PutAllPartialResultException(partialKeys);
+ if (isDebugEnabled) {
+ logger.debug("basicPutAll:"+partialKeys.detailString());
+ }
+ }
+ } else {
+ throw partialKeys.getFailure();
+ }
+ }
+ }
+ }
+ catch (LowMemoryException lme) {
+ throw lme;
+ }
+ catch (RuntimeException ex) {
+ e = ex;
+ }
+ catch (Exception ex) {
+ e = new RuntimeException(ex);
+ } finally {
+ unlockRVVForBulkOp();
+ putAllOp.getBaseEvent().release();
+ putAllOp.freeOffHeapResources();
+ }
+ getDataView().postPutAll(putAllOp, succeeded, this);
+ if (e != null) {
+ throw e;
+ }
+ return succeeded;
+ }
+
+ public VersionedObjectList basicRemoveAll(final Collection keys,
+ final DistributedRemoveAllOperation removeAllOp, final ArrayList retryVersions) {
+ final boolean isDebugEnabled = logger.isDebugEnabled();
+ final boolean isTraceEnabled = logger.isTraceEnabled();
+ final EntryEventImpl event = removeAllOp.getBaseEvent();
+ EventID eventId = event.getEventId();
+ if (eventId == null && generateEventID()) {
+ // Gester: We need to "reserve" the eventIds for the entries in map here
+ event.reserveNewEventId(cache.getDistributedSystem(), keys.size());
+ eventId = event.getEventId();
+ }
+
+ RuntimeException e = null;
+ verifyRemoveAllKeys(keys);
+ VersionedObjectList proxyResult = null;
+ boolean partialResult = false;
+ if (hasServerProxy()) {
+ // send message to bridge server
+ if (isTX()) {
+ TXStateProxyImpl tx = (TXStateProxyImpl) cache.getTxManager().getTXState();
+ tx.getRealDeal(null, this);
+ }
+ try {
+ proxyResult = getServerProxy().removeAll(keys, eventId, event.getCallbackArgument());
+ if (isDebugEnabled) {
+ logger.debug("removeAll received response from server: {}", proxyResult);
+ }
+ } catch (PutAllPartialResultException e1) {
+ // adjust the map to only add succeeded entries, then apply the adjustedMap
+ proxyResult = e1.getSucceededKeysAndVersions();
+ partialResult = true;
+ if (isDebugEnabled) {
+ logger.debug("removeAll in client encountered a BulkOpPartialResultException: {}\n. Adjusted keys are: {}", e1.getMessage(), proxyResult.getKeys());
+ }
+ Throwable txException = e1.getFailure();
+ while (txException != null) {
+ if (txException instanceof TransactionException) {
+ e = (TransactionException) txException;
+ break;
+ }
+ txException = txException.getCause();
+ }
+ if (e == null) {
+ e = new ServerOperationException(LocalizedStrings.Region_RemoveAll_Applied_PartialKeys_At_Server_0.toLocalizedString(getFullPath()), e1.getFailure());
+ }
+ }
+ }
+
+ final VersionedObjectList succeeded = new VersionedObjectList(keys.size(), true, this.concurrencyChecksEnabled);
+ // If this is a transactional removeAll, we will not have version information as it is only generated at commit
+ // so treat transactional removeAll as if the server is not versioned.
+ // If we have no storage then act as if the server is not versioned.
+ final boolean serverIsVersioned = proxyResult != null && proxyResult.regionIsVersioned()
+ && !isTX() && getDataPolicy().withStorage();
+ if (!serverIsVersioned && !partialResult) {
+ // since the server is not versioned and we do not have a partial result
+ // get rid of the proxyResult info returned by the server.
+ proxyResult = null;
+ }
+ lockRVVForBulkOp();
+ try {
+
+ final DistributedRemoveAllOperation op = removeAllOp;
+ int size = (proxyResult == null)? keys.size() : proxyResult.size();
+
+ if (isInternalRegion()) {
+ if (isTraceEnabled) {
+ logger.trace(
+ "size of removeAll result is {} keys are {} proxyResult is {}", size, keys, proxyResult);
+ } else {
+ if (isTraceEnabled) {
+ logger.trace(
+ "size of removeAll result is {} keys are {} proxyResult is {}", size, keys, proxyResult);
+ }
+ }
+ } else {
+ if (isTraceEnabled) {
+ logger.trace(
+ "size of removeAll result is {} keys are {} proxyResult is {}", size, keys, proxyResult);
+ }
+ }
+
+ final PutAllPartialResult partialKeys = new PutAllPartialResult(size);
+ final Iterator iterator;
+ final boolean isVersionedResults;
+ if (proxyResult != null) {
+ iterator = proxyResult.iterator();
+ isVersionedResults = true;
+ } else {
+ iterator = keys.iterator();
+ isVersionedResults = false;
+ }
+ Runnable r = new Runnable() {
+ public void run() {
+ int offset = 0;
+ EntryEventImpl tagHolder = EntryEventImpl.createVersionTagHolder();
+ while (iterator.hasNext()) {
+ stopper.checkCancelInProgress(null);
+ Object key;
+ VersionTag versionTag = null;
+ tagHolder.setVersionTag(null);
+ if (isVersionedResults) {
+ Map.Entry mapEntry = (Map.Entry)iterator.next();
+ key = mapEntry.getKey();
+ versionTag = ((VersionedObjectList.Entry)mapEntry).getVersionTag();
+ if (isDebugEnabled) {
+ logger.debug("removeAll key {} version={}",key, versionTag);
+ }
+ if (versionTag == null) {
+ if (isDebugEnabled) {
+ logger.debug("removeAll found invalid version tag, which means the entry is not found at server for key={}.", key);
+ }
+ succeeded.addKeyAndVersion(key, null);
+ continue;
+ }
+ // No need for special handling here in removeAll.
+ // We can just remove this key from the client with versionTag set to null.
+ } else {
+ key = iterator.next();
+ if (isInternalRegion()) {
+ if (isTraceEnabled) {
+ logger.trace("removeAll {}", key);
+ }
+ } else {
+ if (isTraceEnabled) {
+ logger.trace("removeAll {}", key);
+ }
+ }
+
+ }
+ try {
+ if (serverIsVersioned) {
+ if (isDebugEnabled) {
+ logger.debug("associating version tag with {} version={}", key, versionTag);
+ }
+ //If we have received a version tag from a server, add it to the event
+ tagHolder.setVersionTag(versionTag);
+ tagHolder.setFromServer(true);
+ } else if (retryVersions != null) {
+ VersionTag vt = retryVersions.get(offset);
+ if (vt != null) {
+ //If this is a retried event, and we have a version tag for the retry,
+ //add it to the event.
+ tagHolder.setVersionTag(vt);
+ }
+ }
+
+ basicEntryRemoveAll(key, op, offset, tagHolder);
+ // now we must check again since the cache may have closed during
+ // distribution causing this process to not receive and queue the
+ // event for clients
+ stopper.checkCancelInProgress(null);
+ succeeded.addKeyAndVersion(key, tagHolder.getVersionTag());
+ }
+ catch (Exception ex) {
+ partialKeys.saveFailedKey(key, ex);
+ }
+ offset++;
+ }
+ }
+ };
+ syncBulkOp(r, eventId);
+ if (partialKeys.hasFailure()) {
+ // Bug 51725: Now succeeded contains an order key list, may be missing the version tags.
+ // Save reference of succeeded into partialKeys. The succeeded may be modified by
+ // postRemoveAll() to fill in the version tags.
+ partialKeys.setSucceededKeysAndVersions(succeeded);
+ logger.info(LocalizedMessage.create(LocalizedStrings.Region_RemoveAll_Applied_PartialKeys_0_1,
+ new Object[] {getFullPath(), partialKeys}));
+ if (isDebugEnabled) {
+ logger.debug(partialKeys.detailString());
+ }
+ if (e == null) {
+ // if received exception from server first, ignore local exception
+ if (op.isBridgeOperation()) {
+ if (partialKeys.getFailure() instanceof CancelException) {
+ e = (CancelException)partialKeys.getFailure();
+ } else if (partialKeys.getFailure() instanceof LowMemoryException) {
+ throw partialKeys.getFailure(); // fix for #43589
+ } else {
+ e = new PutAllPartialResultException(partialKeys);
+ if (isDebugEnabled) {
+ logger.debug("basicRemoveAll:"+partialKeys.detailString());
+ }
+ }
+ } else {
+ throw partialKeys.getFailure();
+ }
+ }
+ }
+ }
+ catch (LowMemoryException lme) {
+ throw lme;
+ }
+ catch (RuntimeException ex) {
+ e = ex;
+ }
+ catch (Exception ex) {
+ e = new RuntimeException(ex);
+ } finally {
+ unlockRVVForBulkOp();
+ removeAllOp.getBaseEvent().release();
+ removeAllOp.freeOffHeapResources();
+ }
+ getDataView().postRemoveAll(removeAllOp, succeeded, this);
+ if (e != null) {
+ throw e;
+ }
+ return succeeded;
+ }
+
+ /**
+ * bug #46924 - putAll can be partially applied when a clear() occurs, leaving
+ * the cache in an inconsistent state. Set the RVV to "cache op in progress"
+ * so clear() will block until the putAll completes. This won't work for
+ * non-replicate regions though since they uses one-hop during basicPutPart2
+ * to get a valid version tag.
+ */
+ private void lockRVVForBulkOp() {
+ if (this.versionVector != null && this.dataPolicy.withReplication()) {
+ this.versionVector.lockForCacheModification(this);
+ }
+ }
+
+ private void unlockRVVForBulkOp() {
+ if (this.versionVector != null && this.dataPolicy.withReplication()) {
+ this.versionVector.releaseCacheModificationLock(this);
+ }
+ }
+
+
+ // split into a separate newPutAllOperation since SQLFabric may need to
+ // manipulate event before doing the put (e.g. posDup flag)
+ public final DistributedPutAllOperation newPutAllOperation(Map, ?> map, Object callbackArg) {
+ if (map == null) {
+ throw new NullPointerException(LocalizedStrings
+ .AbstractRegion_MAP_CANNOT_BE_NULL.toLocalizedString());
+ }
+ if (map.isEmpty()) {
+ return null;
+ }
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ discoverJTA();
+
+ // Create a dummy event for the PutAll operation. Always create a
+ // PutAll operation, even if there is no distribution, so that individual
+ // events can be tracked and handed off to callbacks in postPutAll
+ final EntryEventImpl event = EntryEventImpl.create(this,
+ Operation.PUTALL_CREATE, null, null, callbackArg, true, getMyId());
+
+ event.disallowOffHeapValues();
+ DistributedPutAllOperation dpao = new DistributedPutAllOperation(event, map.size(), false);
+ return dpao;
+ }
+ public final DistributedPutAllOperation newPutAllForPUTDmlOperation(Map, ?> map, Object callbackArg) {
+ DistributedPutAllOperation dpao = newPutAllOperation(map, callbackArg);
+ dpao.getEvent().setFetchFromHDFS(false);
+ dpao.getEvent().setPutDML(true);
+ return dpao;
+ }
+
+
+ public final DistributedRemoveAllOperation newRemoveAllOperation(Collection> keys, Object callbackArg) {
+ if (keys == null) {
+ throw new NullPointerException("The keys Collection passed to removeAll was null.");
+ }
+ if (keys.isEmpty()) {
+ return null;
+ }
+ checkReadiness();
+ checkForLimitedOrNoAccess();
+ discoverJTA();
+
+ // Create a dummy event for the removeAll operation. Always create a
+ // removeAll operation, even if there is no distribution, so that individual
+ // events can be tracked and handed off to callbacks in postRemoveAll
+ final EntryEventImpl event = EntryEventImpl.create(this, Operation.REMOVEALL_DESTROY, null,
+ null/* newValue */, callbackArg, false, getMyId());
+ event.disallowOffHeapValues();
+ return new DistributedRemoveAllOperation(event, keys.size(), false);
+ }
+
+
+ /**
+ * This performs the putAll operation for a specific key and value
+ * @param key the cache key
+ * @param value the cache value
+ * @param putallOp the DistributedPutAllOperation associated with the event
+ * @param tagHolder holder for version tag
+ * @throws TimeoutException if the operation times out
+ * @throws CacheWriterException if a cache writer objects to the update
+ */
+ protected final void basicEntryPutAll(Object key,
+ Object value,
+ DistributedPutAllOperation putallOp,
+ int offset,
+ EntryEventImpl tagHolder)
+ throws TimeoutException, CacheWriterException {
+ assert putallOp != null;
+// long startPut = CachePerfStats.getStatTime();
+ checkReadiness();
+ if (value == null) {
+ throw new NullPointerException(LocalizedStrings.LocalRegion_VALUE_CANNOT_BE_NULL.toLocalizedString());
+ }
+ validateArguments(key, value, null);
+ // event is marked as a PUTALL_CREATE but if the entry exists it
+ // will be changed to a PUTALL_UPDATE later on.
+ EntryEventImpl event = EntryEventImpl.createPutAllEvent(
+ putallOp, this, Operation.PUTALL_CREATE, key, value);
+
+ try {
+ event.setFetchFromHDFS(putallOp.getEvent().isFetchFromHDFS());
+ event.setPutDML(putallOp.getEvent().isPutDML());
+
+ if (tagHolder != null) {
+ event.setVersionTag(tagHolder.getVersionTag());
+ event.setFromServer(tagHolder.isFromServer());
+ }
+ if (generateEventID()) {
+ event.setEventId(new EventID(putallOp.getBaseEvent().getEventId(), offset));
+ }
+ discoverJTA(); // TODO: This could be called once for the entire putAll instead of calling it for every key
+
+ /*
+ * If this is tx, do putEntry, unless it is a local region?
+ */
+ performPutAllEntry(event);
+ if (tagHolder != null) {
+ tagHolder.setVersionTag(event.getVersionTag());
+ tagHolder.isConcurrencyConflict(event.isConcurrencyConflict());
+ }
+ } finally {
+ event.release();
+ }
+ }
+
+
+ protected final void basicEntryRemoveAll(Object key,
+ DistributedRemoveAllOperation op,
+ int offset,
+ EntryEventImpl tagHolder)
+ throws TimeoutException, CacheWriterException {
+ assert op != null;
+ checkReadiness();
+ validateKey(key);
+ EntryEventImpl event = EntryEventImpl.createRemoveAllEvent(op, this, key);
+ try {
+ if (tagHolder != null) {
+ event.setVersionTag(tagHolder.getVersionTag());
+ event.setFromServer(tagHolder.isFromServer());
+ }
+ if (generateEventID()) {
+ event.setEventId(new EventID(op.getBaseEvent().getEventId(), offset));
+ }
+ discoverJTA(); // TODO: This could be called once for the entire removeAll instead of calling it for every key
+
+ /*
+ * If this is tx, do removeEntry, unless it is a local region?
+ */
+ try {
+ performRemoveAllEntry(event);
+ } catch (EntryNotFoundException ignore) {
+ if (event.getVersionTag() == null) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("RemoveAll encoutered EntryNotFoundException: event={}", event);
+ }
+ }
+ }
+ if (tagHolder != null) {
+ tagHolder.setVersionTag(event.getVersionTag());
+ tagHolder.isConcurrencyConflict(event.isConcurrencyConflict());
+ }
+ } finally {
+ event.release();
+ }
+ }
+ public void performPutAllEntry(EntryEventImpl event) {
+ getDataView().putEntry(event, false, false, null, false, 0L, false);
+ }
+ public void performRemoveAllEntry(EntryEventImpl event) {
+ basicDestroy(event, true, null);
+ //getDataView().destroyExistingEntry(event, true, null);
+ }
+
+
+ public void postPutAllFireEvents(DistributedPutAllOperation putallOp, VersionedObjectList successfulPuts)
+ {
+ if (!this.dataPolicy.withStorage() && this.concurrencyChecksEnabled
+ && putallOp.getBaseEvent().isBridgeEvent()) {
+ // if there is no local storage we need to transfer version information
+ // to the successfulPuts list for transmission back to the client
+ successfulPuts.clear();
+ putallOp.fillVersionedObjectList(successfulPuts);
+ }
+ Set successfulKeys = new HashSet(successfulPuts.size());
+ for (Object key: successfulPuts.getKeys()) {
+ successfulKeys.add(key);
+ }
+ for (Iterator it=putallOp.eventIterator(); it.hasNext(); ) {
+ EntryEventImpl event = (EntryEventImpl)it.next();
+ if (successfulKeys.contains(event.getKey())) {
+ EnumListenerEvent op = event.getOperation().isCreate() ? EnumListenerEvent.AFTER_CREATE
+ : EnumListenerEvent.AFTER_UPDATE;
+ invokePutCallbacks(op, event, !event.callbacksInvoked() && !event.isPossibleDuplicate(),
+ false /* We must notify gateways inside RegionEntry lock, NOT here, to preserve the order of events sent by gateways for same key*/);
+ }
+ }
+ }
+
+ public void postRemoveAllFireEvents(DistributedRemoveAllOperation op, VersionedObjectList successfulOps) {
+ if (!this.dataPolicy.withStorage() && this.concurrencyChecksEnabled
+ && op.getBaseEvent().isBridgeEvent()) {
+ // if there is no local storage we need to transfer version information
+ // to the successfulOps list for transmission back to the client
+ successfulOps.clear();
+ op.fillVersionedObjectList(successfulOps);
+ }
+ Set successfulKeys = new HashSet(successfulOps.size());
+ for (Object key: successfulOps.getKeys()) {
+ successfulKeys.add(key);
+ }
+ for (Iterator it=op.eventIterator(); it.hasNext(); ) {
+ EntryEventImpl event = (EntryEventImpl)it.next();
+ if (successfulKeys.contains(event.getKey())) {
+ invokeDestroyCallbacks(EnumListenerEvent.AFTER_DESTROY, event, !event.callbacksInvoked() && !event.isPossibleDuplicate(),
+ false /* We must notify gateways inside RegionEntry lock, NOT here, to preserve the order of events sent by gateways for same key*/);
+ }
+ }
+ }
+
+ public void postPutAllSend(DistributedPutAllOperation putallOp, VersionedObjectList successfulPuts)
+ {
+ /* No-op for local region of course */
+ }
+ public void postRemoveAllSend(DistributedRemoveAllOperation op, VersionedObjectList successfulOps)
+ {
+ /* No-op for local region of course */
+ }
+
+
+
+ /**
+ * DistributedRegion overrides isCurrentlyLockGrantor
+ *
+ * @see DistributedRegion#isCurrentlyLockGrantor()
+ */
+ @Override
+ protected boolean isCurrentlyLockGrantor()
+ {
+ return false;
+ }
+
+ /**
+ * Handle a local region destroy or a region close that was done on this
+ * region in a remote vm. Currently the only thing needed is to have the
+ * advisor
+ *
+ * @param sender the id of the member that did the remote operation
+ * @param topSerial the remote serialNumber for the top region (maybe root)
+ * @param subregionSerialNumbers map of remote subregions to serialNumbers
+ * @param regionDestroyed true if the region was destroyed on the remote host (as opposed to closed)
+ * @since 5.0
+ */
+ final void handleRemoteLocalRegionDestroyOrClose(
+ InternalDistributedMember sender,
+ int topSerial,
+ Map subregionSerialNumbers, boolean regionDestroyed) {
+
+ final int oldLevel = setThreadInitLevelRequirement(LocalRegion.ANY_INIT);
+ // go
+ // through
+ // initialization
+ // latches
+ try {
+ basicHandleRemoteLocalRegionDestroyOrClose(
+ sender, topSerial, subregionSerialNumbers, false, regionDestroyed);
+ }
+ finally {
+ setThreadInitLevelRequirement(oldLevel);
+ }
+ }
+
+ /**
+ * Does the core work for handleRemoteLocalRegionDestroyOrClose.
+ *
+ * @param sender the id of the member that did the remote operation
+ * @param topSerial the remote serialNumber for the top region (maybe root)
+ * @param subregionSerialNumbers remote map of subregions to serialNumbers
+ * @param regionDestroyed
+ * @since 5.0
+ */
+ private final void basicHandleRemoteLocalRegionDestroyOrClose(
+ InternalDistributedMember sender,
+ int topSerial,
+ Map subregionSerialNumbers,
+ boolean subregion, boolean regionDestroyed) {
+
+ // use topSerial unless this region is in subregionSerialNumbers map
+ int serialForThisRegion = topSerial;
+
+ if (subregion) {
+ // OPTIMIZE: we don't know if this rgn is a subregion or the top region
+ Integer serialNumber = (Integer) subregionSerialNumbers.get(getFullPath());
+ if (serialNumber == null) {
+ // sender didn't have this subregion
+ return;
+ }
+ else {
+ // non-null means this is a subregion under the destroyed region
+ serialForThisRegion = serialNumber.intValue();
+ }
+ }
+
+ // remove sender's serialForThisRegion from the advisor
+ removeSenderFromAdvisor(sender, serialForThisRegion, regionDestroyed);
+
+ // process subregions...
+ for (Iterator itr = this.subregions.values().iterator(); itr.hasNext();) {
+ LocalRegion r = toRegion(itr.next());
+ if (r != null && !r.isDestroyed()) {
+ // recursively call basicHandleRemoteLocalRegionDestroyOrClose for subregions
+ r.basicHandleRemoteLocalRegionDestroyOrClose(
+ sender, topSerial, subregionSerialNumbers, true, regionDestroyed);
+ }
+ }
+ }
+
+ /**
+ * Remove the specified sender from this regions advisor.
+ * @param regionDestroyed
+ *
+ * @since 5.0
+ */
+ protected void removeSenderFromAdvisor(InternalDistributedMember sender, int serial, boolean regionDestroyed)
+ {
+ // nothing needs to be done here since LocalRegion does not have an advisor.
+ }
+
+ /**
+ * @return Returns the isUsedForPartitionedRegionAdmin.
+ */
+ final public boolean isUsedForPartitionedRegionAdmin()
+ {
+ return this.isUsedForPartitionedRegionAdmin;
+ }
+
+ /**
+ * This method determines whether this region should synchronize with peer replicated regions
+ * when the given member has crashed.
+ * @param id the crashed member
+ * @return true if synchronization should be attempted
+ */
+ public boolean shouldSyncForCrashedMember(InternalDistributedMember id) {
+ return this.concurrencyChecksEnabled
+ && this.dataPolicy.withReplication()
+ && !this.isUsedForPartitionedRegionAdmin
+ && !this.isUsedForMetaRegion
+ && !this.isUsedForSerialGatewaySenderQueue;
+ }
+
+ /**
+ * forces the diskRegion to switch the oplog
+ * @since 5.1
+ */
+ public void forceRolling() throws DiskAccessException {
+ if(this.diskRegion!=null){
+ diskRegion.forceRolling();
+ }
+ }
+
+ /**
+ * @deprecated as of prPersistSprint1 use forceCompaction instead
+ */
+ @Deprecated
+ public boolean notifyToRoll() {
+ return forceCompaction();
+ }
+
+ /**
+ * filterProfile holds CQ and registerInterest information for clients
+ * having this region
+ */
+ FilterProfile filterProfile;
+
+ /**
+ *
+ * @return int array containing the IDs of the oplogs which will potentially
+ * get rolled else null if no oplogs were available at the time of signal or region
+ * is not having disk persistence. Pls note that the actual number of oplogs
+ * rolled may be more than what is indicated
+ * @since prPersistSprint1
+ */
+ @Override
+ public boolean forceCompaction()
+ {
+ DiskRegion dr = getDiskRegion();
+ if (dr != null) {
+ if (dr.isCompactionPossible()) {
+ return dr.forceCompaction();
+ } else {
+ throw new IllegalStateException("To call notifyToCompact you must configure the region with ");
+ }
+ }
+ else {
+ return false;
+ }
+ }
+
+ @Override
+ public File[] getDiskDirs() {
+ if (getDiskStore() != null) {
+ return getDiskStore().getDiskDirs();
+ } else {
+ return this.diskDirs;
+ }
+ }
+
+ public int[] getDiskDirSizes() {
+ if (getDiskStore() != null) {
+ return getDiskStore().getDiskDirSizes();
+ } else {
+ return this.diskSizes;
+ }
+ }
+
+
+ /**
+ * @return Returns the isUsedForPartitionedRegionBucket.
+ */
+ public boolean isUsedForPartitionedRegionBucket()
+ {
+ return this.isUsedForPartitionedRegionBucket;
+ }
+
+ public boolean isUsedForSerialGatewaySenderQueue() {
+ return this.isUsedForSerialGatewaySenderQueue;
+ }
+
+ public boolean isUsedForParallelGatewaySenderQueue() {
+ return this.isUsedForParallelGatewaySenderQueue;
+ }
+
+ public AbstractGatewaySender getSerialGatewaySender(){
+ return this.serialGatewaySender;
+ }
+
+ public boolean isParallelWanEnabled() {
+ Set regionGatewaySenderIds = this.getAllGatewaySenderIds();
+ if (regionGatewaySenderIds.isEmpty()) {
+ return false;
+ }
+ Set cacheGatewaySenders = getCache().getAllGatewaySenders();
+ for (GatewaySender sender : cacheGatewaySenders) {
+ if (regionGatewaySenderIds.contains(sender.getId())
+ && sender.isParallel()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * A convenience method to get the PartitionedRegion for a Bucket
+ * @return If this is an instance of {@link BucketRegion}, returns the
+ * {@link PartitionedRegion} otherwise throws an IllegalArgumentException
+ */
+ public PartitionedRegion getPartitionedRegion() {
+ if (!this.isUsedForPartitionedRegionBucket) {
+ throw new IllegalArgumentException();
+ }
+ return ((BucketRegion)this).getPartitionedRegion();
+ }
+ /**
+ * @return Returns the isUsedForMetaRegion.
+ */
+ final public boolean isUsedForMetaRegion()
+ {
+ return this.isUsedForMetaRegion;
+ }
+
+ final public boolean isMetaRegionWithTransactions()
+ {
+ return this.isMetaRegionWithTransactions;
+ }
+
+ /**
+ * @return true if this is not a user visible region
+ */
+ final public boolean isInternalRegion(){
+ return isSecret() || isUsedForMetaRegion() || isUsedForPartitionedRegionAdmin()
+ || isUsedForPartitionedRegionBucket();
+ }
+
+ public LoaderHelper createLoaderHelper(Object key, Object callbackArgument,
+ boolean netSearchAllowed, boolean netLoadAllowed,
+ SearchLoadAndWriteProcessor searcher)
+ {
+ return new LoaderHelperImpl(this, key, callbackArgument, netSearchAllowed,
+ netLoadAllowed, searcher);
+ }
+
+ /** visitor over the CacheProfiles to check if the region has a CacheLoader */
+ private static final DistributionAdvisor.ProfileVisitor