You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by la...@apache.org on 2017/08/25 21:49:30 UTC

geode git commit: GEODE-3522: create via load needs to add event to AEQ under same lock as update to local region

Repository: geode
Updated Branches:
  refs/heads/feature/GEODE-3522 [created] afbfd7081


GEODE-3522: create via load needs to add event to AEQ under same lock as update to local region

* create via load was invoking handleWANEvent prior to obtaining a lock, which allowed rebalance to
  shift the primary between AEQ update and data region update.
* moved call to handleWANEvent from DistributedRegion to BucketRegion to synchronize these two actions.


Project: http://git-wip-us.apache.org/repos/asf/geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode/commit/afbfd708
Tree: http://git-wip-us.apache.org/repos/asf/geode/tree/afbfd708
Diff: http://git-wip-us.apache.org/repos/asf/geode/diff/afbfd708

Branch: refs/heads/feature/GEODE-3522
Commit: afbfd708185cc7ed82490699c033c40263e4aa76
Parents: 92bf7fa
Author: Lynn Hughes-Godfrey <lh...@pivotal.io>
Authored: Fri Aug 25 14:46:17 2017 -0700
Committer: Lynn Hughes-Godfrey <lh...@pivotal.io>
Committed: Fri Aug 25 14:46:17 2017 -0700

----------------------------------------------------------------------
 .../geode/internal/cache/BucketRegion.java      |  3 ++
 .../geode/internal/cache/DistributedRegion.java |  6 ----
 .../LuceneIndexMaintenanceIntegrationTest.java  | 34 +++++++++++++++++++-
 3 files changed, 36 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/afbfd708/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
index 30ce9e7..465a3dd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
@@ -1304,6 +1304,9 @@ public class BucketRegion extends DistributedRegion implements Bucket {
       throws TimeoutException, CacheWriterException {
     beginLocalWrite(event);
     try {
+      if (getPartitionedRegion().isParallelWanEnabled()) {
+        handleWANEvent(event);
+      }
       event.setInvokePRCallbacks(true);
       forceSerialized(event);
       return super.basicPutEntry(event, lastModified);

http://git-wip-us.apache.org/repos/asf/geode/blob/afbfd708/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
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 d9ed4ed..d9cf1ed 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
@@ -2197,12 +2197,6 @@ public class DistributedRegion extends LocalRegion implements CacheDistributionA
     validateKey(event.getKey());
     // this next step also distributes the object to other processes, if necessary
     try {
-      // set the tail key so that the event is passed to GatewaySender queues.
-      // if the tailKey is not set, the event gets filtered out in ParallelGatewaySenderQueue
-      if (this instanceof BucketRegion) {
-        if (((BucketRegion) this).getPartitionedRegion().isParallelWanEnabled())
-          ((BucketRegion) this).handleWANEvent(event);
-      }
       re = basicPutEntry(event, lastModified);
 
       // Update client event with latest version tag from re.

http://git-wip-us.apache.org/repos/asf/geode/blob/afbfd708/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
index 2f39d78..ad28164 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -18,6 +18,7 @@ import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
 import java.io.Serializable;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -29,9 +30,11 @@ import org.awaitility.Awaitility;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.apache.geode.cache.CacheLoader;
+import org.apache.geode.cache.CacheLoaderException;
 import org.apache.geode.cache.ExpirationAction;
 import org.apache.geode.cache.ExpirationAttributes;
+import org.apache.geode.cache.LoaderHelper;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.internal.LuceneIndexForPartitionedRegion;
@@ -220,6 +223,25 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     verifySerializedValues(region);
   }
 
+  @Test
+  public void cacheLoadUpdatesIndex() throws InterruptedException, LuceneQueryException {
+    luceneService.createIndexFactory().setFields("title", "description").create(INDEX_NAME,
+        REGION_NAME);
+
+    Region region = this.cache.<String, TestObject>createRegionFactory(RegionShortcut.PARTITION)
+        .setCacheLoader(new TestCacheLoader()).create(REGION_NAME);
+
+    region.get("object-1");
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    assertTrue(luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, WAIT_FOR_FLUSH_TIME,
+        TimeUnit.MILLISECONDS));
+
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory().create(INDEX_NAME,
+        REGION_NAME, "hello", "description");
+    assertEquals(Collections.singletonList("object-1"), query.findKeys());
+  }
+
   private void populateRegion(Region region) {
     region.put("object-1", new TestObject("title 1", "hello world"));
     region.put("object-2", new TestObject("title 2", "this will not match"));
@@ -244,6 +266,16 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     Awaitility.await().atMost(30, TimeUnit.SECONDS).until(runnable);
   }
 
+  private static final class TestCacheLoader implements CacheLoader<String, TestObject> {
+    @Override
+    public void close() {}
+
+    @Override
+    public TestObject load(LoaderHelper<String, TestObject> helper) throws CacheLoaderException {
+      return new TestObject("title 1", "hello world");
+    }
+  }
+
   private static class TestObject implements Serializable {
 
     String title;