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/08/13 21:20:51 UTC

[geode] branch featrue/GEODE-5541 updated: Fix a review comment by checking pending callbacks in LocalRegion.

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

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


The following commit(s) were added to refs/heads/featrue/GEODE-5541 by this push:
     new c2fcdf0  Fix a review comment by checking pending callbacks in LocalRegion.
c2fcdf0 is described below

commit c2fcdf0b9eddbe59352d3fd0541623fee76677a5
Author: eshu <es...@pivotal.io>
AuthorDate: Mon Aug 13 14:17:09 2018 -0700

    Fix a review comment by checking pending callbacks in LocalRegion.
---
 .../apache/geode/internal/cache/LocalRegion.java   |  23 +++-
 .../geode/internal/cache/TXCommitMessage.java      |  22 +---
 .../apache/geode/internal/cache/TXEntryState.java  |  13 +--
 .../geode/internal/cache/LocalRegionTest.java      | 118 +++++++++++++++++++++
 .../geode/internal/cache/RegionCommitTest.java     |  51 ---------
 5 files changed, 143 insertions(+), 84 deletions(-)

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 1642a8a..b7f8500 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
@@ -6873,7 +6873,10 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
     try {
       final boolean inTokenMode = needTokensForGII || needRIDestroyToken;
-      this.entries.txApplyDestroy(key, rmtOrigin, event, inTokenMode, needRIDestroyToken, op,
+      if (!needsPendingCallbacksForDestroy(key, txEntryState)) {
+        pendingCallbacks = new ArrayList<>();
+      }
+      getRegionMap().txApplyDestroy(key, rmtOrigin, event, inTokenMode, needRIDestroyToken, op,
           eventId, aCallbackArgument, pendingCallbacks, filterRoutingInfo, bridgeContext,
           isOriginRemote, txEntryState, versionTag, tailKey);
     } finally {
@@ -6883,6 +6886,24 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     }
   }
 
+  boolean needsPendingCallbacksForDestroy(Object key, TXEntryState txEntryState) {
+    if (isProxy()) {
+      return true;
+    }
+    if (txEntryState != null) {
+      // nearside on tx host
+      // a destroy on destroyed or removed token, to local committed state, becomes a noop
+      // since nothing needed to be done locally.
+      // We don't want to actually do the destroy since we told the
+      // transaction listener that no destroy was done.
+      Object originalValue = txEntryState.getOriginalValue();
+      return originalValue != null && !Token.isRemoved(originalValue);
+    }
+    // farside without TXEntryState
+    RegionEntry regionEntry = basicGetEntry(key);
+    return regionEntry != null && !regionEntry.isDestroyedOrRemoved();
+  }
+
   /**
    * Called by lower levels, while still holding the write sync lock, and the low level has
    * completed its part of the basic destroy
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXCommitMessage.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXCommitMessage.java
index bea1930..13f15ea 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXCommitMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXCommitMessage.java
@@ -1297,13 +1297,9 @@ public class TXCommitMessage extends PooledDistributionMessage
         entryOp.versionTag.replaceNullIDs(this.msg.getSender());
       }
       if (entryOp.op.isDestroy()) {
-        boolean invokeCallbacks =
-            isOpDestroyEvent(internalRegion, entryOp);
-        List<EntryEventImpl> whichPendingCallbacks =
-            invokeCallbacks ? pendingCallbacks : new ArrayList<EntryEventImpl>();
         this.internalRegion.txApplyDestroy(entryOp.key, this.msg.txIdent, this.txEvent,
             this.needsUnlock,
-            entryOp.op, getEventId(entryOp), entryOp.callbackArg, whichPendingCallbacks,
+            entryOp.op, getEventId(entryOp), entryOp.callbackArg, pendingCallbacks,
             entryOp.filterRoutingInfo, this.msg.bridgeContext, false /* origin remote */,
             null/* txEntryState */, entryOp.versionTag, entryOp.tailKey);
       } else if (entryOp.op.isInvalidate()) {
@@ -1320,14 +1316,6 @@ public class TXCommitMessage extends PooledDistributionMessage
       }
     }
 
-    boolean isOpDestroyEvent(InternalRegion internalRegion, FarSideEntryOp entryOp) {
-      // Note that if the region is a proxy(empty) then we go ahead and add
-      // the destroy callback because we may need to forward the event to clients.
-      RegionEntry regionEntry = internalRegion.basicGetEntry(entryOp.key);
-      return entryOp.op.isDestroy() && (internalRegion.isProxy()
-          || (regionEntry != null && !Token.isRemoved(regionEntry.getValue())));
-    }
-
     /**
      * Apply a single tx entry op on the far side
      */
@@ -1543,14 +1531,6 @@ public class TXCommitMessage extends PooledDistributionMessage
        */
       public FarSideEntryOp() {}
 
-      public void setOp(Operation op) {
-        this.op = op;
-      }
-
-      public void setKey(Object key) {
-        this.key = key;
-      }
-
       /**
        * Creates and returns a new instance of a tx entry op on the far side. The "toData" that this
        * should match is {@link TXEntryState#toFarSideData}.
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXEntryState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXEntryState.java
index 72a1b2d..d95877d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXEntryState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXEntryState.java
@@ -19,8 +19,6 @@ import static org.apache.geode.internal.offheap.annotations.OffHeapIdentifier.TX
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
@@ -1554,21 +1552,14 @@ public class TXEntryState implements Releasable {
   }
 
   private void txApplyDestroyLocally(InternalRegion r, Object key, TXState txState) {
-    boolean invokeCallbacks = isOpDestroyEvent(r);
-    List<EntryEventImpl> pendingCallbacks =
-        invokeCallbacks ? txState.getPendingCallbacks() : new ArrayList<EntryEventImpl>();
     try {
       r.txApplyDestroy(key, txState.getTransactionId(), null, false/* inTokenMode */,
-          getDestroyOperation(), getNearSideEventId(txState), callBackArgument, pendingCallbacks,
+          getDestroyOperation(), getNearSideEventId(txState), callBackArgument,
+          txState.getPendingCallbacks(),
           getFilterRoutingInfo(), txState.bridgeContext, false, this, null, -1);
     } catch (RegionDestroyedException ignore) {
     } catch (EntryDestroyedException ignore) {
     }
-    // if !isOpDestroyEvent then
-    // this destroy, to local committed state, becomes a noop
-    // since nothing needed to be done locally.
-    // We don't want to actually do the destroy since we told the
-    // transaction listener that no destroy was done.
   }
 
   private void txApplyInvalidateLocally(InternalRegion r, Object key, Object newValue,
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
new file mode 100644
index 0000000..72cc213
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.junit.Test;
+
+
+public class LocalRegionTest {
+
+  private final Object key = new Object();
+
+  @Test
+  public void proxyRegionNeedsPendingCallbacksForDestroy() {
+    TXEntryState txEntryState = mock(TXEntryState.class);
+    LocalRegion region = mock(LocalRegion.class);
+    when(region.isProxy()).thenReturn(true);
+    when(region.needsPendingCallbacksForDestroy(key, txEntryState)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, txEntryState)).isTrue();
+  }
+
+  @Test
+  public void destroyNullOriginalValueOnTransactionHostDoesNotNeedPendingCallbacksForDestroy() {
+    TXEntryState txEntryState = mock(TXEntryState.class);
+    LocalRegion region = mock(LocalRegion.class);
+    when(region.isProxy()).thenReturn(false);
+    when(txEntryState.getOriginalValue()).thenReturn(null);
+    when(region.needsPendingCallbacksForDestroy(key, txEntryState)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, txEntryState)).isFalse();
+  }
+
+  @Test
+  public void destroyDestroyedTokenOnTransactionHostDoesNotNeedPendingCallbacksForDestroy() {
+    TXEntryState txEntryState = mock(TXEntryState.class);
+    LocalRegion region = mock(LocalRegion.class);
+    when(region.isProxy()).thenReturn(false);
+    when(txEntryState.getOriginalValue()).thenReturn(Token.DESTROYED);
+    when(region.needsPendingCallbacksForDestroy(key, txEntryState)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, txEntryState)).isFalse();
+  }
+
+  @Test
+  public void destroyTombstoneOnTransactionHostDoesNotNeedPendingCallbacksForDestroy() {
+    TXEntryState txEntryState = mock(TXEntryState.class);
+    LocalRegion region = mock(LocalRegion.class);
+    when(region.isProxy()).thenReturn(false);
+    when(txEntryState.getOriginalValue()).thenReturn(Token.TOMBSTONE);
+    when(region.needsPendingCallbacksForDestroy(key, txEntryState)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, txEntryState)).isFalse();
+  }
+
+  @Test
+  public void destroyNotATokenOnTransactionHostNeedsPendingCallbacksForDestroy() {
+    TXEntryState txEntryState = mock(TXEntryState.class);
+    LocalRegion region = mock(LocalRegion.class);
+    when(region.isProxy()).thenReturn(false);
+    when(txEntryState.getOriginalValue()).thenReturn(new Token.NotAToken());
+    when(region.needsPendingCallbacksForDestroy(key, txEntryState)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, txEntryState)).isTrue();
+  }
+
+  @Test
+  public void destroyNonExistingRegionEntryOnRemoteHostDoesNotNeedPendingCallbacksForDestroy() {
+    LocalRegion region = mock(LocalRegion.class);
+    when(region.basicGetEntry(key)).thenReturn(null);
+    when(region.isProxy()).thenReturn(false);
+    when(region.needsPendingCallbacksForDestroy(key, null)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, null)).isFalse();
+  }
+
+  @Test
+  public void destroyRemovedTokenOnRemoteHostDoesNotNeedPendingCallbacksForDestroy() {
+    LocalRegion region = mock(LocalRegion.class);
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(region.isProxy()).thenReturn(false);
+    when(region.isProxy()).thenReturn(false);
+    when(region.basicGetEntry(key)).thenReturn(regionEntry);
+    when(regionEntry.isDestroyedOrRemoved()).thenReturn(true);
+    when(region.needsPendingCallbacksForDestroy(key, null)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, null)).isFalse();
+  }
+
+  @Test
+  public void destroyAValueOnRemoteHostNeedsPendingCallbacksForDestroy() {
+    LocalRegion region = mock(LocalRegion.class);
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(region.isProxy()).thenReturn(false);
+    when(region.isProxy()).thenReturn(false);
+    when(region.basicGetEntry(key)).thenReturn(regionEntry);
+    when(regionEntry.isDestroyedOrRemoved()).thenReturn(false);
+    when(region.needsPendingCallbacksForDestroy(key, null)).thenCallRealMethod();
+
+    assertThat(region.needsPendingCallbacksForDestroy(key, null)).isTrue();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionCommitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionCommitTest.java
index c43e076..39218ad 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionCommitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionCommitTest.java
@@ -21,7 +21,6 @@ import static org.mockito.Mockito.when;
 import org.junit.Before;
 import org.junit.Test;
 
-import org.apache.geode.cache.Operation;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.cache.TXCommitMessage.RegionCommit;
@@ -34,7 +33,6 @@ public class RegionCommitTest {
   private TXCommitMessage txCommitMessage;
   private RegionCommit regionCommit;
   private final Object key = new Object();
-  private RegionCommit.FarSideEntryOp entryOp;
 
   @Before
   public void setUp() {
@@ -46,9 +44,6 @@ public class RegionCommitTest {
     region = mock(LocalRegion.class);
     txCommitMessage = mock(TXCommitMessage.class);
     regionCommit = new RegionCommit(txCommitMessage);
-    entryOp = regionCommit.new FarSideEntryOp();
-    entryOp.setOp(Operation.DESTROY);
-    entryOp.setKey(key);
 
     when(dm.getCache()).thenReturn(cache);
     when(cache.getRegionByPath(path)).thenReturn(region);
@@ -59,50 +54,4 @@ public class RegionCommitTest {
   public void getsRegionFromCacheFromDM() {
     assertThat(regionCommit.getRegionByPath(dm, path)).isEqualTo(region);
   }
-
-  @Test
-  public void isOpDestroyedEventReturnsFalseIfNotDestroyOperation() {
-    entryOp.setOp(Operation.UPDATE);
-
-    assertThat(regionCommit.isOpDestroyEvent(mock(InternalRegion.class), entryOp)).isFalse();
-  }
-
-  @Test
-  public void isOpDestroyedEventReturnsFalseIfIsDestroyOperationAndRegionEntryToBeDestroyedIsNull() {
-    when(region.basicGetEntry(key)).thenReturn(null);
-
-    assertThat(regionCommit.isOpDestroyEvent(region, entryOp)).isFalse();
-  }
-
-  @Test
-  public void isOpDestroyedEventReturnsFalseIfIsDestroyOperationAndRegionEntryToBeDestroyedIsRemovedToken() {
-    RegionEntry regionEntry = mock(RegionEntry.class);
-
-    assertThat(regionCommit.isOpDestroyEvent(region, entryOp)).isFalse();
-  }
-
-  @Test
-  public void isOpDestroyedEventReturnsFalseIfIsDestroyOperationAndRegionEntryToBeDestroyedIsTombstone() {
-    RegionEntry regionEntry = mock(RegionEntry.class);
-    when(region.basicGetEntry(key)).thenReturn(regionEntry);
-    when(regionEntry.getValue()).thenReturn(Token.TOMBSTONE);
-
-    assertThat(regionCommit.isOpDestroyEvent(region, entryOp)).isFalse();
-  }
-
-  @Test
-  public void isOpDestroyedEventReturnsTrueIfDestroyEntryOnEmptyRegion() {
-    when(region.isProxy()).thenReturn(true);
-
-    assertThat(regionCommit.isOpDestroyEvent(region, entryOp)).isTrue();
-  }
-
-  @Test
-  public void isOpDestroyedEventReturnsTrueIfIsDestroyOperationAndRegionEntryIsNotAToken() {
-    RegionEntry regionEntry = mock(RegionEntry.class);
-    when(region.basicGetEntry(key)).thenReturn(regionEntry);
-    when(regionEntry.getValue()).thenReturn(new Token.NotAToken());
-
-    assertThat(regionCommit.isOpDestroyEvent(region, entryOp)).isTrue();
-  }
 }