You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2019/04/12 20:21:18 UTC

[geode] 01/01: GEODE-6195 putIfAbsent may get a returned value caused by the same operation due to retry

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

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

commit 9f85d83e031ad8a46267c014baea53bd8970bea0
Author: Bruce Schuchardt <bs...@pivotal.io>
AuthorDate: Fri Apr 12 13:15:03 2019 -0700

    GEODE-6195 putIfAbsent may get a returned value caused by the same operation due to retry
    
    Moving the check for a retried putIfAbsent to be under the RegionEntry
    lock.  This allows the operation to propagate throughout the cluster and
    allows the client to receive a valid version tag, if concurrency checks
    are enabled.
    
    This also fixes GEODE-6630, "RedundancyLogger might throw NPE" because
    tests were frequently running into that issue.
---
 .../cache/RetryPutIfAbsentIntegrationTest.java     |  87 ++++++++++++
 .../apache/geode/internal/cache/LocalRegion.java   |  62 +--------
 .../internal/cache/PRHARedundancyProvider.java     |   3 +-
 .../cache/event/DistributedEventTracker.java       |   7 +
 .../geode/internal/cache/event/EventTracker.java   |   5 +
 .../cache/event/NonDistributedEventTracker.java    |   5 +
 .../geode/internal/cache/map/RegionMapPut.java     |  37 +++++
 .../geode/internal/cache/LocalRegionTest.java      | 150 ---------------------
 .../geode/internal/cache/map/RegionMapPutTest.java |  21 +++
 9 files changed, 167 insertions(+), 210 deletions(-)

diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/RetryPutIfAbsentIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/RetryPutIfAbsentIntegrationTest.java
new file mode 100644
index 0000000..f79c2d2
--- /dev/null
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/RetryPutIfAbsentIntegrationTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cache;
+
+import static org.apache.geode.internal.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.geode.cache.query.CacheUtils;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.EventID;
+import org.apache.geode.internal.cache.EventIDHolder;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.VMCachedDeserializable;
+import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
+
+public class RetryPutIfAbsentIntegrationTest {
+
+  Cache cache;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @Test
+  public void duplicatePutIfAbsentIsAccepted() {
+    final String key = "mykey";
+    final String value = "myvalue";
+
+    LocalRegion myregion =
+        (LocalRegion) CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE)
+            .setConcurrencyChecksEnabled(true).create("myregion");
+
+    ClientProxyMembershipID id =
+        new ClientProxyMembershipID(new InternalDistributedMember("localhost", 1));
+    EventIDHolder clientEvent = new EventIDHolder(new EventID(new byte[] {1, 2, 3, 4, 5}, 1, 1));
+    clientEvent.setRegion(myregion);
+    byte[] valueBytes = new VMCachedDeserializable("myvalue", 7).getSerializedValue();
+    System.out.println("first putIfAbsent");
+    Object oldValue =
+        myregion.basicBridgePutIfAbsent("mykey", valueBytes, true, null, id, true, clientEvent);
+    assertEquals(null, oldValue);
+    assertTrue(myregion.containsKey(key));
+
+    myregion.getEventTracker().clear();
+
+    clientEvent = new EventIDHolder(new EventID(new byte[] {1, 2, 3, 4, 5}, 1, 1));
+    clientEvent.setRegion(myregion);
+    clientEvent.setPossibleDuplicate(true);
+    clientEvent.setOperation(Operation.PUT_IF_ABSENT);
+    assertFalse(myregion.getEventTracker().hasSeenEvent(clientEvent));
+
+    System.out.println("second putIfAbsent");
+    oldValue =
+        myregion.basicBridgePutIfAbsent("mykey", valueBytes, true, null, id, true, clientEvent);
+    assertEquals(null, oldValue);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    CacheUtils.startCache();
+    cache = CacheUtils.getCache();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    CacheUtils.closeCache();
+  }
+
+}
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 2dae4a2..c25d614 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
@@ -3128,44 +3128,8 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   @VisibleForTesting
   void checkPutIfAbsentResult(EntryEventImpl event, Object value, Object result) {
     if (result != null) {
-      // we may see a non null result possibly due to retry
-      if (event.hasRetried() && putIfAbsentResultHasSameValue(true, value, result)) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("retried putIfAbsent and result is the value to be put,"
-              + " treat as a successful putIfAbsent");
-        }
-      } else {
-        // customers don't see this exception
-        throw new EntryNotFoundException("entry existed for putIfAbsent");
-      }
-    }
-  }
-
-  @VisibleForTesting
-  boolean putIfAbsentResultHasSameValue(boolean isClient, Object valueToBePut, Object result) {
-    if (Token.isInvalid(result) || result == null) {
-      return valueToBePut == null;
+      throw new EntryNotFoundException("entry existed for putIfAbsent");
     }
-
-    boolean isCompressedOffHeap =
-        isClient ? false : getAttributes().getOffHeap() && getAttributes().getCompressor() != null;
-    return ValueComparisonHelper.checkEquals(valueToBePut, result, isCompressedOffHeap, getCache());
-  }
-
-  @VisibleForTesting
-  boolean bridgePutIfAbsentResultHasSameValue(byte[] valueToBePut, boolean isValueToBePutObject,
-      Object result) {
-    if (Token.isInvalid(result) || result == null) {
-      return valueToBePut == null;
-    }
-
-    boolean isCompressedOffHeap =
-        getAttributes().getOffHeap() && getAttributes().getCompressor() != null;
-    if (isValueToBePutObject) {
-      return ValueComparisonHelper.checkEquals(EntryEventImpl.deserialize(valueToBePut), result,
-          isCompressedOffHeap, getCache());
-    }
-    return ValueComparisonHelper.checkEquals(valueToBePut, result, isCompressedOffHeap, getCache());
   }
 
   /**
@@ -10623,15 +10587,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
         final boolean ifOld = false;
         final boolean requireOldValue = true;
         if (!basicPut(event, ifNew, ifOld, oldValue, requireOldValue)) {
-          Object result = event.getOldValue();
-          if (event.isPossibleDuplicate() && putIfAbsentResultHasSameValue(false, value, result)) {
-            if (logger.isDebugEnabled()) {
-              logger.debug("possible duplicate putIfAbsent event and result is the value to be put,"
-                  + " treat this as a successful putIfAbsent");
-            }
-            return null;
-          }
-          return result;
+          return event.getOldValue();
         } else {
           if (!getDataView().isDeferredStats()) {
             getCachePerfStats().endPut(startPut, false);
@@ -10832,6 +10788,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
       // if this is a replayed operation we may already have a version tag
       event.setVersionTag(clientEvent.getVersionTag());
+      event.setPossibleDuplicate(clientEvent.isPossibleDuplicate());
 
       // Set the new value to the input byte[] if it isn't null
       if (value != null) {
@@ -10870,19 +10827,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
         clientEvent.setVersionTag(event.getVersionTag());
         clientEvent.isConcurrencyConflict(event.isConcurrencyConflict());
       } else {
-        if (value != null) {
-          assert (value instanceof byte[]);
-        }
-        if (event.isPossibleDuplicate()
-            && bridgePutIfAbsentResultHasSameValue((byte[]) value, isObject, oldValue)) {
-          // result is possibly due to the retry
-          if (logger.isDebugEnabled()) {
-            logger.debug("retried putIfAbsent and got oldValue as the value to be put,"
-                + " treat this as a successful putIfAbsent");
-          }
-          return null;
-        }
-
         if (oldValue == null) {
           // fix for 42189, putIfAbsent on server can return null if the
           // operation was not performed (oldValue in cache was null).
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
index 2942154..7bf76b7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
@@ -1708,12 +1708,13 @@ public class PRHARedundancyProvider {
         new ArrayList<ProxyBucketRegion>(proxyBucketArray.length);
 
 
+    allBucketsRecoveredFromDisk = new CountDownLatch(proxyBucketArray.length);
+
     /*
      * Start the redundancy logger before recovering any proxy buckets.
      */
     startRedundancyLogger(proxyBucketArray.length);
 
-    allBucketsRecoveredFromDisk = new CountDownLatch(proxyBucketArray.length);
     /*
      * Spawn a separate thread for bucket that we previously hosted to recover that bucket.
      *
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/event/DistributedEventTracker.java b/geode-core/src/main/java/org/apache/geode/internal/cache/event/DistributedEventTracker.java
index 6301183..e4f9711 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/event/DistributedEventTracker.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/event/DistributedEventTracker.java
@@ -128,6 +128,13 @@ public class DistributedEventTracker implements EventTracker {
   }
 
   @Override
+  public void clear() {
+    recordedEvents.clear();
+    recordedBulkOps.clear();
+    recordedBulkOpVersionTags.clear();
+  }
+
+  @Override
   public Map<ThreadIdentifier, EventSequenceNumberHolder> getState() {
     Map<ThreadIdentifier, EventSequenceNumberHolder> result = new HashMap<>(recordedEvents.size());
     for (Map.Entry<ThreadIdentifier, EventSequenceNumberHolder> entry : recordedEvents.entrySet()) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/event/EventTracker.java b/geode-core/src/main/java/org/apache/geode/internal/cache/event/EventTracker.java
index 43a0458..e947c4b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/event/EventTracker.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/event/EventTracker.java
@@ -133,4 +133,9 @@ public interface EventTracker {
    */
   boolean isInitialImageProvider(DistributedMember mbr);
 
+  /**
+   * clear the tracker
+   */
+  void clear();
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/event/NonDistributedEventTracker.java b/geode-core/src/main/java/org/apache/geode/internal/cache/event/NonDistributedEventTracker.java
index f291c30..67d75e3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/event/NonDistributedEventTracker.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/event/NonDistributedEventTracker.java
@@ -50,6 +50,11 @@ public class NonDistributedEventTracker implements EventTracker {
   }
 
   @Override
+  public void clear() {
+    // nothing to clear
+  }
+
+  @Override
   public Map<ThreadIdentifier, EventSequenceNumberHolder> getState() {
     return null;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java b/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java
index 1b419f0..4554590 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java
@@ -18,6 +18,8 @@ package org.apache.geode.internal.cache.map;
 
 import java.util.Set;
 
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.cache.CacheWriter;
 import org.apache.geode.cache.DiskAccessException;
 import org.apache.geode.cache.Operation;
@@ -27,10 +29,12 @@ import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionClearedException;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.Token;
+import org.apache.geode.internal.cache.ValueComparisonHelper;
 import org.apache.geode.internal.cache.entries.AbstractRegionEntry;
 import org.apache.geode.internal.cache.versions.ConcurrentCacheModificationException;
 import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl;
+import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.OffHeapHelper;
 import org.apache.geode.internal.offheap.ReferenceCountHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -38,11 +42,14 @@ import org.apache.geode.internal.offheap.annotations.Unretained;
 import org.apache.geode.internal.sequencelog.EntryLogger;
 
 public class RegionMapPut extends AbstractRegionMapPut {
+  protected static final Logger logger = LogService.getLogger();
+
   private final CacheModificationLock cacheModificationLock;
   private final EntryEventSerialization entryEventSerialization;
   private final boolean ifNew;
   private final boolean ifOld;
   private final boolean overwriteDestroyed;
+  private boolean overwritePutIfAbsent;
   private final boolean requireOldValue;
   private final boolean retrieveOldValueForDelta;
   private final boolean replaceOnClient;
@@ -104,6 +111,10 @@ public class RegionMapPut extends AbstractRegionMapPut {
     return replaceOnClient;
   }
 
+  boolean isOverwritePutIfAbsent() {
+    return overwritePutIfAbsent;
+  }
+
   boolean isCacheWrite() {
     return cacheWrite;
   }
@@ -214,6 +225,9 @@ public class RegionMapPut extends AbstractRegionMapPut {
   protected void unsetOldValueForDelta() {
     OffHeapHelper.release(getOldValueForDelta());
     setOldValueForDelta(null);
+    if (this.overwritePutIfAbsent) {
+      getEvent().setOldValue(null);
+    }
   }
 
   @Override
@@ -396,12 +410,35 @@ public class RegionMapPut extends AbstractRegionMapPut {
   private boolean checkCreatePreconditions() {
     if (isIfNew()) {
       if (!getRegionEntry().isDestroyedOrRemoved()) {
+        // retain the version stamp of the existing entry for use in processing failures
+        EntryEventImpl event = getEvent();
+        if (event.getRegion().getConcurrencyChecksEnabled() &&
+            event.getOperation() == Operation.PUT_IF_ABSENT &&
+            !event.hasValidVersionTag() &&
+            event.isPossibleDuplicate()) {
+          if (ValueComparisonHelper.checkEquals(getRegionEntry().getValue(),
+              getEvent().basicGetNewValue(),
+              isCompressedOffHeap(event), event.getRegion().getCache())) {
+            logger.info("retried putIfAbsent found same value already in cache "
+                + "- allowing the operation.  entry={}; event={}", getRegionEntry(), getEvent());
+            event.setPossibleDuplicate(true);
+            this.overwritePutIfAbsent = true;
+            return true;
+          }
+        }
         return false;
       }
     }
     return true;
   }
 
+
+  private boolean isCompressedOffHeap(EntryEventImpl event) {
+    return event.getContext() != null ? false
+        : event.getRegion().getAttributes().getOffHeap()
+            && event.getRegion().getAttributes().getCompressor() != null;
+  }
+
   private boolean checkExpectedOldValuePrecondition() {
     // replace is propagated to server, so no need to check
     // satisfiesOldValue on client
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
index ba4096b..8a4e126 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.internal.cache;
 
-import static org.assertj.core.api.Java6Assertions.assertThat;
 import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
@@ -91,153 +90,4 @@ public class LocalRegionTest {
     region.checkPutIfAbsentResult(event, value, result);
   }
 
-  @Test(expected = EntryNotFoundException.class)
-  public void checkPutIfAbsentResultThrowsIfEventHasRetriedButResultNotHaveSameValue() {
-    Object result = new Object();
-    when(event.hasRetried()).thenReturn(true);
-    when(region.putIfAbsentResultHasSameValue(true, value, result)).thenReturn(false);
-    doCallRealMethod().when(region).checkPutIfAbsentResult(event, value, result);
-
-    region.checkPutIfAbsentResult(event, value, result);
-  }
-
-  @Test
-  public void checkPutIfAbsentResultSucceedsIfEventHasRetriedAndResultHasSameValue() {
-    Object result = new Object();
-    when(event.hasRetried()).thenReturn(true);
-    when(region.putIfAbsentResultHasSameValue(true, value, result)).thenReturn(true);
-    doCallRealMethod().when(region).checkPutIfAbsentResult(event, value, result);
-
-    region.checkPutIfAbsentResult(event, value, result);
-
-    verify(event).hasRetried();
-    verify(region).putIfAbsentResultHasSameValue(true, value, result);
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueReturnTrueIfResultIsInvalidTokenAndValueToBePutIsNull() {
-    when(region.putIfAbsentResultHasSameValue(true, null, Token.INVALID)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(true, null, Token.INVALID)).isTrue();
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueReturnFalseIfResultIsInvalidTokenAndValueToBePutIsNotNull() {
-    when(region.putIfAbsentResultHasSameValue(true, value, Token.INVALID)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(true, value, Token.INVALID)).isFalse();
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueReturnTrueIfResultHasSameValue() {
-    Object result = "value";
-    when(region.putIfAbsentResultHasSameValue(true, value, result)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(true, value, result)).isTrue();
-    verify(region, never()).getAttributes();
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueReturnFalseIfResultDoesNotHaveSameValue() {
-    Object result = "differentValue";
-    when(region.putIfAbsentResultHasSameValue(true, value, result)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(true, value, result)).isFalse();
-    verify(region, never()).getAttributes();
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueChecksRegionAttributesIfNotFromClient() {
-    Object result = "value";
-    when(region.putIfAbsentResultHasSameValue(false, value, result)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(false, value, result)).isTrue();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueReturnFalseIfResultDoesNotHaveSameValueAndNotFromClient() {
-    Object oldValue = "differentValue";
-    Object result = new VMCachedDeserializable(EntryEventImpl.serialize(oldValue));
-    when(region.putIfAbsentResultHasSameValue(false, value, result)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(false, value, result)).isFalse();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void putIfAbsentResultHasSameValueReturnTrueIfResultHasSameValueAndNotFromClient() {
-    Object result = new VMCachedDeserializable(EntryEventImpl.serialize(value));
-    when(region.putIfAbsentResultHasSameValue(false, value, result)).thenCallRealMethod();
-
-    assertThat(region.putIfAbsentResultHasSameValue(false, value, result)).isTrue();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void bridgePutIfAbsentResultHasSameValueCanCheckValueForObject() {
-    Object result = "value";
-    byte[] valueToBePut = EntryEventImpl.serialize(value);
-    when(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result))
-        .thenCallRealMethod();
-
-    assertThat(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result)).isTrue();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void bridgePutIfAbsentResultHasSameValueCanCheckValueForNonObjectByteArray() {
-    byte[] valueToBePut = {0, 1, 2, 3};
-    Object result = new VMCachedDeserializable(EntryEventImpl.serialize(valueToBePut));
-    when(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, false, result))
-        .thenCallRealMethod();
-
-    assertThat(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, false, result)).isTrue();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void bridgePutIfAbsentResultHasSameValueCanCheckValueForIntArray() {
-    int[] newValue = {0, 1, 2, 3};
-    byte[] valueToBePut = EntryEventImpl.serialize(newValue);
-    Object result = newValue;
-    when(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result))
-        .thenCallRealMethod();
-
-    assertThat(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result)).isTrue();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void bridgePutIfAbsentResultHasSameValueCanCheckValueForArrayOfArray() {
-    String[] array1 = {"0", "1", "2"};
-    String[] array2 = {"3", "4", "5"};
-    String[] array3 = {"7"};
-    String[][] newValue = {array1, array2, array3};
-    byte[] valueToBePut = EntryEventImpl.serialize(newValue);
-    Object result = new VMCachedDeserializable(EntryEventImpl.serialize(newValue));
-    when(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result))
-        .thenCallRealMethod();
-
-    assertThat(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result)).isTrue();
-    verify(region).getAttributes();
-  }
-
-  @Test
-  public void bridgePutIfAbsentResultHasSameValueCanCheckDifferentValuesForArrayOfArray() {
-    String[] array1 = {"0", "1", "2"};
-    String[] array2 = {"3", "4", "5"};
-    String[] array3 = {"7"};
-    String[] array4 = {"8"};
-    String[][] newValue = {array1, array2, array3};
-    String[][] returnedValue = {array1, array2, array4};
-    byte[] valueToBePut = EntryEventImpl.serialize(newValue);
-    Object result = new VMCachedDeserializable(EntryEventImpl.serialize(returnedValue));
-    when(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result))
-        .thenCallRealMethod();
-
-    assertThat(region.bridgePutIfAbsentResultHasSameValue(valueToBePut, true, result)).isFalse();
-    verify(region).getAttributes();
-  }
-
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java
index 0aab7c5..1e76598 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java
@@ -134,6 +134,27 @@ public class RegionMapPutTest {
   }
 
   @Test
+  public void doesNotSetEventOldValueIfRetriedPutIfAbsentOperation() {
+    final byte[] bytes = new byte[] {1, 2, 3, 4, 5};
+    givenExistingRegionEntry();
+    when(existingRegionEntry.getValue()).thenReturn(bytes);
+    when(internalRegion.getConcurrencyChecksEnabled()).thenReturn(true);
+    givenPutIfAbsentOperation(bytes); // duplicate operation
+    doPut();
+    verify(event).setOldValue(null, true);
+    assertThat(instance.isOverwritePutIfAbsent()).isTrue();
+  }
+
+  private void givenPutIfAbsentOperation(byte[] bytes) {
+    when(event.isPossibleDuplicate()).thenReturn(true);
+    when(event.basicGetNewValue()).thenReturn(bytes);
+    when(event.getOperation()).thenReturn(Operation.PUT_IF_ABSENT);
+    when(event.hasValidVersionTag()).thenReturn(false);
+    ifNew = true;
+    ifOld = false;
+  }
+
+  @Test
   public void setsEventOldValueToExistingRegionEntryValue_ifOperationGuaranteesOldValue() {
     givenExistingRegionEntry();
     givenAnOperationThatGuaranteesOldValue();