You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by es...@apache.org on 2017/10/12 00:29:35 UTC

[geode] branch feature/GEODE-3521 updated: Add a SystemPropertyHelper class to held system properties to be set.

This is an automated email from the ASF dual-hosted git repository.

eshu11 pushed a commit to branch feature/GEODE-3521
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/feature/GEODE-3521 by this push:
     new 4767785  Add a SystemPropertyHelper class to held system properties to be set.
4767785 is described below

commit 4767785b34c8eeaa05983b446a30db74e8afbf9f
Author: eshu <es...@pivotal.io>
AuthorDate: Wed Oct 11 17:24:20 2017 -0700

    Add a SystemPropertyHelper class to held system properties to be set.
    
      Add a PausedTXStateProxyImpl state to allow product to know if a transaction is paused.
      Product now detects if only same thread can resume a suspended transaction.
---
 .../geode/cache/query/internal/DefaultQuery.java   |   4 +-
 .../cache/query/internal/index/IndexManager.java   |  10 +-
 .../geode/internal/cache/DistributedRegion.java    |   4 +-
 .../geode/internal/cache/GemFireCacheImpl.java     |   4 +-
 .../apache/geode/internal/cache/LocalRegion.java   |  39 +-
 .../internal/cache/PartitionedRegionDataView.java  |   8 +-
 .../internal/cache/PausedTXStateProxyImpl.java     | 422 +++++++++++++++++++++
 .../apache/geode/internal/cache/TXManagerImpl.java |  90 ++++-
 .../org/apache/geode/internal/cache/TXState.java   |   4 +-
 .../apache/geode/internal/cache/TXStateProxy.java  |   8 +-
 .../geode/internal/cache/TXStateProxyImpl.java     |  39 +-
 .../geode/internal/lang/SystemPropertyHelper.java  |  52 +++
 .../geode/pdx/internal/PeerTypeRegistration.java   |   4 +-
 .../test/java/org/apache/geode/TX2JUnitTest.java   |  35 +-
 .../java/org/apache/geode/TXExpiryJUnitTest.java   |   4 +-
 .../test/java/org/apache/geode/TXJUnitTest.java    | 194 +++++-----
 .../geode/cache30/MultiVMRegionTestCase.java       |  12 +-
 .../cache/ClientServerTransactionDUnitTest.java    |  24 +-
 .../internal/cache/RemoteTransactionDUnitTest.java | 100 ++---
 .../cache/TransactionsWithDeltaDUnitTest.java      |   4 +-
 .../cache/execute/MyTransactionFunction.java       |  16 +-
 .../internal/cache/execute/PRJTADUnitTest.java     |  65 ++--
 .../cache/execute/PRTransaction2DUnitTest.java     |  61 ++-
 .../internal/jta/ClientServerJTADUnitTest.java     |   8 +-
 .../internal/jta/JtaIntegrationJUnitTest.java      |  43 +--
 .../lang/SystemPropertyHelperJUnitTest.java        |  56 +++
 26 files changed, 915 insertions(+), 395 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java
index c242d89..01ecd8f 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java
@@ -538,7 +538,7 @@ public class DefaultQuery implements Query {
     QueryObserver observer = QueryObserverHolder.getInstance();
 
     long startTime = CachePerfStats.getStatTime();
-    TXStateProxy tx = ((TXManagerImpl) this.cache.getCacheTransactionManager()).internalSuspend();
+    TXStateProxy tx = ((TXManagerImpl) this.cache.getCacheTransactionManager()).pauseTransaction();
     try {
       observer.startQuery(this);
       observer.beforeQueryEvaluation(this.compiledQuery, context);
@@ -575,7 +575,7 @@ public class DefaultQuery implements Query {
       updateStatistics(endTime - startTime);
       pdxClassToFieldsMap.remove();
       pdxClassToMethodsMap.remove();
-      ((TXManagerImpl) this.cache.getCacheTransactionManager()).internalResume(tx);
+      ((TXManagerImpl) this.cache.getCacheTransactionManager()).unpauseTransaction(tx);
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java
index 12e7fd0..c61a497 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java
@@ -277,7 +277,7 @@ public class IndexManager {
 
     TXStateProxy tx = null;
     if (!((InternalCache) this.region.getCache()).isClient()) {
-      tx = ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).internalSuspend();
+      tx = ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).pauseTransaction();
     }
 
     try {
@@ -431,7 +431,8 @@ public class IndexManager {
       DefaultQuery.setPdxReadSerialized(this.region.getCache(), oldReadSerialized);
 
       if (tx != null) {
-        ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).internalResume(tx);
+        ((TXManagerImpl) this.region.getCache().getCacheTransactionManager())
+            .unpauseTransaction(tx);
       }
     }
   }
@@ -1005,7 +1006,7 @@ public class IndexManager {
     DefaultQuery.setPdxReadSerialized(this.region.getCache(), true);
     TXStateProxy tx = null;
     if (!((InternalCache) this.region.getCache()).isClient()) {
-      tx = ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).internalSuspend();
+      tx = ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).pauseTransaction();
     }
 
     try {
@@ -1149,7 +1150,8 @@ public class IndexManager {
     } finally {
       DefaultQuery.setPdxReadSerialized(this.region.getCache(), false);
       if (tx != null) {
-        ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).internalResume(tx);
+        ((TXManagerImpl) this.region.getCache().getCacheTransactionManager())
+            .unpauseTransaction(tx);
       }
       getCachePerfStats().endIndexUpdate(startPA);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
index 20d9f15..c76e813 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
@@ -3833,7 +3833,7 @@ public class DistributedRegion extends LocalRegion implements CacheDistributionA
   protected VersionTag fetchRemoteVersionTag(Object key) {
     VersionTag tag = null;
     assert this.dataPolicy != DataPolicy.REPLICATE;
-    final TXStateProxy tx = cache.getTXMgr().internalSuspend();
+    final TXStateProxy tx = cache.getTXMgr().pauseTransaction();
     try {
       boolean retry = true;
       InternalDistributedMember member = getRandomReplicate();
@@ -3856,7 +3856,7 @@ public class DistributedRegion extends LocalRegion implements CacheDistributionA
       }
     } finally {
       if (tx != null) {
-        cache.getTXMgr().internalResume(tx);
+        cache.getTXMgr().unpauseTransaction(tx);
       }
     }
     return tag;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index e2975ea..f8624ec 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -2155,7 +2155,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       try {
 
         if (this.transactionManager != null) {
-          tx = this.transactionManager.internalSuspend();
+          tx = this.transactionManager.pauseTransaction();
         }
 
         // do this before closing regions
@@ -2394,7 +2394,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
         }
         ((DynamicRegionFactoryImpl) DynamicRegionFactory.get()).close();
         if (this.transactionManager != null) {
-          this.transactionManager.internalResume(tx);
+          this.transactionManager.unpauseTransaction(tx);
         }
         TXCommitMessage.getTracker().clearForCacheClose();
       }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 87bb481..420ca4a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -202,6 +202,7 @@ import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.internal.cache.wan.GatewaySenderEventCallbackArgument;
 import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
 import org.apache.geode.internal.logging.log4j.LogMarker;
@@ -6149,7 +6150,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       boolean callbackEvents) throws CacheWriterException, TimeoutException {
     preDestroyChecks();
 
-    final TXStateProxy tx = this.cache.getTXMgr().internalSuspend();
+    final TXStateProxy tx = this.cache.getTXMgr().pauseTransaction();
     try {
       boolean acquiredLock = false;
       if (lock) {
@@ -6264,7 +6265,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       }
 
     } finally {
-      this.cache.getTXMgr().internalResume(tx);
+      this.cache.getTXMgr().unpauseTransaction(tx);
     }
   }
 
@@ -6400,22 +6401,8 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     getDataView().destroyExistingEntry(event, cacheWrite, expectedOldValue);
   }
 
-  /**
-   * A set operation now can bootstrap a transaction now. User need to specifically disable this by
-   * setting this system property to true to get the old behavior.
-   *
-   * @Since Geode 1.3.0
-   */
-  public static final String PREVENT_SET_OP_BOOTSTRAP_TRANSACTION =
-      "preventSetOpBootstrapTransaction";
-
-  protected static boolean isSetOpBootstrapTransactionDisabled() {
-    return Boolean
-        .getBoolean(DistributionConfig.GEMFIRE_PREFIX + PREVENT_SET_OP_BOOTSTRAP_TRANSACTION)
-        || Boolean.getBoolean("geode." + PREVENT_SET_OP_BOOTSTRAP_TRANSACTION);
-  }
-
-  protected final boolean preventSetOpBootstrapTransaction = isSetOpBootstrapTransactionDisabled();
+  protected final boolean preventSetOpBootstrapTransaction =
+      SystemPropertyHelper.preventSetOpBootstrapTransaction();
 
   /**
    * Do the expensive work of discovering an existing JTA transaction Only needs to be called at
@@ -6430,9 +6417,9 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     }
   }
 
-  private boolean isTransactionInternalSuspended() {
+  private boolean isTransactionPaused() {
     TXManagerImpl txMgr = (TXManagerImpl) getCache().getCacheTransactionManager();
-    return txMgr.isTransactionInternalSuspendedByThread(Thread.currentThread());
+    return txMgr.isTransactionPaused();
   }
 
   /**
@@ -6834,7 +6821,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   }
 
   void basicInvalidateRegion(RegionEventImpl event) {
-    final TXStateProxy tx = this.cache.getTXMgr().internalSuspend();
+    final TXStateProxy tx = this.cache.getTXMgr().pauseTransaction();
     try {
       this.regionInvalid = true;
       getImageState().setRegionInvalidated(true);
@@ -6878,7 +6865,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       }
 
     } finally {
-      this.cache.getTXMgr().internalResume(tx);
+      this.cache.getTXMgr().unpauseTransaction(tx);
     }
   }
 
@@ -8313,8 +8300,8 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
               || jtaTransaction.getStatus() == Status.STATUS_NO_TRANSACTION) {
             return null;
           }
-          if (isTransactionInternalSuspended()) {
-            // Do not bootstrap JTA again, if the thread has been internal suspended.
+          if (isTransactionPaused()) {
+            // Do not bootstrap JTA again, if the transaction has been paused.
             return null;
           }
           txState = this.cache.getTXMgr().beginJTA();
@@ -9222,7 +9209,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
             if (!alreadyInvalid(key, event)) {
               // bug #47716 - don't update if it's already here & invalid
-              TXStateProxy txState = this.cache.getTXMgr().internalSuspend();
+              TXStateProxy txState = this.cache.getTXMgr().pauseTransaction();
               try {
                 basicPutEntry(event, 0L);
               } catch (ConcurrentCacheModificationException e) {
@@ -9232,7 +9219,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
                       key, e);
                 }
               } finally {
-                this.cache.getTXMgr().internalResume(txState);
+                this.cache.getTXMgr().unpauseTransaction(txState);
               }
               getCachePerfStats().endPut(startPut, event.isOriginRemote());
             }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataView.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataView.java
index 998f944..489ad33 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataView.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataView.java
@@ -53,12 +53,12 @@ public class PartitionedRegionDataView extends LocalRegionDataView {
 
   @Override
   public Entry getEntry(KeyInfo keyInfo, LocalRegion localRegion, boolean allowTombstones) {
-    TXStateProxy tx = localRegion.cache.getTXMgr().internalSuspend();
+    TXStateProxy tx = localRegion.cache.getTXMgr().pauseTransaction();
     try {
       PartitionedRegion pr = (PartitionedRegion) localRegion;
       return pr.nonTXGetEntry(keyInfo, false, allowTombstones);
     } finally {
-      localRegion.cache.getTXMgr().internalResume(tx);
+      localRegion.cache.getTXMgr().unpauseTransaction(tx);
     }
   }
 
@@ -67,12 +67,12 @@ public class PartitionedRegionDataView extends LocalRegionDataView {
       Object value, boolean disableCopyOnRead, boolean preferCD,
       ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent,
       boolean returnTombstones) {
-    TXStateProxy tx = r.cache.getTXMgr().internalSuspend();
+    TXStateProxy tx = r.cache.getTXMgr().pauseTransaction();
     try {
       return r.findObjectInSystem(key, isCreate, tx, generateCallbacks, value, disableCopyOnRead,
           preferCD, requestingClient, clientEvent, returnTombstones);
     } finally {
-      r.cache.getTXMgr().internalResume(tx);
+      r.cache.getTXMgr().unpauseTransaction(tx);
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PausedTXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PausedTXStateProxyImpl.java
new file mode 100644
index 0000000..de9e91e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PausedTXStateProxyImpl.java
@@ -0,0 +1,422 @@
+/*
+ * 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 org.apache.geode.internal.cache;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CommitConflictException;
+import org.apache.geode.cache.EntryNotFoundException;
+import org.apache.geode.cache.Region.Entry;
+import org.apache.geode.cache.TransactionId;
+import org.apache.geode.cache.UnsupportedOperationInTransactionException;
+import org.apache.geode.cache.client.internal.ServerRegionDataAccess;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
+import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
+import org.apache.geode.internal.cache.tx.TransactionalOperation.ServerRegionOperation;
+
+public class PausedTXStateProxyImpl implements TXStateProxy {
+
+  @Override
+  public TransactionId getTransactionId() {
+    return null;
+  }
+
+  @Override
+  public TXRegionState readRegion(LocalRegion r) {
+    return null;
+  }
+
+  @Override
+  public TXRegionState writeRegion(LocalRegion r) {
+    return null;
+  }
+
+  @Override
+  public long getBeginTime() {
+    return 0;
+  }
+
+  @Override
+  public int getChanges() {
+    return 0;
+  }
+
+  @Override
+  public boolean isInProgress() {
+    return false;
+  }
+
+  @Override
+  public int nextModSerialNum() {
+    return 0;
+  }
+
+  @Override
+  public boolean needsLargeModCount() {
+    return false;
+  }
+
+  @Override
+  public void precommit()
+      throws CommitConflictException, UnsupportedOperationInTransactionException {}
+
+  @Override
+  public void commit() throws CommitConflictException {}
+
+  @Override
+  public void rollback() {}
+
+  @Override
+  public List getEvents() {
+    return null;
+  }
+
+  @Override
+  public Cache getCache() {
+    return null;
+  }
+
+  @Override
+  public Collection<LocalRegion> getRegions() {
+    return null;
+  }
+
+  @Override
+  public void invalidateExistingEntry(EntryEventImpl event, boolean invokeCallbacks,
+      boolean forceNewEntry) {}
+
+  @Override
+  public Entry getEntry(KeyInfo keyInfo, LocalRegion region, boolean allowTombstones) {
+    return null;
+  }
+
+  @Override
+  public Object getDeserializedValue(KeyInfo keyInfo, LocalRegion localRegion, boolean updateStats,
+      boolean disableCopyOnRead, boolean preferCD, EntryEventImpl clientEvent,
+      boolean returnTombstones, boolean retainResult) {
+    return null;
+  }
+
+  @Override
+  public TXEvent getEvent() {
+    return null;
+  }
+
+  @Override
+  public TXRegionState txWriteRegion(LocalRegion localRegion, KeyInfo entryKey) {
+    return null;
+  }
+
+  @Override
+  public TXRegionState txReadRegion(LocalRegion localRegion) {
+    return null;
+  }
+
+  @Override
+  public boolean txPutEntry(EntryEventImpl event, boolean ifNew, boolean requireOldValue,
+      boolean checkResources, Object expectedOldValue) {
+    return false;
+  }
+
+  @Override
+  public TXEntryState txReadEntry(KeyInfo entryKey, LocalRegion localRegion, boolean rememberRead,
+      boolean createTxEntryIfAbsent) {
+    return null;
+  }
+
+  @Override
+  public void rmRegion(LocalRegion r) {
+
+  }
+
+  @Override
+  public boolean isInProgressAndSameAs(TXStateInterface state) {
+    return false;
+  }
+
+  @Override
+  public boolean isFireCallbacks() {
+    return false;
+  }
+
+  @Override
+  public ReentrantLock getLock() {
+    return null;
+  }
+
+  @Override
+  public boolean isRealDealLocal() {
+    return false;
+  }
+
+  @Override
+  public boolean isMemberIdForwardingRequired() {
+    return false;
+  }
+
+  @Override
+  public InternalDistributedMember getOriginatingMember() {
+    return null;
+  }
+
+  @Override
+  public TXCommitMessage getCommitMessage() {
+    return null;
+  }
+
+  @Override
+  public void close() {}
+
+  @Override
+  public boolean isTxState() {
+    return false;
+  }
+
+  @Override
+  public boolean isTxStateStub() {
+    return false;
+  }
+
+  @Override
+  public boolean isTxStateProxy() {
+    return false;
+  }
+
+  @Override
+  public boolean isDistTx() {
+    return false;
+  }
+
+  @Override
+  public boolean isCreatedOnDistTxCoordinator() {
+    return false;
+  }
+
+  @Override
+  public void beforeCompletion() {}
+
+  @Override
+  public void afterCompletion(int status) {}
+
+  @Override
+  public void destroyExistingEntry(EntryEventImpl event, boolean cacheWrite,
+      Object expectedOldValue) throws EntryNotFoundException {}
+
+  @Override
+  public int entryCount(LocalRegion localRegion) {
+    return 0;
+  }
+
+  @Override
+  public Object getValueInVM(KeyInfo keyInfo, LocalRegion localRegion, boolean rememberRead) {
+    return null;
+  }
+
+  @Override
+  public boolean containsKey(KeyInfo keyInfo, LocalRegion localRegion) {
+    return false;
+  }
+
+  @Override
+  public boolean containsValueForKey(KeyInfo keyInfo, LocalRegion localRegion) {
+    return false;
+  }
+
+  @Override
+  public Entry getEntryOnRemote(KeyInfo key, LocalRegion localRegion, boolean allowTombstones)
+      throws DataLocationException {
+    return null;
+  }
+
+  @Override
+  public boolean putEntry(EntryEventImpl event, boolean ifNew, boolean ifOld,
+      Object expectedOldValue, boolean requireOldValue, long lastModified,
+      boolean overwriteDestroyed) {
+    return false;
+  }
+
+  @Override
+  public boolean putEntryOnRemote(EntryEventImpl event, boolean ifNew, boolean ifOld,
+      Object expectedOldValue, boolean requireOldValue, long lastModified,
+      boolean overwriteDestroyed) throws DataLocationException {
+    return false;
+  }
+
+  @Override
+  public void destroyOnRemote(EntryEventImpl event, boolean cacheWrite, Object expectedOldValue)
+      throws DataLocationException {}
+
+  @Override
+  public void invalidateOnRemote(EntryEventImpl event, boolean invokeCallbacks,
+      boolean forceNewEntry) throws DataLocationException {}
+
+  @Override
+  public boolean isDeferredStats() {
+    return false;
+  }
+
+  @Override
+  public Object findObject(KeyInfo key, LocalRegion r, boolean isCreate, boolean generateCallbacks,
+      Object value, boolean disableCopyOnRead, boolean preferCD,
+      ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent,
+      boolean returnTombstones) {
+    return null;
+  }
+
+  @Override
+  public Object getEntryForIterator(KeyInfo key, LocalRegion currRgn, boolean rememberReads,
+      boolean allowTombstones) {
+    return null;
+  }
+
+  @Override
+  public Object getKeyForIterator(KeyInfo keyInfo, LocalRegion currRgn, boolean rememberReads,
+      boolean allowTombstones) {
+    return null;
+  }
+
+  @Override
+  public Set getAdditionalKeysForIterator(LocalRegion currRgn) {
+    return null;
+  }
+
+  @Override
+  public Collection<?> getRegionKeysForIteration(LocalRegion currRegion) {
+    return null;
+  }
+
+  @Override
+  public Object getSerializedValue(LocalRegion localRegion, KeyInfo key, boolean doNotLockEntry,
+      ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent,
+      boolean returnTombstones) throws DataLocationException {
+    return null;
+  }
+
+  @Override
+  public void checkSupportsRegionDestroy() throws UnsupportedOperationInTransactionException {}
+
+  @Override
+  public void checkSupportsRegionInvalidate() throws UnsupportedOperationInTransactionException {}
+
+  @Override
+  public void checkSupportsRegionClear() throws UnsupportedOperationInTransactionException {}
+
+  @Override
+  public Set getBucketKeys(LocalRegion localRegion, int bucketId, boolean allowTombstones) {
+    return null;
+  }
+
+  @Override
+  public void postPutAll(DistributedPutAllOperation putallOp, VersionedObjectList successfulPuts,
+      LocalRegion region) {}
+
+  @Override
+  public void postRemoveAll(DistributedRemoveAllOperation op, VersionedObjectList successfulOps,
+      LocalRegion region) {}
+
+  @Override
+  public Entry accessEntry(KeyInfo keyInfo, LocalRegion localRegion) {
+    return null;
+  }
+
+  @Override
+  public void updateEntryVersion(EntryEventImpl event) throws EntryNotFoundException {}
+
+  @Override
+  public void checkJTA(String errmsg) throws IllegalStateException {}
+
+  @Override
+  public void setIsJTA(boolean isJTA) {}
+
+  @Override
+  public TXId getTxId() {
+    return null;
+  }
+
+  @Override
+  public TXManagerImpl getTxMgr() {
+    return null;
+  }
+
+  @Override
+  public void setLocalTXState(TXStateInterface state) {}
+
+  @Override
+  public void setTarget(DistributedMember target) {}
+
+  @Override
+  public DistributedMember getTarget() {
+    return null;
+  }
+
+  @Override
+  public boolean isCommitOnBehalfOfRemoteStub() {
+    return false;
+  }
+
+  @Override
+  public boolean setCommitOnBehalfOfRemoteStub(boolean requestedByOwner) {
+    return false;
+  }
+
+  @Override
+  public boolean isOnBehalfOfClient() {
+    return false;
+  }
+
+  @Override
+  public boolean isJCATransaction() {
+    return false;
+  }
+
+  @Override
+  public void setJCATransaction() {}
+
+  @Override
+  public void setSynchronizationRunnable(TXSynchronizationRunnable sync) {}
+
+  @Override
+  public TXSynchronizationRunnable getSynchronizationRunnable() {
+    return null;
+  }
+
+  @Override
+  public void suspend() {}
+
+  @Override
+  public void resume() {}
+
+  @Override
+  public void recordTXOperation(ServerRegionDataAccess proxy, ServerRegionOperation op, Object key,
+      Object[] arguments) {}
+
+  @Override
+  public int operationCount() {
+    return 0;
+  }
+
+  @Override
+  public void setInProgress(boolean progress) {}
+
+  @Override
+  public void updateProxyServer(InternalDistributedMember proxy) {}
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java
index 07f25c6..ed5f921 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java
@@ -590,6 +590,10 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
    */
   public TXStateProxy getTXState() {
     TXStateProxy tsp = txContext.get();
+    if (tsp == pausedTXState) {
+      // treats paused transaction as no transaction.
+      return null;
+    }
     if (tsp != null && !tsp.isInProgress()) {
       this.txContext.set(null);
       tsp = null;
@@ -608,6 +612,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
   public boolean setInProgress(boolean progress) {
     boolean retVal = false;
     TXStateProxy tsp = txContext.get();
+    assert tsp != pausedTXState;
     if (tsp != null) {
       retVal = tsp.isInProgress();
       tsp.setInProgress(progress);
@@ -669,8 +674,18 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     }
   }
 
-  private final ConcurrentMap<Thread, Boolean> transactionInternalSuspendedByThreads =
-      new ConcurrentHashMap<Thread, Boolean>();
+  private static final TXStateProxy pausedTXState = new PausedTXStateProxyImpl();
+
+  /**
+   * If the current thread is in a transaction then pause will cause it to no longer be in a
+   * transaction. The same thread is expected to unpause/resume the transaction later.
+   *
+   * @return the state of the transaction or null. Pass this value to {@link TXManagerImpl#resume}
+   *         to reactivate the suspended transaction.
+   */
+  public TXStateProxy pauseTransaction() {
+    return internalSuspend(true);
+  }
 
   /**
    * If the current thread is in a transaction then suspend will cause it to no longer be in a
@@ -680,22 +695,63 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
    *         to reactivate the suspended transaction.
    */
   public TXStateProxy internalSuspend() {
+    return internalSuspend(false);
+  }
+
+  /**
+   * If the current thread is in a transaction then suspend will cause it to no longer be in a
+   * transaction.
+   *
+   * @param needToResumeBySameThread whether a suspended transaction needs to be resumed by the same
+   *        thread.
+   * @return the state of the transaction or null. Pass this value to {@link TXManagerImpl#resume}
+   *         to reactivate the suspended transaction.
+   */
+  public TXStateProxy internalSuspend(boolean needToResumeBySameThread) {
     TXStateProxy result = getTXState();
     if (result != null) {
       result.suspend();
-      setTXState(null);
-      transactionInternalSuspendedByThreads.put(Thread.currentThread(), true);
+      if (needToResumeBySameThread) {
+        setTXState(pausedTXState);
+      } else {
+        setTXState(null);
+      }
     }
     return result;
   }
 
   /**
-   * Activates the specified transaction on the calling thread.
-   * 
+   * Activates the specified transaction on the calling thread. Only the same thread that pause the
+   * transaction can unpause it.
+   *
+   * @param tx the transaction to be unpaused.
+   * @throws IllegalStateException if this thread already has an active transaction or this thread
+   *         did not pause the transaction.
+   */
+  public void unpauseTransaction(TXStateProxy tx) {
+    internalResume(tx, true);
+  }
+
+  /**
+   * Activates the specified transaction on the calling thread. Does not require the same thread to
+   * resume it.
+   *
    * @param tx the transaction to activate.
    * @throws IllegalStateException if this thread already has an active transaction
    */
   public void internalResume(TXStateProxy tx) {
+    internalResume(tx, false);
+  }
+
+  /**
+   * Activates the specified transaction on the calling thread.
+   *
+   * @param tx the transaction to activate.
+   * @param needToResumeBySameThread whether a suspended transaction needs to be resumed by the same
+   *        thread.
+   * @throws IllegalStateException if this thread already has an active transaction
+   */
+  public void internalResume(TXStateProxy tx, boolean needToResumeBySameThread) {
     if (tx != null) {
       TransactionId tid = getTransactionId();
       if (tid != null) {
@@ -703,14 +759,21 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
             LocalizedStrings.TXManagerImpl_TRANSACTION_0_ALREADY_IN_PROGRESS
                 .toLocalizedString(tid));
       }
+      if (needToResumeBySameThread) {
+        TXStateProxy result = txContext.get();
+        if (result != pausedTXState) {
+          throw new java.lang.IllegalStateException(
+              "try to unpause a transaction not paused by the same thread");
+        }
+      }
       setTXState(tx);
-      transactionInternalSuspendedByThreads.remove(Thread.currentThread());
+
       tx.resume();
     }
   }
 
-  public boolean isTransactionInternalSuspendedByThread(Thread thread) {
-    return transactionInternalSuspendedByThreads.get(thread) != null;
+  public boolean isTransactionPaused() {
+    return txContext.get() == pausedTXState;
   }
 
   /**
@@ -758,7 +821,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
 
   public int getMyTXUniqueId() {
     TXStateProxy t = txContext.get();
-    if (t != null) {
+    if (t != null && t != pausedTXState) {
       return t.getTxId().getUniqId();
     } else {
       return NOTX;
@@ -1237,7 +1300,8 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     TXStateProxy result = getTXState();
     if (result != null) {
       TransactionId txId = result.getTransactionId();
-      internalSuspend();
+      result.suspend();
+      setTXState(null);
       this.suspendedTXs.put(txId, result);
       // wake up waiting threads
       Queue<Thread> waitingThreads = this.waitMap.get(txId);
@@ -1294,7 +1358,9 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
 
   private void resumeProxy(TXStateProxy txProxy) {
     assert txProxy != null;
-    internalResume(txProxy);
+    assert getTXState() == null;
+    setTXState(txProxy);
+    txProxy.resume();
     SystemTimerTask task = this.expiryTasks.remove(txProxy.getTransactionId());
     if (task != null) {
       if (task.cancel()) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
index 662f7b0..2d109d8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
@@ -1666,11 +1666,11 @@ public class TXState implements TXStateInterface {
       if (!pr.getBucketPrimary(curr.getBucketId()).equals(pr.cache.getMyId())) {
         // to fix bug 47893 suspend the tx before calling nonTXGetEntry
         final TXManagerImpl txmgr = pr.getGemFireCache().getTXMgr();
-        final TXStateProxy tx = txmgr.internalSuspend();
+        final TXStateProxy tx = txmgr.pauseTransaction();
         try {
           return pr.nonTXGetEntry(curr, false, allowTombstones);
         } finally {
-          txmgr.internalResume(tx);
+          txmgr.unpauseTransaction(tx);
         }
       }
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxy.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxy.java
index d392c41..355900e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxy.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxy.java
@@ -64,14 +64,14 @@ public interface TXStateProxy extends TXStateInterface {
   public TXSynchronizationRunnable getSynchronizationRunnable();
 
   /**
-   * Called by {@link TXManagerImpl#internalSuspend()} to perform additional tasks required to
-   * suspend a transaction
+   * Called by {@link TXManagerImpl#internalSuspend(boolean)} to perform additional tasks required
+   * to suspend a transaction
    */
   public void suspend();
 
   /**
-   * Called by {@link TXManagerImpl#internalResume(TXStateProxy)} to perform additional tasks
-   * required to resume a transaction
+   * Called by {@link TXManagerImpl#internalResume(TXStateProxy, boolean)} to perform additional
+   * tasks required to resume a transaction
    */
   public void resume();
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java
index da4f919..d01b8d6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java
@@ -43,6 +43,7 @@ import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
 import org.apache.geode.internal.cache.tx.ClientTXStateStub;
 import org.apache.geode.internal.cache.tx.TransactionalOperation.ServerRegionOperation;
 import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
 
@@ -465,7 +466,7 @@ public class TXStateProxyImpl implements TXStateProxy {
     TXStateProxy txp = null;
     boolean txUnlocked = false;
     if (resetTXState) {
-      txp = getTxMgr().internalSuspend();
+      txp = getTxMgr().pauseTransaction();
     } else {
       if (getLock().isHeldByCurrentThread()) {
         txUnlocked = true; // bug #42945 - hang trying to compute size for PR
@@ -479,7 +480,7 @@ public class TXStateProxyImpl implements TXStateProxy {
       return getRealDeal(null, localRegion).entryCount(localRegion);
     } finally {
       if (resetTXState) {
-        getTxMgr().internalResume(txp);
+        getTxMgr().unpauseTransaction(txp);
       } else if (txUnlocked) {
         getLock().lock();
       }
@@ -508,29 +509,15 @@ public class TXStateProxyImpl implements TXStateProxy {
     return getRealDeal(null, currRgn).getAdditionalKeysForIterator(currRgn);
   }
 
-  /**
-   * A set operation now can bootstrap a transaction now. User need to specifically disable this by
-   * setting this system property to true to get the old behavior.
-   *
-   * @Since Geode 1.3.0
-   */
-  public static final String PREVENT_SET_OP_BOOTSTRAP_TRANSACTION =
-      "preventSetOpBootstrapTransaction";
-
-  protected static boolean isSetOpBootstrapTransactionDisabled() {
-    return Boolean
-        .getBoolean(DistributionConfig.GEMFIRE_PREFIX + PREVENT_SET_OP_BOOTSTRAP_TRANSACTION)
-        || Boolean.getBoolean("geode." + PREVENT_SET_OP_BOOTSTRAP_TRANSACTION);
-  }
-
-  protected final boolean preventSetOpBootstrapTransaction = isSetOpBootstrapTransactionDisabled();
+  protected final boolean preventSetOpBootstrapTransaction =
+      SystemPropertyHelper.preventSetOpBootstrapTransaction();
 
   public Object getEntryForIterator(KeyInfo key, LocalRegion currRgn, boolean rememberReads,
       boolean allowTombstones) {
     boolean resetTxState = isTransactionInternalSuspendNeeded(currRgn);
     TXStateProxy txp = null;
     if (resetTxState) {
-      txp = getTxMgr().internalSuspend();
+      txp = getTxMgr().pauseTransaction();
     }
     try {
       if (resetTxState) {
@@ -540,7 +527,7 @@ public class TXStateProxyImpl implements TXStateProxy {
           allowTombstones);
     } finally {
       if (resetTxState) {
-        getTxMgr().internalResume(txp);
+        getTxMgr().unpauseTransaction(txp);
       }
     }
   }
@@ -556,7 +543,7 @@ public class TXStateProxyImpl implements TXStateProxy {
     boolean resetTxState = isTransactionInternalSuspendNeeded(currRgn);
     TXStateProxy txp = null;
     if (resetTxState) {
-      txp = getTxMgr().internalSuspend();
+      txp = getTxMgr().pauseTransaction();
     }
     try {
       if (resetTxState) {
@@ -567,7 +554,7 @@ public class TXStateProxyImpl implements TXStateProxy {
           allowTombstones);
     } finally {
       if (resetTxState) {
-        getTxMgr().internalResume(txp);
+        getTxMgr().unpauseTransaction(txp);
       }
     }
   }
@@ -663,7 +650,7 @@ public class TXStateProxyImpl implements TXStateProxy {
     boolean resetTxState = isTransactionInternalSuspendNeeded(localRegion);
     TXStateProxy txp = null;
     if (resetTxState) {
-      txp = getTxMgr().internalSuspend();
+      txp = getTxMgr().pauseTransaction();
     }
     try {
       if (resetTxState) {
@@ -672,7 +659,7 @@ public class TXStateProxyImpl implements TXStateProxy {
       return getRealDeal(null, localRegion).getBucketKeys(localRegion, bucketId, false);
     } finally {
       if (resetTxState) {
-        getTxMgr().internalResume(txp);
+        getTxMgr().unpauseTransaction(txp);
       }
     }
   }
@@ -705,7 +692,7 @@ public class TXStateProxyImpl implements TXStateProxy {
       boolean resetTxState = isTransactionInternalSuspendNeeded(currRegion);
       TXStateProxy txp = null;
       if (resetTxState) {
-        txp = getTxMgr().internalSuspend();
+        txp = getTxMgr().pauseTransaction();
       }
       try {
         if (resetTxState) {
@@ -714,7 +701,7 @@ public class TXStateProxyImpl implements TXStateProxy {
         return getRealDeal(null, currRegion).getRegionKeysForIteration(currRegion);
       } finally {
         if (resetTxState) {
-          getTxMgr().internalResume(txp);
+          getTxMgr().unpauseTransaction(txp);
         }
       }
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java b/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java
new file mode 100644
index 0000000..784da68
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java
@@ -0,0 +1,52 @@
+/*
+ * 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 org.apache.geode.internal.lang;
+
+/**
+ * The SystemPropertyHelper class is an helper class for accessing system properties used in geode.
+ * The method name to get the system property should be the same as the system property name.
+ * 
+ * @since Geode 1.3.0
+ */
+
+public class SystemPropertyHelper {
+  private static final String GEODE_PREFIX = "geode.";
+  private static final String GEMFIRE_PREFIX = "gemfire.";
+
+  /**
+   * This method will try to look up "geode." and "gemfire." versions of the system property. It
+   * will check and prefer "geode." setting first, then try to check "gemfire." setting.
+   * 
+   * @param name system property name set in Geode
+   * @return a boolean value of the system property
+   */
+  private static boolean getProductBooleanProperty(String name) {
+    String property = System.getProperty(GEODE_PREFIX + name);
+    if (property != null) {
+      return Boolean.getBoolean(GEODE_PREFIX + name);
+    }
+    return Boolean.getBoolean(GEMFIRE_PREFIX + name);
+  }
+
+  /**
+   * A set operation can bootstrap a transaction now. User need to specifically disable this by
+   * setting this system property to true to get the old behavior.
+   *
+   * @since Geode 1.3.0
+   */
+  public static boolean preventSetOpBootstrapTransaction() {
+    return getProductBooleanProperty("preventSetOpBootstrapTransaction");
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/pdx/internal/PeerTypeRegistration.java b/geode-core/src/main/java/org/apache/geode/pdx/internal/PeerTypeRegistration.java
index ea9bf3e..ad56b78 100644
--- a/geode-core/src/main/java/org/apache/geode/pdx/internal/PeerTypeRegistration.java
+++ b/geode-core/src/main/java/org/apache/geode/pdx/internal/PeerTypeRegistration.java
@@ -618,13 +618,13 @@ public class PeerTypeRegistration implements TypeRegistration {
   private TXStateProxy suspendTX() {
     InternalCache cache = (InternalCache) getIdToType().getRegionService();
     TXManagerImpl txManager = (TXManagerImpl) cache.getCacheTransactionManager();
-    return txManager.internalSuspend();
+    return txManager.pauseTransaction();
   }
 
   private void resumeTX(TXStateProxy state) {
     if (state != null) {
       TXManagerImpl txManager = state.getTxMgr();
-      txManager.internalResume(state);
+      txManager.unpauseTransaction(state);
     }
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/TX2JUnitTest.java b/geode-core/src/test/java/org/apache/geode/TX2JUnitTest.java
index 10e1c91..8139652 100644
--- a/geode-core/src/test/java/org/apache/geode/TX2JUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/TX2JUnitTest.java
@@ -39,8 +39,13 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 @Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class TX2JUnitTest {
 
   private static final Logger logger = LogService.getLogger();
@@ -68,33 +73,15 @@ public class TX2JUnitTest {
   }
 
   @Test
-  public void testValuesCallStartsTx() {
+  @Parameters
+  public void testRegionSetOpWithTx(SetOp op, boolean preventSetOpToStartTx) {
     verifySetOp(SetOp.VALUES, false);
   }
 
-  @Test
-  public void testKeySetCallStartsTx() {
-    verifySetOp(SetOp.KEYSET, false);
-  }
-
-  @Test
-  public void testEntrySetCallStartsTx() {
-    verifySetOp(SetOp.ENTRYSET, false);
-  }
-
-  @Test
-  public void testValuesCallNotStartTxIfDisabled() {
-    verifySetOp(SetOp.VALUES, true);
-  }
-
-  @Test
-  public void testKeySetCallNotStartTxIfDisabled() {
-    verifySetOp(SetOp.KEYSET, true);
-  }
-
-  @Test
-  public void testEntrySetCallNotStartTxIfDisabled() {
-    verifySetOp(SetOp.ENTRYSET, true);
+  private Object[] parametersForTestRegionSetOpWithTx() {
+    return new Object[] {new Object[] {SetOp.VALUES, false}, new Object[] {SetOp.VALUES, true},
+        new Object[] {SetOp.KEYSET, false}, new Object[] {SetOp.KEYSET, true},
+        new Object[] {SetOp.ENTRYSET, false}, new Object[] {SetOp.ENTRYSET, true},};
   }
 
   private void verifySetOp(SetOp op, boolean preventSetOpToStartTx) {
diff --git a/geode-core/src/test/java/org/apache/geode/TXExpiryJUnitTest.java b/geode-core/src/test/java/org/apache/geode/TXExpiryJUnitTest.java
index 077d3e6..f5903ed 100644
--- a/geode-core/src/test/java/org/apache/geode/TXExpiryJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/TXExpiryJUnitTest.java
@@ -198,9 +198,9 @@ public class TXExpiryJUnitTest {
         } else {
           checkVal = "conflictVal";
           final TXManagerImpl txMgrImpl = (TXManagerImpl) this.txMgr;
-          TXStateProxy tx = txMgrImpl.internalSuspend();
+          TXStateProxy tx = txMgrImpl.pauseTransaction();
           exprReg.put("key0", checkVal);
-          txMgrImpl.internalResume(tx);
+          txMgrImpl.unpauseTransaction(tx);
           try {
             this.txMgr.commit();
             fail("Expected CommitConflictException!");
diff --git a/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java b/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
index cf1281d..1c060a7 100644
--- a/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
@@ -4332,11 +4332,11 @@ public class TXJUnitTest {
       this.txMgr.begin();
       reg1.create("key1", "txValue");
       assertEquals("txValue", reg1.getEntry("key1").getValue());
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
       assertTrue(!reg1.containsKey("key1"));
       assertEquals("LV 4", reg1.get("key1"));
       assertTrue(reg1.containsKey("key1"));
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       assertEquals("txValue", reg1.getEntry("key1").getValue());
       assertEquals("txValue", reg1.get("key1"));
       try {
@@ -4377,12 +4377,12 @@ public class TXJUnitTest {
       this.txMgr.begin();
       assertEquals("LV 8", reg1.get("key1"));
       assertEquals("LV 8", reg1.getEntry("key1").getValue());
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
       assertTrue(!reg1.containsKey("key1"));
       reg1.create("key1", "txValue");
       assertTrue(reg1.containsKey("key1"));
       assertEquals("txValue", reg1.get("key1"));
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       assertEquals("LV 8", reg1.getEntry("key1").getValue());
       try {
         this.txMgr.commit(); // should conflict
@@ -4400,7 +4400,7 @@ public class TXJUnitTest {
       TXStateProxy tx;
       this.txMgr.begin();
       reg1.create("key1", "txValue");
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
       assertTrue(!reg1.containsKey("key1"));
       // new transaction, load(create) + put
       this.txMgr.begin();
@@ -4413,7 +4413,7 @@ public class TXJUnitTest {
       assertTrue(reg1.containsKey("key1"));
       assertEquals("txValue2", reg1.get("key1"));
       assertEquals("txValue2", reg1.getEntry("key1").getValue());
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       assertEquals("txValue", reg1.getEntry("key1").getValue());
       assertEquals("txValue", reg1.get("key1"));
       try {
@@ -4755,10 +4755,10 @@ public class TXJUnitTest {
     } catch (EntryExistsException ok) {
     }
     // begin other tx simulation
-    TXStateProxy tx = txMgrImpl.internalSuspend();
+    TXStateProxy tx = txMgrImpl.pauseTransaction();
     this.region.put("stats1", "stats success1");
     this.region.put("stats2", "stats success2");
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     // end other tx simulation
     pause(SLEEP_MS);
     try {
@@ -4908,26 +4908,26 @@ public class TXJUnitTest {
   public void testSuspendResume() {
     TXManagerImpl txMgrImpl = (TXManagerImpl) this.txMgr;
     assertTrue(!this.txMgr.exists());
-    assertEquals(null, txMgrImpl.internalSuspend());
+    assertEquals(null, txMgrImpl.pauseTransaction());
     TXStateProxy txProxy = null;
-    txMgrImpl.internalResume(txProxy);
+    txMgrImpl.unpauseTransaction(txProxy);
     assertTrue(!this.txMgr.exists());
 
     this.txMgr.begin();
     TransactionId origId = this.txMgr.getTransactionId();
     assertTrue(this.txMgr.exists());
     {
-      TXStateProxy tx = txMgrImpl.internalSuspend();
+      TXStateProxy tx = txMgrImpl.pauseTransaction();
       assertTrue(!this.txMgr.exists());
       this.txMgr.begin();
       try {
-        txMgrImpl.internalResume(tx);
+        txMgrImpl.unpauseTransaction(tx);
         fail("expected IllegalStateException");
       } catch (IllegalStateException expected) {
       }
       this.txMgr.rollback();
       assertTrue(!this.txMgr.exists());
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
     }
     assertTrue(this.txMgr.exists());
     assertEquals(origId, this.txMgr.getTransactionId());
@@ -5040,9 +5040,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     assertEquals("value1", this.region.get("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
 
     assertEquals("value1", this.region.get("key1"));
     txMgrImpl.commit();
@@ -5051,9 +5051,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     assertEquals("value1", this.region.get("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals("value1", this.region.get("key1"));
     this.region.put("key1", "value3");
     assertEquals("value3", this.region.get("key1"));
@@ -5067,9 +5067,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     this.region.getEntry("key1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
 
     assertEquals("value1", this.region.get("key1"));
     txMgrImpl.commit();
@@ -5078,9 +5078,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     this.region.getEntry("key1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     this.region.put("key1", "value3");
     try {
       txMgrImpl.commit();
@@ -5093,9 +5093,9 @@ public class TXJUnitTest {
     txMgrImpl.begin();
     this.region.get("key1"); // bootstrap the tx, entrySet does not
     this.region.entrySet(false).iterator().next();
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
 
     assertEquals("value1", this.region.get("key1"));
     txMgrImpl.commit();
@@ -5105,9 +5105,9 @@ public class TXJUnitTest {
     txMgrImpl.begin();
     this.region.get("key1"); // bootstrap the tx, entrySet does not
     this.region.entrySet(false).iterator().next();
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals("value1", this.region.get("key1"));
     this.region.put("key1", "value3");
     try {
@@ -5120,17 +5120,17 @@ public class TXJUnitTest {
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     assertEquals(true, this.region.containsKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(true, this.region.containsKey("key1"));
     txMgrImpl.commit();
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     assertEquals(true, this.region.containsKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(true, this.region.containsKey("key1"));
     this.region.put("key1", "value3");
     assertEquals(true, this.region.containsKey("key1"));
@@ -5143,17 +5143,17 @@ public class TXJUnitTest {
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     assertEquals(true, this.region.containsValueForKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(true, this.region.containsValueForKey("key1"));
     txMgrImpl.commit();
     this.region.put("key1", "value1"); // non-tx
     txMgrImpl.begin();
     assertEquals(true, this.region.containsValueForKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(true, this.region.containsValueForKey("key1"));
     this.region.put("key1", "value3");
     assertEquals(true, this.region.containsValueForKey("key1"));
@@ -5169,9 +5169,9 @@ public class TXJUnitTest {
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(null, this.region.get("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
 
     assertEquals(null, this.region.get("key1"));
     txMgrImpl.commit();
@@ -5180,9 +5180,9 @@ public class TXJUnitTest {
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(null, this.region.get("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(null, this.region.get("key1"));
     this.region.put("key1", "value3");
     assertEquals("value3", this.region.get("key1"));
@@ -5196,9 +5196,9 @@ public class TXJUnitTest {
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(null, this.region.getEntry("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
 
     assertEquals(null, this.region.getEntry("key1"));
     txMgrImpl.commit();
@@ -5207,9 +5207,9 @@ public class TXJUnitTest {
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(null, this.region.getEntry("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(null, this.region.getEntry("key1"));
     this.region.put("key1", "value3");
     try {
@@ -5222,17 +5222,17 @@ public class TXJUnitTest {
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(false, this.region.containsKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(false, this.region.containsKey("key1"));
     txMgrImpl.commit();
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(false, this.region.containsKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(false, this.region.containsKey("key1"));
     this.region.put("key1", "value3");
     assertEquals(true, this.region.containsKey("key1"));
@@ -5246,17 +5246,17 @@ public class TXJUnitTest {
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(false, this.region.containsValueForKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(false, this.region.containsValueForKey("key1"));
     txMgrImpl.commit();
     this.region.remove("key1"); // non-tx
     txMgrImpl.begin();
     assertEquals(false, this.region.containsValueForKey("key1"));
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(false, this.region.containsValueForKey("key1"));
     this.region.put("key1", "value3");
     assertEquals(true, this.region.containsValueForKey("key1"));
@@ -5272,9 +5272,9 @@ public class TXJUnitTest {
     txMgrImpl.begin();
     this.region.get("key1");
     this.region.localInvalidate("key1"); // should be a noop since it is already invalid
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     txMgrImpl.commit();
     assertEquals(false, this.region.containsKey("key1"));
 
@@ -5283,9 +5283,9 @@ public class TXJUnitTest {
     this.region.create("key1", null); // non-tx
     txMgrImpl.begin();
     this.region.localInvalidate("key1"); // should be a noop since it is already invalid
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(true, this.region.containsKey("key1"));
     assertEquals(false, this.region.containsValueForKey("key1"));
     txMgrImpl.commit();
@@ -5299,9 +5299,9 @@ public class TXJUnitTest {
       fail("expected EntryNotFoundException");
     } catch (EntryNotFoundException expected) {
     }
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.create("key1", "value1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(false, this.region.containsKey("key1"));
     txMgrImpl.commit();
     assertEquals(true, this.region.containsKey("key1"));
@@ -5315,9 +5315,9 @@ public class TXJUnitTest {
       fail("expected EntryExistsException");
     } catch (EntryExistsException expected) {
     }
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.remove("key1"); // non-tx
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     assertEquals(true, this.region.containsKey("key1"));
     txMgrImpl.commit();
     assertEquals(false, this.region.containsKey("key1"));
@@ -5337,9 +5337,9 @@ public class TXJUnitTest {
     // now try a put with a conflict and make sure it is detected
     txMgrImpl.begin();
     this.region.put("key1", "value1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5360,9 +5360,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value0");
     txMgrImpl.begin();
     this.region.put("key1", "value1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5381,9 +5381,9 @@ public class TXJUnitTest {
     // now try a create with a conflict and make sure it is detected
     txMgrImpl.begin();
     this.region.create("key1", "value1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5404,9 +5404,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value0");
     txMgrImpl.begin();
     this.region.localInvalidate("key1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5427,9 +5427,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value0");
     txMgrImpl.begin();
     this.region.invalidate("key1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5441,9 +5441,9 @@ public class TXJUnitTest {
     // check C + DD is a NOOP that still gets conflict if non-tx entry created */
     this.txMgr.begin();
     this.region.create("newKey", "valueTX");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.create("newKey", "valueNONTX");
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     this.region.destroy("newKey");
     assertTrue(!this.region.containsKey("key1"));
     try {
@@ -5457,9 +5457,9 @@ public class TXJUnitTest {
     // check C + LD is a NOOP that still gets conflict if non-tx entry created */
     this.txMgr.begin();
     this.region.create("newKey", "valueTX");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.create("newKey", "valueNONTX");
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     this.region.localDestroy("newKey");
     assertTrue(!this.region.containsKey("key1"));
     try {
@@ -5481,9 +5481,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value0");
     txMgrImpl.begin();
     this.region.localDestroy("key1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5503,9 +5503,9 @@ public class TXJUnitTest {
     this.region.put("key1", "value0");
     txMgrImpl.begin();
     this.region.destroy("key1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.put("key1", "value2"); // do a non-tx put to force conflict
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     try {
       txMgrImpl.commit();
       fail("expected CommitConflictException");
@@ -5522,9 +5522,9 @@ public class TXJUnitTest {
     this.region.localInvalidate("key1");
     txMgrImpl.begin();
     this.region.put("key1", "txVal1");
-    tx = txMgrImpl.internalSuspend();
+    tx = txMgrImpl.pauseTransaction();
     this.region.invalidate("key1");
-    txMgrImpl.internalResume(tx);
+    txMgrImpl.unpauseTransaction(tx);
     txMgrImpl.commit();
     assertEquals("txVal1", this.region.getEntry("key1").getValue());
     this.region.destroy("key1");
@@ -5532,9 +5532,9 @@ public class TXJUnitTest {
     // now try a put and a region destroy.
     txMgrImpl.begin();
     this.region.create("key1", "value1");
-    TXStateProxy tis = txMgrImpl.internalSuspend();
+    TXStateProxy tis = txMgrImpl.pauseTransaction();
     this.region.localDestroyRegion(); // non-tx
-    txMgrImpl.internalResume(tis);
+    txMgrImpl.unpauseTransaction(tis);
 
     try {
       txMgrImpl.commit();
@@ -5800,7 +5800,7 @@ public class TXJUnitTest {
         lruRegion.put("key" + i, new Long(i));
       }
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
 
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
       for (int i = 0; i < numToPut; ++i) {
@@ -5809,7 +5809,7 @@ public class TXJUnitTest {
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
       assertNull(lruRegion.get("non-tx key0"));
 
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       this.txMgr.commit();
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_LONG);
     }
@@ -5841,7 +5841,7 @@ public class TXJUnitTest {
         lruRegion.get("key" + i, new Integer(i));
       }
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_STRING);
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
 
       assertEquals(lruSize, lruRegion.entrySet(false).size());
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_NULL);
@@ -5851,7 +5851,7 @@ public class TXJUnitTest {
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_NULL);
       assertNull(lruRegion.getEntry("non-tx key0"));
 
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       this.txMgr.commit();
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_STRING);
       Iterator it = lruRegion.keySet().iterator();
@@ -5884,7 +5884,7 @@ public class TXJUnitTest {
         lruRegion.put("key" + i, new Long(i));
       }
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
-      tx1 = txMgrImpl.internalSuspend();
+      tx1 = txMgrImpl.pauseTransaction();
 
       this.txMgr.begin();
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
@@ -5893,7 +5893,7 @@ public class TXJUnitTest {
         lruRegion.put("key" + i, new Double(i));
       }
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_DOUBLE);
-      tx2 = txMgrImpl.internalSuspend();
+      tx2 = txMgrImpl.pauseTransaction();
 
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
 
@@ -5906,13 +5906,13 @@ public class TXJUnitTest {
       assertNull(lruRegion.get("non-tx key0"));
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
 
-      txMgrImpl.internalResume(tx1);
+      txMgrImpl.unpauseTransaction(tx1);
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
       // Check to make sure no conflict was caused by non-TX put evictions
       // This should remove all references for each committed entry
       this.txMgr.commit();
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_LONG);
-      txMgrImpl.internalResume(tx2);
+      txMgrImpl.unpauseTransaction(tx2);
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_DOUBLE);
       this.txMgr.rollback();
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_LONG);
@@ -5946,7 +5946,7 @@ public class TXJUnitTest {
         lruRegion.put("key" + i, new Long(i));
       }
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
 
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
       // Force the Non-Tx "put" to remove each attempt since region is full
@@ -5957,7 +5957,7 @@ public class TXJUnitTest {
       assertNull(lruRegion.get("non-tx key0"));
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
 
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
       // This should remove all references for each committed entry
       this.txMgr.rollback();
@@ -5992,12 +5992,12 @@ public class TXJUnitTest {
         lruRegion.put("key" + i, new Long(i));
       }
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
-      tx = txMgrImpl.internalSuspend();
+      tx = txMgrImpl.pauseTransaction();
 
       // Cause a conflict
       lruRegion.put("key" + (numToPut - 1), new Integer(numToPut - 1));
 
-      txMgrImpl.internalResume(tx);
+      txMgrImpl.unpauseTransaction(tx);
       assertLRUEntries(lruRegion.entrySet(false), numToPut, "key", LRUENTRY_LONG);
       // This should remove all references for each committed entry
       try {
@@ -6102,12 +6102,12 @@ public class TXJUnitTest {
       this.region.put("syncKey3", "syncVal3");
       assertEquals("syncVal3", this.region.getEntry("syncKey3").getValue());
 
-      TXStateProxy gfTx = gfTxMgrImpl.internalSuspend();
+      TXStateProxy gfTx = gfTxMgrImpl.pauseTransaction();
       javax.transaction.Transaction jtaTx = jtaTxMgr.suspend();
       assertNull(jtaTxMgr.getTransaction());
       this.region.put("syncKey3", "syncVal4");
       assertEquals("syncVal4", this.region.getEntry("syncKey3").getValue());
-      gfTxMgrImpl.internalResume(gfTx);
+      gfTxMgrImpl.unpauseTransaction(gfTx);
       try {
         jtaTxMgr.resume(jtaTx);
       } catch (Exception failure) {
@@ -6346,7 +6346,7 @@ public class TXJUnitTest {
     assertNotNull(this.txMgr.getTransactionId());
     {
       TXManagerImpl gfTxMgrImpl = (TXManagerImpl) this.txMgr;
-      TXStateProxy gfTx = gfTxMgrImpl.internalSuspend();
+      TXStateProxy gfTx = gfTxMgrImpl.pauseTransaction();
 
       javax.transaction.TransactionManager jtaTxMgr = this.cache.getJTATransactionManager();
       javax.transaction.Transaction jtaTx = jtaTxMgr.suspend();
@@ -6359,7 +6359,7 @@ public class TXJUnitTest {
       } catch (Exception failure) {
         fail("JTA resume failed");
       }
-      gfTxMgrImpl.internalResume(gfTx);
+      gfTxMgrImpl.unpauseTransaction(gfTx);
     }
     assertEquals("enlistVal", this.region.get("enlistKey"));
     try {
@@ -6804,10 +6804,10 @@ public class TXJUnitTest {
     ctm.begin();
     pr.putAll(map);
     r.putAll(map);
-    TXStateProxy tx = ctm.internalSuspend();
+    TXStateProxy tx = ctm.pauseTransaction();
     assertTrue(!pr.containsKey("stuff"));
     assertTrue(!r.containsKey("stuff"));
-    ctm.internalResume(tx);
+    ctm.unpauseTransaction(tx);
     ctm.commit();
     assertTrue(pr.containsKey("stuff"));
     assertTrue(r.containsKey("stuff"));
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/MultiVMRegionTestCase.java b/geode-core/src/test/java/org/apache/geode/cache30/MultiVMRegionTestCase.java
index e7e6ee1..d408721 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/MultiVMRegionTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/MultiVMRegionTestCase.java
@@ -6343,10 +6343,10 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
       vm0.invoke(create);
 
       {
-        TXStateProxy tx = ((TXManagerImpl) txMgr).internalSuspend();
+        TXStateProxy tx = ((TXManagerImpl) txMgr).pauseTransaction();
         assertTrue(rgn.containsKey("key"));
         assertEquals("LV 1", rgn.getEntry("key").getValue());
-        ((TXManagerImpl) txMgr).internalResume(tx);
+        ((TXManagerImpl) txMgr).unpauseTransaction(tx);
       }
       // make sure transactional view is still correct
       assertEquals("txValue", rgn.getEntry("key").getValue());
@@ -6391,11 +6391,11 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
       rgn.create("key3", "txValue3");
 
       {
-        TXStateProxy tx = ((TXManagerImpl) txMgr).internalSuspend();
+        TXStateProxy tx = ((TXManagerImpl) txMgr).pauseTransaction();
         // do a get outside of the transaction to force a net load
         Object v3 = rgn.get("key3");
         assertEquals("LV 3", v3);
-        ((TXManagerImpl) txMgr).internalResume(tx);
+        ((TXManagerImpl) txMgr).unpauseTransaction(tx);
       }
       // make sure transactional view is still correct
       assertEquals("txValue3", rgn.getEntry("key3").getValue());
@@ -6449,11 +6449,11 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
       rgn.put("key", "new txValue");
 
       {
-        TXStateProxy tx = ((TXManagerImpl) txMgr).internalSuspend();
+        TXStateProxy tx = ((TXManagerImpl) txMgr).pauseTransaction();
         // do a get outside of the transaction to force a netsearch
         assertEquals("txValue", rgn.get("key")); // does a netsearch
         assertEquals("txValue", rgn.getEntry("key").getValue());
-        ((TXManagerImpl) txMgr).internalResume(tx);
+        ((TXManagerImpl) txMgr).unpauseTransaction(tx);
       }
       // make sure transactional view is still correct
       assertEquals("new txValue", rgn.getEntry("key").getValue());
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/ClientServerTransactionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/ClientServerTransactionDUnitTest.java
index c7ae750..29b4674 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/ClientServerTransactionDUnitTest.java
@@ -725,7 +725,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
         }
         org.apache.geode.test.dunit.LogWriterUtils.getLogWriter().info("suspending transaction");
         if (!useJTA) {
-          TXStateProxy tx = mgr.internalSuspend();
+          TXStateProxy tx = mgr.pauseTransaction();
           if (prePopulateData) {
             for (int i = 0; i < 5; i++) {
               CustId custId = new CustId(i);
@@ -739,7 +739,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
             assertNull(pr.get(new CustId(i)));
           }
           org.apache.geode.test.dunit.LogWriterUtils.getLogWriter().info("resuming transaction");
-          mgr.internalResume(tx);
+          mgr.unpauseTransaction(tx);
         }
         assertEquals("r sized should be " + MAX_ENTRIES + " but it is:" + r.size(), MAX_ENTRIES,
             r.size());
@@ -838,9 +838,9 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     client.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         TXManagerImpl mgr = getGemfireCache().getTxManager();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(tx);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         if (commit) {
           mgr.commit();
         } else {
@@ -1620,10 +1620,10 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
         mgr.begin();
         pr.put(custId, new Customer("name10", "address10"));
         r.put(10, "value10");
-        TXStateProxy txState = mgr.internalSuspend();
+        TXStateProxy txState = mgr.pauseTransaction();
         assertNull(pr.get(custId));
         assertNull(r.get(10));
-        mgr.internalResume(txState);
+        mgr.unpauseTransaction(txState);
         mgr.commit();
         return null;
       }
@@ -2369,12 +2369,12 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
         assertEquals(new Customer("name0", "address0"), pr.get(new CustId(0)));
         assertEquals(new Customer("name10", "address10"), pr.get(new CustId(10)));
         assertEquals(new Customer("name10", "address10"), r.get(new CustId(10)));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertEquals(new Customer("oldname0", "oldaddress0"), pr.get(new CustId(0)));
         assertEquals(new Customer("oldname1", "oldaddress1"), pr.get(new CustId(1)));
         assertNull(pr.get(new CustId(10)));
         assertNull(r.get(new CustId(10)));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         assertEquals(new Customer("name0", "address0"), pr.get(new CustId(0)));
         assertEquals(new Customer("name1", "address1"), pr.get(new CustId(1)));
@@ -2435,13 +2435,13 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
         mgr.begin();
         pr.put(custId, new Customer("name10", "address10"));
         r.put(10, "value10");
-        final TXStateProxy txState = mgr.internalSuspend();
+        final TXStateProxy txState = mgr.pauseTransaction();
         assertNull(pr.get(custId));
         assertNull(r.get(10));
         final CountDownLatch latch = new CountDownLatch(1);
         Thread t = new Thread(new Runnable() {
           public void run() {
-            mgr.internalResume(txState);
+            mgr.unpauseTransaction(txState);
             mgr.commit();
             latch.countDown();
           }
@@ -2916,7 +2916,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
             Map<CustId, Customer> m = new HashMap<CustId, Customer>();
             m.put(new CustId(2), new Customer("name2", "address2"));
             r.putAll(m);
-            TXStateProxyImpl tx = (TXStateProxyImpl) mgr.internalSuspend();
+            TXStateProxyImpl tx = (TXStateProxyImpl) mgr.pauseTransaction();
             ClientTXStateStub txStub = (ClientTXStateStub) tx.getRealDeal(null, null);
             txStub.setAfterLocalLocks(new Runnable() {
               public void run() {
@@ -2928,7 +2928,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
                 }
               }
             });
-            mgr.internalResume(tx);
+            mgr.unpauseTransaction(tx);
             mgr.commit();
           }
         });
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/RemoteTransactionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/RemoteTransactionDUnitTest.java
index 3c4908c..84d3e9d 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/RemoteTransactionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/RemoteTransactionDUnitTest.java
@@ -553,9 +553,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
     accessor.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         TXManagerImpl mgr = getGemfireCache().getTxManager();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(tx);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         if (commit) {
           mgr.commit();
         } else {
@@ -660,9 +660,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         Customer s = (Customer) cust.get(new CustId(8));
         assertEquals(new Customer("sup dawg", "add"), s);
         assertTrue(cust.containsKey(new CustId(8)));
-        TXStateProxy tx = ((TXManagerImpl) mgr).internalSuspend();
+        TXStateProxy tx = ((TXManagerImpl) mgr).pauseTransaction();
         assertFalse(cust.containsKey(new CustId(8)));
-        ((TXManagerImpl) mgr).internalResume(tx);
+        ((TXManagerImpl) mgr).unpauseTransaction(tx);
         mgr.commit();
         Customer s2 = (Customer) cust.get(new CustId(8));
         Customer ex = new Customer("sup dawg", "add");
@@ -932,10 +932,10 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         assertNull(oldOrder);
         assertNotNull(cust.get(newCustId));
         assertNotNull(rr.get(newCustId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNull(cust.get(newCustId));
         assertNull(rr.get(newCustId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         cust.put(oldCustId, new Customer("foo", "bar"));
         rr.put(oldCustId, new Customer("foo", "bar"));
         return mgr.getTransactionId();
@@ -978,9 +978,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         mgr.begin();
         CustId conflictCust = new CustId(11);
         cust.putIfAbsent(conflictCust, new Customer("name11", "address11"));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         cust.put(conflictCust, new Customer("foo", "bar"));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         try {
           mgr.commit();
           fail("expected exception not thrown");
@@ -1017,10 +1017,10 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         assertTrue(cust.remove(custId, customer));
         assertFalse(ref.remove(custId, fakeCust));
         assertTrue(ref.remove(custId, customer));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(cust.get(custId));
         assertNotNull(ref.get(custId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return mgr.getTransactionId();
       }
     });
@@ -1058,9 +1058,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         Customer customer = new Customer("customer2", "address2");
         getGemfireCache().getLoggerI18n().fine("SWAP:removeConflict");
         assertTrue(cust.remove(conflictCust, customer));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         cust.put(conflictCust, new Customer("foo", "bar"));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         try {
           mgr.commit();
           fail("expected exception not thrown");
@@ -1093,10 +1093,10 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         Customer fakeCust = new Customer("foo2", "bar2");
         cust.removeAll(Arrays.asList(custId1, custId2, custId20));
         ref.removeAll(Arrays.asList(custId1, custId2, custId20));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(cust.get(custId1));
         assertNotNull(ref.get(custId2));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return mgr.getTransactionId();
       }
     });
@@ -1136,12 +1136,12 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         CustId custId4 = new CustId(4);
         getGemfireCache().getLoggerI18n().fine("SWAP:removeConflict");
         cust.removeAll(Arrays.asList(custId3, custId20, custId4));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         // cust.put(custId3, new Customer("foo", "bar"));
         cust.put(custId20, new Customer("foo", "bar"));
         assertNotNull(cust.get(custId20));
         cust.put(custId4, new Customer("foo", "bar"));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         try {
           mgr.commit();
           fail("expected exception not thrown");
@@ -1156,9 +1156,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         mgr.begin();
         getGemfireCache().getLoggerI18n().fine("SWAP:removeConflict");
         cust.removeAll(Arrays.asList(custId2, custId3));
-        tx = mgr.internalSuspend();
+        tx = mgr.pauseTransaction();
         cust.put(custId2, new Customer("foo", "bar"));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         assertNotNull(cust.get(custId2));
         assertNull(cust.get(custId3));
@@ -1283,10 +1283,10 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         assertTrue(cust.replace(custId, customer, updatedCust));
         assertFalse(ref.replace(custId, fakeCust, updatedCust));
         assertTrue(ref.replace(custId, customer, updatedCust));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertEquals(cust.get(custId), customer);
         assertEquals(ref.get(custId), customer);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return mgr.getTransactionId();
       }
     });
@@ -1324,9 +1324,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         Customer customer = new Customer("customer2", "address2");
         getGemfireCache().getLoggerI18n().fine("SWAP:removeConflict");
         assertTrue(cust.replace(conflictCust, customer, new Customer("conflict", "conflict")));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         cust.put(conflictCust, new Customer("foo", "bar"));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         try {
           mgr.commit();
           fail("expected exception not thrown");
@@ -2205,11 +2205,11 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         assertEquals(6, i);
         assertEquals(6, rr.keySet().size());
         assertNotNull(rr.get(custId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertEquals(getCustIdSet(5), rr.keySet());
         assertEquals(5, rr.keySet().size());
         assertNull(rr.get(custId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
@@ -2255,11 +2255,11 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         assertEquals(6, i);
         assertEquals(6, rr.values().size());
         assertNotNull(rr.get(custId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertEquals(getCustomerSet(5), rr.values());
         assertEquals(5, rr.values().size());
         assertNull(rr.get(custId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
@@ -2305,11 +2305,11 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         assertEquals(6, i);
         assertEquals(6, rr.entrySet().size());
         assertNotNull(rr.get(custId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         // assertIndexDetailsEquals(getCustIdSet(5), rr.entrySet());
         assertEquals(5, rr.entrySet().size());
         assertNull(rr.get(custId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
@@ -2522,10 +2522,10 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
             FunctionService.onMember(owner).execute(TXFunction.id).getResult();
             break;
         }
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         GemFireCacheImpl.getInstance().getLogger().warning("TX SUSPENDO:" + tx);
         assertNull(custRegion.get(expectedCustId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return null;
       }
     });
@@ -2568,9 +2568,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
             FunctionService.onMember(owner).execute(TXFunction.id).getResult();
             break;
         }
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         custRegion.put(expectedCustId, new Customer("Cust6", "updated6"));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         try {
           mgr.commit();
           fail("expected commit conflict not thrown");
@@ -2714,12 +2714,12 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         mgr.begin();
         FunctionService.onRegion(custRegion).execute(TXFunction.id).getResult();
         assertNotNull(mgr.getTXState());
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNull(mgr.getTXState());
         getGemfireCache().getLogger().fine("SWAP:callingget");
         assertNull("expected null but was:" + custRegion.get(expectedCustId),
             custRegion.get(expectedCustId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         assertEquals(expectedCustomer, custRegion.get(expectedCustId));
         return null;
@@ -2744,9 +2744,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
             context.getResultSender().lastResult(Boolean.TRUE);
           }
         }).getResult();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertEquals(custRegion.get(expectedCustId), expectedCustomer);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         assertNull(custRegion.get(expectedCustId));
         return null;
@@ -2787,9 +2787,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         filter.add(expectedCustId);
         FunctionService.onRegion(custRegion).withFilter(filter).execute(TXFunction.id).getResult();
         assertEquals(expectedCustomer, custRegion.get(expectedCustId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNull(custRegion.get(expectedCustId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return null;
       }
     });
@@ -2839,9 +2839,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         }
         FunctionService.onMember(owner).execute(TXFunction.id).getResult();
         assertEquals(expectedCustomer, pr.get(expectedCustId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNull(pr.get(expectedCustId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return null;
       }
     });
@@ -2878,9 +2878,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
         filter.add(keyOnDs);
         FunctionService.onRegion(pr).withFilter(filter).execute(TXFunction.id).getResult();
         assertEquals(expectedCustomer, pr.get(expectedCustId));
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNull(pr.get(expectedCustId));
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         return null;
       }
     };
@@ -3092,9 +3092,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
     accessor.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         TXManagerImpl mgr = getGemfireCache().getTxManager();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(tx);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
@@ -3105,9 +3105,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
     accessor.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         TXManagerImpl mgr = getGemfireCache().getTxManager();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(tx);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
@@ -3119,9 +3119,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
     accessor.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         TXManagerImpl mgr = getGemfireCache().getTxManager();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(tx);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
@@ -3163,9 +3163,9 @@ public class RemoteTransactionDUnitTest extends JUnit4CacheTestCase {
     accessor.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         TXManagerImpl mgr = getGemfireCache().getTxManager();
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertNotNull(tx);
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         return null;
       }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TransactionsWithDeltaDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TransactionsWithDeltaDUnitTest.java
index c3aa056..e153605 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/TransactionsWithDeltaDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TransactionsWithDeltaDUnitTest.java
@@ -365,9 +365,9 @@ public class TransactionsWithDeltaDUnitTest extends JUnit4CacheTestCase {
         LogWriterUtils.getLogWriter().info("SWAP:getfromtx:" + pr.get(cust1));
         LogWriterUtils.getLogWriter().info("SWAP:doingCommit");
         assertEquals("updatedName", pr.get(cust1).getName());
-        TXStateProxy tx = mgr.internalSuspend();
+        TXStateProxy tx = mgr.pauseTransaction();
         assertEquals("name1", pr.get(cust1).getName());
-        mgr.internalResume(tx);
+        mgr.unpauseTransaction(tx);
         mgr.commit();
         assertTrue(c.isToDeltaCalled());
         assertEquals(c, pr.get(cust1));
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/MyTransactionFunction.java b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/MyTransactionFunction.java
index 10e3282..d11c82b 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/MyTransactionFunction.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/MyTransactionFunction.java
@@ -415,7 +415,7 @@ public class MyTransactionFunction implements Function {
     TXManagerImpl mImp = (TXManagerImpl) mgr;
     mImp.begin();
     orderPR.put(vOrderId, vOrder);
-    TXStateProxy txState = mImp.internalSuspend();
+    TXStateProxy txState = mImp.pauseTransaction();
     Iterator it = txState.getRegions().iterator();
     Assert.assertTrue(txState.getRegions().size() == 1,
         "Expected 1 region; " + "found:" + txState.getRegions().size());
@@ -423,9 +423,9 @@ public class MyTransactionFunction implements Function {
     Assert.assertTrue(lr instanceof BucketRegion);
     TXRegionState txRegion = txState.readRegion(lr);
     TXEntryState txEntry = txRegion.readEntry(txRegion.getEntryKeys().iterator().next());
-    mImp.internalResume(txState);
+    mImp.unpauseTransaction(txState);
     orderPR.put(vOrderId, new Order("foo"));
-    txState = mImp.internalSuspend();
+    txState = mImp.pauseTransaction();
     // since both puts were on same key, verify that
     // TxRegionState and TXEntryState are same
     LocalRegion lr1 = (LocalRegion) txState.getRegions().iterator().next();
@@ -439,7 +439,7 @@ public class MyTransactionFunction implements Function {
     orderPR.put(vOrderId, new Order("foobar"));
     mImp.commit();
     // now begin the first
-    mImp.internalResume(txState);
+    mImp.unpauseTransaction(txState);
     boolean caughtException = false;
     try {
       mImp.commit();
@@ -463,18 +463,18 @@ public class MyTransactionFunction implements Function {
     mImp.begin();
     custPR.put(custId, cust);
     Assert.assertTrue(cust.equals(custPR.get(custId)));
-    TXStateProxy txState = mImp.internalSuspend();
+    TXStateProxy txState = mImp.pauseTransaction();
     Assert.assertTrue(custPR.get(custId) == null);
-    mImp.internalResume(txState);
+    mImp.unpauseTransaction(txState);
     mImp.commit();
     // change value
     mImp.begin();
     Customer oldCust = (Customer) custPR.get(custId);
     Assert.assertTrue(oldCust.equals(cust));
-    txState = mImp.internalSuspend();
+    txState = mImp.pauseTransaction();
     Customer newCust = new Customer("fooNew", "barNew");
     custPR.put(custId, newCust);
-    mImp.internalResume(txState);
+    mImp.unpauseTransaction(txState);
     Assert.assertTrue(oldCust.equals(custPR.get(custId)));
     mImp.commit();
   }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRJTADUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRJTADUnitTest.java
index 0bd5f8c..35beec5 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRJTADUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRJTADUnitTest.java
@@ -43,11 +43,16 @@ import org.apache.logging.log4j.Logger;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 /**
  * Test for JTA with PR.
  */
 @Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
 @SuppressWarnings("serial")
 public class PRJTADUnitTest extends JUnit4CacheTestCase {
 
@@ -95,45 +100,27 @@ public class PRJTADUnitTest extends JUnit4CacheTestCase {
     dataStore2.invoke(() -> createCache(preventSetOpToStartJTA));
     dataStore3.invoke(() -> createCache(preventSetOpToStartJTA));
 
-    accessor.invoke(() -> createPR());
-    dataStore1.invoke(() -> createPR());
-    dataStore2.invoke(() -> createPR());
-    dataStore3.invoke(() -> createPR());
-  }
-
-  @Test
-  public void testValuesCallStartsJTA() {
-    verifyJTASetOp(SetOp.VALUES, false);
-  }
-
-  @Test
-  public void testKeySetCallStartsJTA() {
-    verifyJTASetOp(SetOp.KEYSET, false);
-  }
-
-  @Test
-  public void testEntrySetCallStartsJTA() {
-    verifyJTASetOp(SetOp.ENTRYSET, false);
+    accessor.invoke(() -> createPR(true));
+    dataStore1.invoke(() -> createPR(false));
+    dataStore2.invoke(() -> createPR(false));
+    dataStore3.invoke(() -> createPR(false));
   }
 
   @Test
-  public void testValuesCallNotStartJTAIfDisabled() {
-    verifyJTASetOp(SetOp.VALUES, true);
+  @Parameters
+  public void testJTAWithRegionSetOperation(final SetOp op, final boolean preventSetOpToStartJTA) {
+    verifyJTASetOp(op, preventSetOpToStartJTA);
   }
 
-  @Test
-  public void testKeySetCallNotStartJTAIfDisabled() {
-    verifyJTASetOp(SetOp.KEYSET, true);
-  }
-
-  @Test
-  public void testEntrySetCallNotStartJTAIfDisabled() {
-    verifyJTASetOp(SetOp.ENTRYSET, true);
+  private Object[] parametersForTestJTAWithRegionSetOperation() {
+    return new Object[] {new Object[] {SetOp.VALUES, false}, new Object[] {SetOp.VALUES, true},
+        new Object[] {SetOp.KEYSET, false}, new Object[] {SetOp.KEYSET, true},
+        new Object[] {SetOp.ENTRYSET, false}, new Object[] {SetOp.ENTRYSET, true},};
   }
 
   String regionName = "region1";
 
-  private void basicVerifySetOp(SetOp op, boolean preventSetOpToStartJTA)
+  private void basicVerifySetOp(SetOp op, boolean preventSetOpToStartJTA, boolean isAccessor)
       throws NotSupportedException, SystemException, NamingException {
     Cache cache = basicGetCache();
     Region<Long, String> region = cache.getRegion(Region.SEPARATOR + regionName);
@@ -164,7 +151,9 @@ public class PRJTADUnitTest extends JUnit4CacheTestCase {
         assertNull(TXManagerImpl.getCurrentTXState());
       } else {
         assertNotNull(TXManagerImpl.getCurrentTXState());
-        ta.rollback();
+        if (!isAccessor) {
+          ta.rollback();
+        }
       }
     }
   }
@@ -173,10 +162,10 @@ public class PRJTADUnitTest extends JUnit4CacheTestCase {
     createRegion(preventSetOpToStartJTA);
     dataStore1.invoke(() -> initRegion());
 
-    accessor.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA));
-    dataStore1.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA));
-    dataStore2.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA));
-    dataStore3.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA));
+    accessor.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA, true));
+    dataStore1.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA, false));
+    dataStore2.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA, false));
+    dataStore3.invoke(() -> basicVerifySetOp(op, preventSetOpToStartJTA, false));
   }
 
   @SuppressWarnings("rawtypes")
@@ -257,10 +246,10 @@ public class PRJTADUnitTest extends JUnit4CacheTestCase {
     getCache();
   }
 
-  private void createPR() {
+  private void createPR(boolean isAccessor) {
     basicGetCache().createRegionFactory(RegionShortcut.PARTITION)
-        .setPartitionAttributes(
-            new PartitionAttributesFactory<Long, String>().setTotalNumBuckets(3).create())
+        .setPartitionAttributes(new PartitionAttributesFactory<Long, String>().setTotalNumBuckets(3)
+            .setLocalMaxMemory(isAccessor ? 0 : 1).create())
         .create(regionName);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRTransaction2DUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRTransaction2DUnitTest.java
index 611b234..4342228 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRTransaction2DUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/PRTransaction2DUnitTest.java
@@ -47,8 +47,13 @@ import org.apache.logging.log4j.Logger;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 @Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
 @SuppressWarnings("serial")
 public class PRTransaction2DUnitTest extends JUnit4CacheTestCase {
 
@@ -113,11 +118,16 @@ public class PRTransaction2DUnitTest extends JUnit4CacheTestCase {
     });
   }
 
-  @SuppressWarnings("rawtypes")
   private void createPartitionedRegion(String regionName, int copies, int totalBuckets) {
+    createPartitionedRegion(regionName, copies, totalBuckets, false);
+  }
+
+  @SuppressWarnings("rawtypes")
+  private void createPartitionedRegion(String regionName, int copies, int totalBuckets,
+      boolean isAccessor) {
     RegionFactory<Integer, String> factory = getCache().createRegionFactory();
     PartitionAttributes pa = new PartitionAttributesFactory().setTotalNumBuckets(totalBuckets)
-        .setRedundantCopies(copies).create();
+        .setRedundantCopies(copies).setLocalMaxMemory(isAccessor ? 0 : 1).create();
     factory.setPartitionAttributes(pa).create(regionName);
   }
 
@@ -195,44 +205,26 @@ public class PRTransaction2DUnitTest extends JUnit4CacheTestCase {
     dataStore3.invoke(() -> createCache(preventSetOpToStartTx));
 
     int totalBuckets = 3;
-    accessor.invoke(() -> createPartitionedRegion(regionName, 0, totalBuckets));
+    accessor.invoke(() -> createPartitionedRegion(regionName, 0, totalBuckets, true));
     dataStore1.invoke(() -> createPartitionedRegion(regionName, 0, totalBuckets));
     dataStore2.invoke(() -> createPartitionedRegion(regionName, 0, totalBuckets));
     dataStore3.invoke(() -> createPartitionedRegion(regionName, 0, totalBuckets));
   }
 
   @Test
-  public void testValuesCallStartsTx() {
-    verifySetOp(SetOp.VALUES, false);
+  @Parameters
+  public void testRegionSetOpWithTx(SetOp op, boolean preventSetOpToStartTx) {
+    verifySetOp(op, preventSetOpToStartTx);
   }
 
-  @Test
-  public void testKeySetCallStartsTx() {
-    verifySetOp(SetOp.KEYSET, false);
-  }
-
-  @Test
-  public void testEntrySetCallStartsTx() {
-    verifySetOp(SetOp.ENTRYSET, false);
-  }
-
-  @Test
-  public void testValuesCallNotStartTxIfDisabled() {
-    verifySetOp(SetOp.VALUES, true);
-  }
-
-  @Test
-  public void testKeySetCallNotStartTxIfDisabled() {
-    verifySetOp(SetOp.KEYSET, true);
-  }
-
-  @Test
-  public void testEntrySetCallNotStartTxIfDisabled() {
-    verifySetOp(SetOp.ENTRYSET, true);
+  private Object[] parametersForTestRegionSetOpWithTx() {
+    return new Object[] {new Object[] {SetOp.VALUES, false}, new Object[] {SetOp.VALUES, true},
+        new Object[] {SetOp.KEYSET, false}, new Object[] {SetOp.KEYSET, true},
+        new Object[] {SetOp.ENTRYSET, false}, new Object[] {SetOp.ENTRYSET, true},};
   }
 
   @SuppressWarnings("rawtypes")
-  private void basicVerifySetOp(SetOp op, boolean preventSetOpToStartTx) {
+  private void basicVerifySetOp(SetOp op, boolean preventSetOpToStartTx, boolean isAccessor) {
     Cache cache = basicGetCache();
     Region<Long, String> region = cache.getRegion(Region.SEPARATOR + regionName);
 
@@ -255,7 +247,7 @@ public class PRTransaction2DUnitTest extends JUnit4CacheTestCase {
       verifySetOp(op, region, keys, values, set);
     } finally {
       assertNotNull(TXManagerImpl.getCurrentTXState());
-      if (preventSetOpToStartTx) {
+      if (preventSetOpToStartTx || isAccessor) {
         assertFalse(((TXStateProxyImpl) TXManagerImpl.getCurrentTXState()).hasRealDeal());
       } else {
         assertTrue(((TXStateProxyImpl) TXManagerImpl.getCurrentTXState()).hasRealDeal());
@@ -268,10 +260,10 @@ public class PRTransaction2DUnitTest extends JUnit4CacheTestCase {
     createRegion(preventSetOpToStartTx);
     dataStore1.invoke(() -> initRegion());
 
-    accessor.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx));
-    dataStore1.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx));
-    dataStore2.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx));
-    dataStore3.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx));
+    accessor.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx, true));
+    dataStore1.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx, false));
+    dataStore2.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx, false));
+    dataStore3.invoke(() -> basicVerifySetOp(op, preventSetOpToStartTx, false));
   }
 
   @SuppressWarnings("rawtypes")
@@ -336,5 +328,4 @@ public class PRTransaction2DUnitTest extends JUnit4CacheTestCase {
     }
     getCache();
   }
-
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java
index f888fef..a83a6b7 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java
@@ -85,9 +85,9 @@ public class ClientServerJTADUnitTest extends JUnit4CacheTestCase {
     TXManagerImpl mgr = (TXManagerImpl) getCache().getCacheTransactionManager();
     mgr.begin();
     region.put(key, newValue);
-    TXStateProxyImpl tx = (TXStateProxyImpl) mgr.internalSuspend();
+    TXStateProxyImpl tx = (TXStateProxyImpl) mgr.pauseTransaction();
     ClientTXStateStub txStub = (ClientTXStateStub) tx.getRealDeal(null, null);
-    mgr.internalResume(tx);
+    mgr.unpauseTransaction(tx);
     try {
       txStub.beforeCompletion();
       fail("expected to get CommitConflictException");
@@ -145,9 +145,9 @@ public class ClientServerJTADUnitTest extends JUnit4CacheTestCase {
     TXManagerImpl mgr = (TXManagerImpl) getCache().getCacheTransactionManager();
     mgr.begin();
     region.put(key, newValue);
-    TXStateProxyImpl tx = (TXStateProxyImpl) mgr.internalSuspend();
+    TXStateProxyImpl tx = (TXStateProxyImpl) mgr.pauseTransaction();
     ClientTXStateStub txStub = (ClientTXStateStub) tx.getRealDeal(null, null);
-    mgr.internalResume(tx);
+    mgr.unpauseTransaction(tx);
     txStub.beforeCompletion();
     if (withWait) {
       getBlackboard().signalGate(first);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/jta/JtaIntegrationJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/jta/JtaIntegrationJUnitTest.java
index c7d8a06..b6a7fab 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/jta/JtaIntegrationJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/jta/JtaIntegrationJUnitTest.java
@@ -26,6 +26,10 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 import javax.naming.Context;
 import javax.naming.NamingException;
@@ -47,6 +51,7 @@ import java.util.Iterator;
  * 
  */
 @Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class JtaIntegrationJUnitTest {
 
   private static final Logger logger = LogService.getLogger();
@@ -141,41 +146,17 @@ public class JtaIntegrationJUnitTest {
     }
   }
 
-
-  @Test
-  public void testValuesCallStartsJTA()
-      throws NotSupportedException, SystemException, NamingException {
-    verifyJTASetOp(SetOp.VALUES, false);
-  }
-
-  @Test
-  public void testKeySetCallStartsJTA()
-      throws NotSupportedException, SystemException, NamingException {
-    verifyJTASetOp(SetOp.KEYSET, false);
-  }
-
   @Test
-  public void testEntrySetCallStartsJTA()
+  @Parameters
+  public void testRegionSetOpWithJTA(SetOp op, boolean preventSetOpToStartJTA)
       throws NotSupportedException, SystemException, NamingException {
-    verifyJTASetOp(SetOp.ENTRYSET, false);
+    verifyJTASetOp(op, preventSetOpToStartJTA);
   }
 
-  @Test
-  public void testValuesCallNotStartJTAIfDisabled()
-      throws NotSupportedException, SystemException, NamingException {
-    verifyJTASetOp(SetOp.VALUES, true);
-  }
-
-  @Test
-  public void testKeySetCallNotStartJTAIfDisabled()
-      throws NotSupportedException, SystemException, NamingException {
-    verifyJTASetOp(SetOp.KEYSET, true);
-  }
-
-  @Test
-  public void testEntrySetCallNotStartJTAIfDisabled()
-      throws NotSupportedException, SystemException, NamingException {
-    verifyJTASetOp(SetOp.ENTRYSET, true);
+  private Object[] parametersForTestRegionSetOpWithJTA() {
+    return new Object[] {new Object[] {SetOp.VALUES, false}, new Object[] {SetOp.VALUES, true},
+        new Object[] {SetOp.KEYSET, false}, new Object[] {SetOp.KEYSET, true},
+        new Object[] {SetOp.ENTRYSET, false}, new Object[] {SetOp.ENTRYSET, true},};
   }
 
   long k1 = 1;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java
new file mode 100644
index 0000000..f5afc43
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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 org.apache.geode.internal.lang;
+
+import static org.junit.Assert.*;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class SystemPropertyHelperJUnitTest {
+  String preventSetOpBootstrapTransaction = "preventSetOpBootstrapTransaction";
+
+  @Test
+  public void testPreventSetOpBootstrapTransactionDefaultToFalse() {
+    assertFalse(SystemPropertyHelper.preventSetOpBootstrapTransaction());
+  }
+
+  @Test
+  public void testPreventSetOpBootstrapTransactionSystemProperty() {
+    String gemfirePrefixProperty = "gemfire." + preventSetOpBootstrapTransaction;
+    System.setProperty(gemfirePrefixProperty, "true");
+    assertTrue(SystemPropertyHelper.preventSetOpBootstrapTransaction());
+    System.clearProperty(gemfirePrefixProperty);
+
+    String geodePrefixProperty = "geode." + preventSetOpBootstrapTransaction;
+    System.setProperty(geodePrefixProperty, "true");
+    assertTrue(SystemPropertyHelper.preventSetOpBootstrapTransaction());
+    System.clearProperty(geodePrefixProperty);
+  }
+
+  @Test
+  public void testPreventSetOpBootstrapTransactionGeodePreference() {
+    String gemfirePrefixProperty = "gemfire." + preventSetOpBootstrapTransaction;
+    String geodePrefixProperty = "geode." + preventSetOpBootstrapTransaction;
+    System.setProperty(geodePrefixProperty, "false");
+    System.setProperty(gemfirePrefixProperty, "true");
+    assertFalse(SystemPropertyHelper.preventSetOpBootstrapTransaction());
+    System.clearProperty(geodePrefixProperty);
+    System.clearProperty(gemfirePrefixProperty);
+  }
+
+}

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].