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 2018/01/11 22:20:40 UTC

[geode] branch feature/GEODE-4204-4207-pre updated (cecc9f7 -> fb54d3c)

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

eshu11 pushed a change to branch feature/GEODE-4204-4207-pre
in repository https://gitbox.apache.org/repos/asf/geode.git.


 discard cecc9f7  GEODE-4204: GEODE-4207: pass in cache instead of using singleton
     add 0530964  GEODE-4203: Remove GemFireCacheImpl.getInstance from PartitionedRegionHelper
     add 2c64247  GEODE-4213: do not call getExisting
     add 0d33da8  GEODE-4206: remove getInstance calls in GFSnapshot
     add 0cd1436  GEODE-4196: Fix incorrect creation of CacheClosedException
     add 97474de  Revert "GEODE-4200: get cache from DM instead of using singleton."
     add 283c16e  Merge branch 'feature/GEODE-3583-storage' of ssh://github.com/apache/geode into feature/GEODE-3583-storage
     new 72b1cfb  GEODE-4204: GEODE-4207: pass in cache instead of using singleton
     new cf3a92d  GEODE-4196: fix mocking of dm.getCache in PartitionMessageTest
     new fb54d3c  GEODE-4204: fix TXRegionLockRequestImpl in AnalyzeSerializables

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (cecc9f7)
            \
             N -- N -- N   refs/heads/feature/GEODE-4204-4207-pre (fb54d3c)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/geode/cache/CacheFactory.java  |  2 +-
 .../geode/cache/snapshot/SnapshotReader.java       | 16 ++++-
 .../internal/cache/CacheDistributionAdvisor.java   |  2 +-
 .../geode/internal/cache/CloseCacheMessage.java    |  2 +-
 .../internal/cache/InitialImageOperation.java      | 62 ++++---------------
 .../apache/geode/internal/cache/LocalRegion.java   | 16 +++++
 .../internal/cache/PartitionedRegionHelper.java    | 16 +++--
 .../geode/internal/cache/TXCommitMessage.java      | 10 +---
 .../internal/cache/TXRegionLockRequestImpl.java    |  2 +-
 .../cache/snapshot/CacheSnapshotServiceImpl.java   |  2 +-
 .../geode/internal/cache/snapshot/GFSnapshot.java  | 27 +++------
 .../cache/snapshot/RegionSnapshotServiceImpl.java  |  7 +--
 .../geode/internal/cache/CacheProfileTest.java     | 57 ------------------
 .../internal/cache/InitialImageOperationTest.java  | 56 ------------------
 .../geode/internal/cache/RegionCommitTest.java     | 58 ------------------
 .../cache/RequestFilterInfoMessageTest.java        | 30 ----------
 .../cache/partitioned/PartitionMessageTest.java    |  1 +
 .../snapshot/GFSnapshotJUnitPerformanceTest.java   |  4 +-
 .../codeAnalysis/sanctionedDataSerializables.txt   | 69 +++++++++++-----------
 19 files changed, 112 insertions(+), 327 deletions(-)
 delete mode 100644 geode-core/src/test/java/org/apache/geode/internal/cache/CacheProfileTest.java
 delete mode 100644 geode-core/src/test/java/org/apache/geode/internal/cache/InitialImageOperationTest.java
 delete mode 100644 geode-core/src/test/java/org/apache/geode/internal/cache/RegionCommitTest.java

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

[geode] 02/03: GEODE-4196: fix mocking of dm.getCache in PartitionMessageTest

Posted by es...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit cf3a92d6d9bffcc4678f0efc817bfe5223976e82
Author: eshu <es...@pivotal.io>
AuthorDate: Thu Jan 11 13:57:23 2018 -0800

    GEODE-4196: fix mocking of dm.getCache in PartitionMessageTest
---
 .../apache/geode/internal/cache/partitioned/PartitionMessageTest.java    | 1 +
 1 file changed, 1 insertion(+)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java
index 7ee108f..cfdb368 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java
@@ -62,6 +62,7 @@ public class PartitionMessageTest {
     when(msg.getInternalCache()).thenReturn(cache);
     when(msg.getStartPartitionMessageProcessingTime(pr)).thenReturn(startTime);
     when(msg.getTXManagerImpl(cache)).thenReturn(txMgr);
+    when(dm.getCache()).thenReturn(cache);
 
     doAnswer(CALLS_REAL_METHODS).when(msg).process(dm);
   }

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

[geode] 01/03: GEODE-4204: GEODE-4207: pass in cache instead of using singleton

Posted by es...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 72b1cfbb5e2c7dd02efc9bc7463f52f34777b54c
Author: eshu <es...@pivotal.io>
AuthorDate: Thu Jan 11 10:36:09 2018 -0800

    GEODE-4204: GEODE-4207: pass in cache instead of using singleton
---
 .../cache/DistTXStateProxyImplOnCoordinator.java   |  3 +-
 .../internal/cache/TXRegionLockRequestImpl.java    | 33 +++++----
 .../apache/geode/internal/cache/TXRegionState.java |  4 +-
 .../geode/internal/cache/TXStateProxyImpl.java     |  3 +-
 .../geode/internal/cache/tx/ClientTXStateStub.java | 61 +++++++++-------
 .../internal/cache/tx/DistClientTXStateStub.java   | 16 ++---
 .../cache/TXRegionLockRequestImplTest.java         | 55 +++++++++++++++
 .../internal/cache/TXReservationMgrJUnitTest.java  |  2 +-
 .../internal/cache/tx/ClientTXStateStubTest.java   | 82 ++++++++++++++++++++++
 9 files changed, 201 insertions(+), 58 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java
index a477d6c..0b0e260 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java
@@ -331,7 +331,8 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
         // Code to keep going forward
         if (r.hasServerProxy()) {
           // TODO [DISTTX] See what we need for client?
-          this.realDeal = new DistClientTXStateStub(this, target, r);
+          this.realDeal =
+              new DistClientTXStateStub(r.getCache(), r.getDistributionManager(), this, target, r);
           if (r.scope.isDistributed()) {
             if (txDistributedClientWarningIssued.compareAndSet(false, true)) {
               logger.warn(LocalizedMessage.create(
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java
index 1e586aa..e0e058c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java
@@ -23,6 +23,7 @@ import java.util.Set;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.TestingOnly;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.locks.TXRegionLockRequest;
@@ -38,6 +39,8 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
   private static final long serialVersionUID = 5840033961584078082L;
   private static final Logger logger = LogService.getLogger();
 
+  private InternalCache cache;
+
   private transient LocalRegion r;
 
   private String regionPath;
@@ -46,9 +49,11 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
 
   public TXRegionLockRequestImpl() {
     // for DataSerializer
+    this.cache = null;
   }
 
-  public TXRegionLockRequestImpl(LocalRegion r) {
+  public TXRegionLockRequestImpl(InternalCache cache, LocalRegion r) {
+    this.cache = cache;
     this.r = r;
     this.regionPath = null;
     this.entryKeys = null;
@@ -57,7 +62,9 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
   /**
    * Used by unit tests
    */
+  @TestingOnly
   public TXRegionLockRequestImpl(String regionPath, Set<Object> entryKeys) {
+    this.cache = null;
     this.regionPath = regionPath;
     this.entryKeys = entryKeys;
   }
@@ -66,6 +73,7 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
     return this.entryKeys == null || this.entryKeys.isEmpty();
   }
 
+  @Override
   public void addEntryKeys(Set<Object> s) {
     if (s == null || s.isEmpty()) {
       return;
@@ -85,6 +93,7 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
     }
   }
 
+  @Override
   public void addEntryKey(Object key) {
     if (this.entryKeys == null) {
       this.entryKeys = new HashSet<Object>();
@@ -92,10 +101,11 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
     this.entryKeys.add(key);
   }
 
+  @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
     this.regionPath = DataSerializer.readString(in);
 
-    final InternalCache cache = getCache(false);
+    cache = GemFireCacheImpl.getInstance();
     try {
       final int size = InternalDataSerializer.readArrayLength(in);
       if (cache != null && size > 0) {
@@ -129,6 +139,7 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
     return set;
   }
 
+  @Override
   public void toData(DataOutput out) throws IOException {
     DataSerializer.writeString(getRegionFullPath(), out);
     InternalDataSerializer.writeSet(this.entryKeys, out);
@@ -141,6 +152,7 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
     return result;
   }
 
+  @Override
   public String getRegionFullPath() {
     if (this.regionPath == null) {
       this.regionPath = this.r.getFullPath();
@@ -148,28 +160,15 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
     return this.regionPath;
   }
 
+  @Override
   public Set<Object> getKeys() {
     if (this.entryKeys == null) {
       // check for cache closed/closing
-      getCache(true);
+      cache.getCancelCriterion().checkCancelInProgress(null);
     }
     return this.entryKeys;
   }
 
-  private InternalCache getCache(boolean throwIfClosing) {
-    final InternalCache cache = GemFireCacheImpl.getInstance();
-    if (cache != null && !cache.isClosed()) {
-      if (throwIfClosing) {
-        cache.getCancelCriterion().checkCancelInProgress(null);
-      }
-      return cache;
-    }
-    if (throwIfClosing) {
-      throw cache.getCacheClosedException("The cache is closed.", null);
-    }
-    return null;
-  }
-
   /**
    * Only safe to call in the vm that creates this request. Once it is serialized this method will
    * return null.
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionState.java
index 7d16574..20febe9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionState.java
@@ -222,7 +222,7 @@ public class TXRegionState {
     final boolean distributedTX = !byPassDLock && r.getScope().isDistributedAck();
     if (this.uaMods != null || (!distributedTX && this.entryMods.size() > 0)) {
       // need some local locks
-      TXRegionLockRequestImpl rlr = new TXRegionLockRequestImpl(r);
+      TXRegionLockRequestImpl rlr = new TXRegionLockRequestImpl(r.getCache(), r);
       if (this.uaMods != null) {
         rlr.addEntryKeys(this.uaMods.keySet());
       }
@@ -235,7 +235,7 @@ public class TXRegionState {
     }
     if (distributedTX && this.entryMods.size() > 0) {
       // need some distributed locks
-      TXRegionLockRequestImpl rlr = new TXRegionLockRequestImpl(r);
+      TXRegionLockRequestImpl rlr = new TXRegionLockRequestImpl(r.getCache(), r);
       rlr.addEntryKeys(getLockRequestEntryKeys());
       if (!rlr.isEmpty()) {
         req.setOtherMembers(this.otherMembers);
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 7e0fea4..d33c171 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
@@ -121,7 +121,8 @@ public class TXStateProxyImpl implements TXStateProxy {
       } else {
         // Code to keep going forward
         if (r.hasServerProxy()) {
-          this.realDeal = new ClientTXStateStub(this, target, r);
+          this.realDeal =
+              new ClientTXStateStub(r.getCache(), r.getDistributionManager(), this, target, r);
           if (r.scope.isDistributed()) {
             if (txDistributedClientWarningIssued.compareAndSet(false, true)) {
               logger.warn(LocalizedMessage.create(
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
index 2c52538..f0a3871 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
@@ -32,10 +32,8 @@ import org.apache.geode.cache.client.internal.ServerRegionProxy;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.TXCommitMessage;
@@ -54,19 +52,6 @@ public class ClientTXStateStub extends TXStateStub {
   /** test hook - used to find out what operations were performed in the last tx */
   private static ThreadLocal<List<TransactionalOperation>> recordedTransactionalOperations = null;
 
-  private final ServerRegionProxy firstProxy;
-
-  private ServerLocation serverAffinityLocation;
-
-  /** the operations performed in the current transaction are held in this list */
-  private List<TransactionalOperation> recordedOperations =
-      Collections.synchronizedList(new LinkedList<TransactionalOperation>());
-
-  /** lock request for obtaining local locks */
-  private TXLockRequest lockReq;
-
-  private Runnable internalAfterLocalLocks;
-
   /**
    * System property to disable conflict checks on clients.
    */
@@ -80,6 +65,21 @@ public class ClientTXStateStub extends TXStateStub {
     return !DISABLE_CONFLICT_CHECK_ON_CLIENT || recordedTransactionalOperations != null;
   }
 
+  private final ServerRegionProxy firstProxy;
+  private final InternalCache cache;
+  private final DistributionManager dm;
+
+  /** the operations performed in the current transaction are held in this list */
+  private final List<TransactionalOperation> recordedOperations =
+      Collections.synchronizedList(new LinkedList<TransactionalOperation>());
+
+  private ServerLocation serverAffinityLocation;
+
+  /** lock request for obtaining local locks */
+  private TXLockRequest lockReq;
+
+  private Runnable internalAfterLocalLocks;
+
   private boolean txRolledback = false;
 
   /**
@@ -92,10 +92,12 @@ public class ClientTXStateStub extends TXStateStub {
     recordedTransactionalOperations = t;
   }
 
-  public ClientTXStateStub(TXStateProxy stateProxy, DistributedMember target,
-      LocalRegion firstRegion) {
+  public ClientTXStateStub(InternalCache cache, DistributionManager dm, TXStateProxy stateProxy,
+      DistributedMember target, LocalRegion firstRegion) {
     super(stateProxy, target);
-    firstProxy = firstRegion.getServerProxy();
+    this.cache = cache;
+    this.dm = dm;
+    this.firstProxy = firstRegion.getServerProxy();
     this.firstProxy.getPool().setupServerAffinity(true);
     if (recordedTransactionalOperations != null) {
       recordedTransactionalOperations.set(this.recordedOperations);
@@ -116,19 +118,26 @@ public class ClientTXStateStub extends TXStateStub {
     }
   }
 
+  TXLockRequest createTXLockRequest() {
+    return new TXLockRequest();
+  }
+
+  TXRegionLockRequestImpl createTXRegionLockRequestImpl(InternalCache cache, LocalRegion region) {
+    return new TXRegionLockRequestImpl(cache, region);
+  }
+
   /**
    * Lock the keys in a local transaction manager
    *
    * @throws CommitConflictException if the key is already locked by some other transaction
    */
   private void obtainLocalLocks() {
-    lockReq = new TXLockRequest();
-    InternalCache cache = GemFireCacheImpl.getExisting("");
+    lockReq = createTXLockRequest();
     for (TransactionalOperation txOp : this.recordedOperations) {
       if (ServerRegionOperation.lockKeyForTx(txOp.getOperation())) {
         TXRegionLockRequest rlr = lockReq.getRegionLockRequest(txOp.getRegionName());
         if (rlr == null) {
-          rlr = new TXRegionLockRequestImpl(cache.getRegionByPath(txOp.getRegionName()));
+          rlr = createTXRegionLockRequestImpl(cache, cache.getRegionByPath(txOp.getRegionName()));
           lockReq.addLocalRequest(rlr);
         }
         if (txOp.getOperation() == ServerRegionOperation.PUT_ALL
@@ -159,14 +168,12 @@ public class ClientTXStateStub extends TXStateStub {
       this.internalAfterSendCommit.run();
     }
 
-    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache == null) {
+      // we can probably delete this block because cache is now a final var
       // fixes bug 42933
       return;
     }
     cache.getCancelCriterion().checkCancelInProgress(null);
-    InternalDistributedSystem ds = cache.getInternalDistributedSystem();
-    DistributionManager dm = ds.getDistributionManager();
 
     txcm.setDM(dm);
     txcm.setAckRequired(false);
@@ -247,6 +254,7 @@ public class ClientTXStateStub extends TXStateStub {
     }
   }
 
+  @Override
   public InternalDistributedMember getOriginatingMember() {
     /*
      * Client member id is implied from the connection so we don't need this
@@ -254,6 +262,7 @@ public class ClientTXStateStub extends TXStateStub {
     return null;
   }
 
+  @Override
   public boolean isMemberIdForwardingRequired() {
     /*
      * Client member id is implied from the connection so we don't need this Forwarding will occur
@@ -262,11 +271,13 @@ public class ClientTXStateStub extends TXStateStub {
     return false;
   }
 
+  @Override
   public TXCommitMessage getCommitMessage() {
     /* client gets the txcommit message during Op processing and doesn't need it here */
     return null;
   }
 
+  @Override
   public void suspend() {
     this.serverAffinityLocation = this.firstProxy.getPool().getServerAffinityLocation();
     this.firstProxy.getPool().releaseServerAffinity();
@@ -276,6 +287,7 @@ public class ClientTXStateStub extends TXStateStub {
     }
   }
 
+  @Override
   public void resume() {
     this.firstProxy.getPool().setupServerAffinity(true);
     this.firstProxy.getPool().setServerAffinityLocation(this.serverAffinityLocation);
@@ -288,6 +300,7 @@ public class ClientTXStateStub extends TXStateStub {
   /**
    * test hook - maintain a list of tx operations
    */
+  @Override
   public void recordTXOperation(ServerRegionDataAccess region, ServerRegionOperation op, Object key,
       Object arguments[]) {
     if (ClientTXStateStub.transactionRecordingEnabled()) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/DistClientTXStateStub.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/DistClientTXStateStub.java
index 6356f7a..1464124 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/DistClientTXStateStub.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/DistClientTXStateStub.java
@@ -25,24 +25,16 @@ import org.apache.geode.internal.cache.DistTXCommitMessage;
 import org.apache.geode.internal.cache.DistTXCoordinatorInterface;
 import org.apache.geode.internal.cache.DistTXPrecommitMessage;
 import org.apache.geode.internal.cache.DistTXRollbackMessage;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.TXStateProxy;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
-/**
- *
- */
 public class DistClientTXStateStub extends ClientTXStateStub implements DistTXCoordinatorInterface {
 
-  /**
-   * @param stateProxy
-   * @param target
-   * @param firstRegion
-   */
-  public DistClientTXStateStub(TXStateProxy stateProxy, DistributedMember target,
-      LocalRegion firstRegion) {
-    super(stateProxy, target, firstRegion);
-    // TODO Auto-generated constructor stub
+  public DistClientTXStateStub(InternalCache cache, DistributionManager dm, TXStateProxy stateProxy,
+      DistributedMember target, LocalRegion firstRegion) {
+    super(cache, dm, stateProxy, target, firstRegion);
   }
 
   @Override
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TXRegionLockRequestImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TXRegionLockRequestImplTest.java
new file mode 100644
index 0000000..19cb5b8
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TXRegionLockRequestImplTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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 static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.CancelException;
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class TXRegionLockRequestImplTest {
+
+  private InternalCache cache;
+  private LocalRegion region;
+  private CancelCriterion cancelCriterion;
+
+  @Before
+  public void setUp() {
+    cache = mock(InternalCache.class);
+    region = mock(LocalRegion.class);
+    cancelCriterion = mock(CancelCriterion.class);
+
+    when(cache.getCancelCriterion()).thenReturn(cancelCriterion);
+    doThrow(new CacheClosedException()).when(cancelCriterion).checkCancelInProgress(any());
+  }
+
+  @Test
+  public void getKeysThrowsCancelExceptionIfCacheIsClosed() {
+    TXRegionLockRequestImpl txRegionLockRequest = new TXRegionLockRequestImpl(cache, region);
+    assertThatThrownBy(() -> txRegionLockRequest.getKeys()).isInstanceOf(CancelException.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TXReservationMgrJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TXReservationMgrJUnitTest.java
index 6a9072a..90d1574 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/TXReservationMgrJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TXReservationMgrJUnitTest.java
@@ -70,7 +70,7 @@ public class TXReservationMgrJUnitTest {
       do {
         try {
           IdentityArrayList l = new IdentityArrayList(1);
-          TXRegionLockRequestImpl lr = new TXRegionLockRequestImpl(this.r);
+          TXRegionLockRequestImpl lr = new TXRegionLockRequestImpl(this.r.getCache(), this.r);
           lr.addEntryKeys(Collections.singleton(key));
           l.add(lr);
           mgr.makeReservation(l);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tx/ClientTXStateStubTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tx/ClientTXStateStubTest.java
new file mode 100644
index 0000000..e6336ef
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tx/ClientTXStateStubTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.tx;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.CancelException;
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.client.internal.InternalPool;
+import org.apache.geode.cache.client.internal.ServerRegionProxy;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.TXId;
+import org.apache.geode.internal.cache.TXLockRequest;
+import org.apache.geode.internal.cache.TXRegionLockRequestImpl;
+import org.apache.geode.internal.cache.TXStateProxy;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ClientTXStateStubTest {
+
+  private InternalCache cache;
+  private DistributionManager dm;
+  private TXStateProxy stateProxy;
+  private DistributedMember target;
+  private LocalRegion region;
+  private ServerRegionProxy serverRegionProxy;
+  private CancelCriterion cancelCriterion;
+
+  @Before
+  public void setUp() {
+    cache = mock(InternalCache.class);
+    dm = mock(DistributionManager.class);
+    stateProxy = mock(TXStateProxy.class);
+    target = mock(DistributedMember.class);
+    region = mock(LocalRegion.class);
+    serverRegionProxy = mock(ServerRegionProxy.class);
+    cancelCriterion = mock(CancelCriterion.class);
+
+    when(region.getServerProxy()).thenReturn(serverRegionProxy);
+    when(serverRegionProxy.getPool()).thenReturn(mock(InternalPool.class));
+    when(stateProxy.getTxId()).thenReturn(mock(TXId.class));
+    when(cache.getCancelCriterion()).thenReturn(cancelCriterion);
+    doThrow(new CacheClosedException()).when(cancelCriterion).checkCancelInProgress(any());
+  }
+
+  @Test
+  public void commitThrowsCancelExceptionIfCacheIsClosed() {
+    ClientTXStateStub stub = spy(new ClientTXStateStub(cache, dm, stateProxy, target, region));
+
+    when(stub.createTXLockRequest()).thenReturn(mock(TXLockRequest.class));
+    when(stub.createTXRegionLockRequestImpl(any(), any()))
+        .thenReturn(mock(TXRegionLockRequestImpl.class));
+
+    assertThatThrownBy(() -> stub.commit()).isInstanceOf(CancelException.class);
+  }
+
+}

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

[geode] 03/03: GEODE-4204: fix TXRegionLockRequestImpl in AnalyzeSerializables

Posted by es...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit fb54d3c671abd756112dd17837d937856018c905
Author: eshu <es...@pivotal.io>
AuthorDate: Thu Jan 11 14:18:04 2018 -0800

    GEODE-4204: fix TXRegionLockRequestImpl in AnalyzeSerializables
---
 .../internal/cache/TXRegionLockRequestImpl.java    |  2 +-
 .../codeAnalysis/sanctionedDataSerializables.txt   | 69 +++++++++++-----------
 2 files changed, 36 insertions(+), 35 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java
index e0e058c..ca33352 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXRegionLockRequestImpl.java
@@ -39,7 +39,7 @@ public class TXRegionLockRequestImpl implements TXRegionLockRequest {
   private static final long serialVersionUID = 5840033961584078082L;
   private static final Logger logger = LogService.getLogger();
 
-  private InternalCache cache;
+  private transient InternalCache cache;
 
   private transient LocalRegion r;
 
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
index 5194f46..e842624 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
@@ -867,20 +867,20 @@ fromData,113,2a2bb700692bb9006a01003d2a1cb6001e2a1cb7006b9900162abb006c59b7006db
 toData,101,2a2bb700612b2ab6001db9006202002ab40028c6000b2ab400282bb800632ab4000eb9002e01009a000c2a2ab4000e2bb700642ab4000fb9002e01009a000c2a2ab4000f2bb700642ab400192bb800652ab40018b9003201009a000b2ab400182bb80065b1
 
 org/apache/geode/internal/cache/CacheServerAdvisor$CacheServerProfile,2
-fromData,53,2a2bb700112a2bb80012b500042a2bb900130100b500062abb001459b70015b500052ab400052bb800162a2bb900170100b60018b1
-toData,42,2a2bb7000b2ab400042bb8000c2b2ab40006b9000d02002ab400052bb8000e2b2ab6000fb900100300b1
+fromData,53,2a2bb700122a2bb80013b500042a2bb900140100b500062abb001559b70016b500052ab400052bb800172a2bb900180100b60019b1
+toData,42,2a2bb7000c2ab400042bb8000d2b2ab40006b9000e02002ab400052bb8000f2b2ab60010b900110300b1
 
 org/apache/geode/internal/cache/ClientRegionEventImpl,2
 fromData,14,2a2bb700142a2bb80015b60003b1
 toData,14,2a2bb700122ab6000d2bb80013b1
 
 org/apache/geode/internal/cache/CloseCacheMessage,2
-fromData,16,2a2bb700172a2bb900180100b50002b1
-toData,16,2a2bb700192b2ab40002b9001a0200b1
+fromData,16,2a2bb700182a2bb900190100b50002b1
+toData,16,2a2bb7001a2b2ab40002b9001b0200b1
 
 org/apache/geode/internal/cache/ControllerAdvisor$ControllerProfile,2
-fromData,6,2a2bb70007b1
-toData,6,2a2bb70006b1
+fromData,6,2a2bb70008b1
+toData,6,2a2bb70007b1
 
 org/apache/geode/internal/cache/CreateRegionProcessor$CreateRegionMessage,2
 fromData,45,2a2bb7009d2a2bb8009eb500082a2bb8009fc00053b500412a2bb900a00100b500032a2bb900a10100b50063b1
@@ -962,8 +962,8 @@ fromData,197,2a2bb7003a2a2bb8003bc0003cb500052a2bb8003d88b500152a2ab40015bd003eb
 toData,165,2a2bb700492ab400052bb8004a2ab40015852bb8004b2ab400159e007bbb004c592ab40015b7004d4d033e03360415042ab40015a200511d9a00122ab40006150432b4002cc60005043e2ab40006150432b4002c3a052c1905b6004e572ab4000615043201b5002c2ab400061504322bb6004f2ab400061504321905b5002c840401a7ffac2b1db9005002001d9900082c2bb800512ab4000bc6000b2ab4000b2bb8004ab1
 
 org/apache/geode/internal/cache/DistributedRegionFunctionStreamingMessage,2
-fromData,171,2a2bb700642bb9006501003d1c047e9900142a2bb900660100b500092ab40009b800671c077e99000d2a2bb900660100b500061c057e99000e2a2bb80068c00069b500072bb800684e2dc1006a9900252a03b5000e2a2dc0006ab8006bb500082ab40008c7001b2a2dc0006ab5004fa700102a2dc0006cb500082a04b5000e2a2bb80068c0006db5000a2a2bb8006eb5000c2a2bb8006fb5000b2a1c10407e99000704a7000403b5000db1
-toData,173,2a2bb70071033d2ab400099900081c0480933d2ab40006029f00081c0780933d2ab40007c600081c0580933d2ab4000d9900091c104080933d2b1cb9007202002ab4000999000d2b2ab40009b9007302002ab40006029f000d2b2ab40006b9007302002ab40007c6000b2ab400072bb800742ab4000e99000e2ab400082bb80074a700102ab40008b9005a01002bb800742ab4000a2bb800742ab4000cc000752bb800762ab4000b2bb80077b1
+fromData,171,2a2bb700652bb9006601003d1c047e9900142a2bb900670100b500092ab40009b800681c077e99000d2a2bb900670100b500061c057e99000e2a2bb80069c0006ab500072bb800694e2dc1006b9900252a03b5000e2a2dc0006bb8006cb500082ab40008c7001b2a2dc0006bb50050a700102a2dc0006db500082a04b5000e2a2bb80069c0006eb5000a2a2bb8006fb5000c2a2bb80070b5000b2a1c10407e99000704a7000403b5000db1
+toData,173,2a2bb70072033d2ab400099900081c0480933d2ab40006029f00081c0780933d2ab40007c600081c0580933d2ab4000d9900091c104080933d2b1cb9007302002ab4000999000d2b2ab40009b9007402002ab40006029f000d2b2ab40006b9007402002ab40007c6000b2ab400072bb800752ab4000e99000e2ab400082bb80075a700102ab40008b9005b01002bb800752ab4000a2bb800752ab4000cc000762bb800772ab4000b2bb80078b1
 
 org/apache/geode/internal/cache/DistributedRemoveAllOperation$RemoveAllEntryData,1
 toData,136,2ab4000a4d2c2bb800412b2ab40010b40042b9004302002ab400063e2ab40024c600081d0780913e2ab40015c600191d100880913e2ab40015c100449900091d102080913e1d108080913e2b1db9004302002ab40024c6000b2ab400242bb800412ab40015c6000b2ab400152bb800452ab6002699000b2ab400122bb800452ab400082bb80046b1
@@ -1051,8 +1051,8 @@ fromData,64,2a2bb700102a2bb900110100b5000b2a2bb900120100b5000c2a2bb900110100b500
 toData,90,2a2bb700182b2ab4000bb9001902002b2ab4000cb9001a02002b2ab40004b9001902002ab4000d2bb8001ba7002e4d2cc1001c99000cbb001d592cb7001ebfbb0015592ab4000db6001fb60020b700214e2d2cb60017572dbfb1
 
 org/apache/geode/internal/cache/GridAdvisor$GridProfile,2
-fromData,26,2a2bb7001e2a2bb8001fb500052a2bb80020b500062ab60007b1
-toData,22,2a2bb7001b2ab400052bb8001c2ab400062bb8001db1
+fromData,26,2a2bb7001d2a2bb8001eb500052a2bb8001fb500062ab60007b1
+toData,22,2a2bb7001a2ab400052bb8001b2ab400062bb8001cb1
 
 org/apache/geode/internal/cache/HARegion$HARegionAdvisor$HAProfile,2
 fromData,47,2a2bb700032bb9000401003d2a1cb200057e99000704a7000403b500062a1cb200077e99000704a7000403b50008b1
@@ -1131,8 +1131,8 @@ fromData,30,2a2bb700122a2bb80013b500042a2bb80014b500072a2bb80015b50005b1
 toData,30,2a2bb700162ab400042bb800172ab400072bb800182ab400052bb80019b1
 
 org/apache/geode/internal/cache/MemberFunctionStreamingMessage,2
-fromData,163,2a2bb700522bb9005301003d1c047e9900142a2bb900540100b500082ab40008b800551c077e99000d2a2bb900540100b500051c057e99000e2a2bb80056c00057b500062bb800564e2dc1002b9900252a03b5000a2a2dc0002bb80058b500072ab40007c7001b2a2dc0002bb5001ba700102a2dc00059b500072a04b5000a2a2bb80056b500092a2bb80056c0005ab500102a1c10407e99000704a7000403b5000bb1
-toData,162,2a2bb7005c033d2ab400089900081c0480933d2ab40005029f00081c0780933d2ab40006c600081c0580933d2ab4000b9900091c104080933d2b1cb9005d02002ab4000899000d2b2ab40008b9005e02002ab40005029f000d2b2ab40005b9005e02002ab40006c6000b2ab400062bb8005f2ab4000a99000e2ab400072bb8005fa700102ab40007b9002001002bb8005f2ab400092bb8005f2ab400102bb8005fb1
+fromData,163,2a2bb700532bb9005401003d1c047e9900142a2bb900550100b500082ab40008b800561c077e99000d2a2bb900550100b500051c057e99000e2a2bb80057c00058b500062bb800574e2dc1002b9900252a03b5000a2a2dc0002bb80059b500072ab40007c7001b2a2dc0002bb5001ba700102a2dc0005ab500072a04b5000a2a2bb80057b500092a2bb80057c0005bb500102a1c10407e99000704a7000403b5000bb1
+toData,162,2a2bb7005d033d2ab400089900081c0480933d2ab40005029f00081c0780933d2ab40006c600081c0580933d2ab4000b9900091c104080933d2b1cb9005e02002ab4000899000d2b2ab40008b9005f02002ab40005029f000d2b2ab40005b9005f02002ab40006c6000b2ab400062bb800602ab4000a99000e2ab400072bb80060a700102ab40007b9002001002bb800602ab400092bb800602ab400102bb80060b1
 
 org/apache/geode/internal/cache/Node,2
 fromData,60,2abb001c59b7001db500052ab400052bb8001e2a2bb9001f0100b500032a2bb900200100b500042a2bb900210100b500152a2bb9001f0100b50006b1
@@ -1222,8 +1222,8 @@ fromData,52,2a2bb700242bb9002501003d1c047e99000704a70004033e1c057e99000704a70004
 toData,57,2a2bb70020033d2ab40003c600081c0480913d2ab40003c100219900081c0580913d2b1cb9002202002ab40003c6000b2ab400032bb80023b1
 
 org/apache/geode/internal/cache/RemoteOperationMessage,2
-fromData,43,2a2bb700552a2bb900560100b500572a2ab400572bb600582a2bb80059b5000a2a2bb9005a0100b50006b1
-toData,103,2a2bb7005b2ab6005c3d2b1cb9005d02002ab4000c99000d2b2ab4000cb9005e02002ab4005f99000d2b2ab4005fb9006002002ab60015029f000d2b2ab60015b9005e02002ab60016c6000b2ab600162bb800612ab4000a2bb800622b2ab40006b900630200b1
+fromData,43,2a2bb700562a2bb900570100b500582a2ab400582bb600592a2bb8005ab5000a2a2bb9005b0100b50006b1
+toData,103,2a2bb7005c2ab6005d3d2b1cb9005e02002ab4000c99000d2b2ab4000cb9005f02002ab4006099000d2b2ab40060b9006102002ab60015029f000d2b2ab60015b9005f02002ab60016c6000b2ab600162bb800622ab4000a2bb800632b2ab40006b900640200b1
 
 org/apache/geode/internal/cache/RemotePutAllMessage,2
 fromData,223,2a2bb700502a2bb80051c00052b500382a2bb80051b5003a2a2ab4005310087e99000704a7000403b500052ab4005310407e99000e2a2bb80051c00054b5004e2a2ab400531100807e99000704a7000403b500042a2bb8005588b500072a2ab40007bd0056b500062ab400079e00722bb800574dbb005859b700594e03360415042ab40007a200202ab400061504bb0056592b2ab4003815042c2db7005a53840401a7ffdd2bb9005b01003604150499002f2bb8005c3a0503360615062ab40007a2001d2ab4000615063219051506b6005dc0001fb50020840601a7ffe0b1
@@ -1298,8 +1298,8 @@ fromData,16,2a2bb7002d2a2bb9002e0100b50002b1
 toData,16,2a2bb7002b2b2ab40002b9002c0200b1
 
 org/apache/geode/internal/cache/StateFlushOperation$StateMarkerMessage,2
-fromData,62,2a2bb700552a2bb80056c00057b500182a2bb900580100b500022a2bb900580100b500032a2bb900590100b5001b2ab4001b9a000b2a2bb8005ab50008b1
-toData,59,2a2bb7004f2ab400182bb800502b2ab40002b9005102002b2ab40003b9005102002b2ab4001bb9005202002ab4001b9a000b2ab400082bb80053b1
+fromData,62,2a2bb7004f2a2bb80050c00051b500182a2bb900520100b500022a2bb900520100b500032a2bb900530100b5001b2ab4001b9a000b2a2bb80054b50008b1
+toData,59,2a2bb700492ab400182bb8004a2b2ab40002b9004b02002b2ab40003b9004b02002b2ab4001bb9004c02002ab4001b9a000b2ab400082bb8004db1
 
 org/apache/geode/internal/cache/StateFlushOperation$StateStabilizationMessage,2
 fromData,45,2a2bb700202a2bb900210100b500172a2bb80022b500192a2bb80023c00024b5001b2a2bb900250100b5001db1
@@ -1354,12 +1354,12 @@ fromData,9,2a2bb80009b50004b1
 toData,12,2ab40004c000072bb80008b1
 
 org/apache/geode/internal/cache/TXMessage,2
-fromData,37,2a2bb7004f2a2bb900500100b500052a2bb900500100b500032a2bb80051c00052b50002b1
-toData,34,2a2bb7004c2b2ab40005b9004d02002b2ab40003b9004d02002ab400022bb8004eb1
+fromData,37,2a2bb700502a2bb900510100b500052a2bb900510100b500032a2bb80052c00053b50002b1
+toData,34,2a2bb7004d2b2ab40005b9004e02002b2ab40003b9004e02002ab400022bb8004fb1
 
 org/apache/geode/internal/cache/TXRegionLockRequestImpl,2
-fromData,64,2a2bb8000cb500032a03b7000d4d2bb8000e3e2cc600181d9e00142a2c2ab40003b9000f0200c00010b500022a2a1d2bb70011b50004a700094e2a01b50004b1
-toData,17,2ab6001e2bb8001f2ab400042bb80020b1
+fromData,71,2a2bb8000db500042ab8000eb500022bb8000f3d2ab40002c6001b1c9e00172a2ab400022ab40004b900100200c00011b500032a2a1c2bb70012b50005a700094d2a01b50005b1
+toData,17,2ab6001f2bb800202ab400052bb80021b1
 
 org/apache/geode/internal/cache/TXRemoteCommitMessage$TXRemoteCommitReplyMessage,2
 fromData,17,2a2bb700192a2bb8001ac0001bb50004b1
@@ -1712,8 +1712,8 @@ fromData,49,2a2bb700362a2bb80037b500062a2bb900380100b80039b500082a2bb80037c0003a
 toData,43,2a2bb7003c2ab7000f2bb8003d2b2ab40008b4003eb9003f02002ab4000a2bb8003d2ab4000c2bb8003db1
 
 org/apache/geode/internal/cache/partitioned/PartitionMessage,2
-fromData,58,2a2bb7007b2a2bb9007c0100b500052a2ab400052bb6007d2a2bb9007e0100b5000e2bb8007fb20080b600819b000d2a2bb900820100b5000ab1
-toData,104,2a2bb70086033d2a1cb600873d2b1cb9008802002ab4001099000d2b2ab40010b9008902002ab40008029f000d2b2ab40008b9008902002ab40006c6000b2ab400062bb8008a2b2ab4000eb9008902002bb8008bb20080b600819b000d2b2ab4000ab9008c0200b1
+fromData,58,2a2bb7007d2a2bb9007e0100b500052a2ab400052bb6007f2a2bb900800100b5000e2bb80081b20082b600839b000d2a2bb900840100b5000ab1
+toData,104,2a2bb70088033d2a1cb600893d2b1cb9008a02002ab4001099000d2b2ab40010b9008b02002ab40008029f000d2b2ab40008b9008b02002ab40006c6000b2ab400062bb8008c2b2ab4000eb9008b02002bb8008db20082b600839b000d2b2ab4000ab9008e0200b1
 
 org/apache/geode/internal/cache/partitioned/PartitionedRegionFunctionStreamingMessage,2
 fromData,17,2a2bb7003c2a2bb8003dc0003eb50003b1
@@ -1796,8 +1796,8 @@ fromData,36,2a2bb700082a2bb900090100b500032a2bb9000a0100b500042a2bb9000a0100b500
 toData,36,2a2bb7000b2b2ab40003b9000c02002b2ab40004b9000d02002b2ab40005b9000d0200b1
 
 org/apache/geode/internal/cache/partitioned/SizeMessage,2
-fromData,14,2a2bb700362a2bb80037b50006b1
-toData,14,2a2bb700382ab400062bb80039b1
+fromData,14,2a2bb700382a2bb80039b50006b1
+toData,14,2a2bb7003a2ab400062bb8003bb1
 
 org/apache/geode/internal/cache/partitioned/SizeMessage$SizeReplyMessage,2
 fromData,17,2a2bb7001d2a2bb8001ec0001fb50003b1
@@ -1971,8 +1971,8 @@ toData,93,2ab700132ab400042bb8005e2ab40012c7000703a7000c2ab40012b9001501003d1c85
 fromData,71,2a2bb80061b500042bb80061883d2a2bb900620100b500191c9e002d2abb001a59b7001bb50012033e1d1ca2001b2bb800633a042ab400121904b90020020057840301a7ffe6b1
 
 org/apache/geode/internal/cache/versions/RegionVersionVector,2
-fromData,212,2a2a2bb600bfb500132bb900c001003d2a1c047e04a0000704a7000403b500992ab4000b2bb900c10100b600902ab4000c2bb900c10100b600902bb900c001003e03360415041da2002d2a2bb600bf3a05bb0016592bb700c23a0619061905b500882ab4001e19051906b6002b57840401a7ffd32bb900c00100360403360515051504a200442a2bb600bf3a062ab4001e1906b60039c000163a071907c6000d1907b40088c000693a062bb900c1010037082ab4001f19061608b80035b6002b57840501a7ffbb2abb0016592bb700c2b50018b1
-toData,239,2ab4001499000dbb00b25912bab700b4bf2a2ab400132bb600bb033d2ab400999900071c04803d2b1cb900bc02002b2ab4000bb6002eb900bd03002b2ab4000cb6002eb900bd03002b2ab4001eb6002cb900bc02002ab4001eb6009bb9002401004e2db90025010099002b2db900260100c000273a042a1904b900280100c000692bb600bb1904b9002901002bb800bea7ffd22b2ab4001fb6002cb900bc02002ab4001fb6009bb9002401004e2db9002501009900332db900260100c000273a042a1904b900280100c000692bb600bb2b1904b900290100c00037b60038b900bd0300a7ffca2ab400182bb800beb1
+fromData,212,2a2a2bb600c1b500122bb900c201003d2a1c047e04a0000704a7000403b500162ab4000a2bb900c30100b600152ab4000b2bb900c30100b600152bb900c201003e03360415041da2002d2a2bb600c13a05bb0021592bb700c43a0619061905b5008c2ab4001319051906b6002e57840401a7ffd32bb900c20100360403360515051504a200442a2bb600c13a062ab400131906b6003cc000213a071907c6000d1907b4008cc0006c3a062bb900c3010037082ab4001419061608b80038b6002e57840501a7ffbb2abb0021592bb700c4b50017b1
+toData,239,2ab4001f99000dbb00b45912bcb700b6bf2a2ab400122bb600bd033d2ab400169900071c04803d2b1cb900be02002b2ab4000ab60031b900bf03002b2ab4000bb60031b900bf03002b2ab40013b6002fb900be02002ab40013b6009db9002701004e2db90028010099002b2db900290100c0002a3a042a1904b9002b0100c0006c2bb600bd1904b9002c01002bb800c0a7ffd22b2ab40014b6002fb900be02002ab40014b6009db9002701004e2db9002801009900332db900290100c0002a3a042a1904b9002b0100c0006c2bb600bd2b1904b9002c0100c0003ab6003bb900bf0300a7ffca2ab400172bb800c0b1
 
 org/apache/geode/internal/cache/versions/VersionTag,2
 fromData,187,2bb9002101003db20014b20015b900160200990018b20014b2001512222ab600181cb80019b9001a0500b200232a2bb900210100b600242a2bb900250100b500101c077e9900132a2bb900260100121d7eb50006a7000f2a2bb900270100027eb500061c10107e99000d2a2bb900260100b5000a2a2bb900270100b5000d2a2bb80028b500071c047e99000c2a2a2bb60029b5000e1c057e99001e1c10087e99000e2a2ab4000eb5000fa7000c2a2a2bb60029b5000f2ab6002ab1
@@ -2069,10 +2069,10 @@ fromData,41,2a2bb80020b500062a2bb80020b500072a2bb80020b500082a2bb80021b500092a2b
 toData,41,2ab400062bb8001d2ab400072bb8001d2ab400082bb8001d2ab400092bb8001e2ab400022bb8001fb1
 
 org/apache/geode/management/internal/cli/functions/CliFunctionResult,4
-fromData,61,2a2bb80011b500052a2bb80012b500062a2bb80013c00014b500072a2bb80015c00016c00016b500032a2bb80013c00017b500082a2bb80018b50004b1
-fromDataPre_GFE_8_0_0_0,34,2a2bb80011b500052a2bb80013c00017b500082a2bb80015c00016c00016b50003b1
-toData,49,2ab400052bb8000c2ab400062bb8000d2ab400072bb8000e2ab400032bb8000f2ab400082bb8000e2ab400042bb80010b1
-toDataPre_GFE_8_0_0_0,25,2ab400052bb8000c2ab400032bb8000f2ab400082bb8000eb1
+fromData,61,2a2bb8001cb500052a2bb8001db500062a2bb8001ec0001fb500072a2bb80020c00021c00021b500032a2bb8001ec00022b500082a2bb80023b50004b1
+fromDataPre_GFE_8_0_0_0,34,2a2bb8001cb500052a2bb8001ec00022b500082a2bb80020c00021c00021b50003b1
+toData,49,2ab400052bb800172ab400062bb800182ab400072bb800192ab400032bb8001a2ab400082bb800192ab400042bb8001bb1
+toDataPre_GFE_8_0_0_0,25,2ab400052bb800172ab400032bb8001a2ab400082bb80019b1
 
 org/apache/geode/management/internal/configuration/domain/Configuration,2
 fromData,49,2a2bb80028b500022a2bb80028b500042a2bb80028b500032a2bb80028b500052a2bb80029b500082a2bb8002ab5000bb1
@@ -2083,10 +2083,10 @@ fromData,31,2a2bb80019b6001ab500022a2bb8001bb500032a2bb8001cc0001db50004b1
 toData,28,2ab40002b800152bb800162ab400032bb800172ab400042bb80018b1
 
 org/apache/geode/management/internal/configuration/domain/XmlEntity,4
-fromData,22,2a2bb6006b2a2bb8006cb500162a2bb8006cb50017b1
-fromDataPre_GEODE_1_1_1_0,59,2a2bb8006cb500062a2bb8006dc0006eb500032a2bb8006cb500022a2bb8006cb5001f2a2bb8006cb500052a2bb8006cb500042ab80011b50012b1
-toData,22,2a2bb600682ab400162bb800692ab400172bb80069b1
-toDataPre_GEODE_1_1_1_0,49,2ab400062bb800692ab400032bb8006a2ab400022bb800692ab4001f2bb800692ab400052bb800692ab400042bb80069b1
+fromData,22,2a2bb6006a2a2bb8006bb500162a2bb8006bb50017b1
+fromDataPre_GEODE_1_1_1_0,59,2a2bb8006bb500062a2bb8006cc0006db500032a2bb8006bb500022a2bb8006bb5001e2a2bb8006bb500052a2bb8006bb500042ab80011b50012b1
+toData,22,2a2bb600672ab400162bb800682ab400172bb80068b1
+toDataPre_GEODE_1_1_1_0,49,2ab400062bb800682ab400032bb800692ab400022bb800682ab4001e2bb800682ab400052bb800682ab400042bb80068b1
 
 org/apache/geode/management/internal/configuration/messages/ConfigurationResponse,2
 fromData,28,2a2bb8000bb500042a2bb8000bb500052a2bb8000cb6000db50006b1
@@ -2135,3 +2135,4 @@ toData,9,2ab400022bb80005b1
 org/apache/geode/redis/internal/DoubleWrapper,2
 fromData,9,2a2bb80004b50002b1
 toData,9,2ab400022bb80003b1
+

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