You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2021/04/09 23:09:55 UTC

[geode] branch feature/GEODE-7674 created (now 9e80b12)

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

zhouxj pushed a change to branch feature/GEODE-7674
in repository https://gitbox.apache.org/repos/asf/geode.git.


      at 9e80b12  feature/GEODE-7674: Clear on PR with lucene index should throw exception

This branch includes the following new commits:

     new 9e80b12  feature/GEODE-7674: Clear on PR with lucene index should throw exception

The 1 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.


[geode] 01/01: feature/GEODE-7674: Clear on PR with lucene index should throw exception

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

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

commit 9e80b12226006aeda4779a1def09397c529a6633
Author: zhouxh <gz...@pivotal.io>
AuthorDate: Fri Apr 9 16:08:33 2021 -0700

    feature/GEODE-7674: Clear on PR with lucene index should throw exception
---
 .../internal/AsyncEventQueueFactoryImpl.java       |  6 ++++++
 .../asyncqueue/internal/AsyncEventQueueImpl.java   |  5 +++++
 .../org/apache/geode/cache/wan/GatewaySender.java  |  2 ++
 .../internal/cache/PartitionedRegionClear.java     | 14 +++++++++++++
 .../internal/cache/wan/AbstractGatewaySender.java  |  8 ++++++++
 .../cache/wan/GatewaySenderAttributes.java         |  7 +++++++
 .../internal/cache/wan/InternalGatewaySender.java  |  2 ++
 .../cache/lucene/LuceneIndexCreationDUnitTest.java | 23 ++++++++++++++++++++++
 .../cache/lucene/internal/LuceneIndexImpl.java     |  1 +
 9 files changed, 68 insertions(+)

diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
index 004cb4e..4f5a990 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
@@ -281,6 +281,12 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
     return this;
   }
 
+  // keep this method internal
+  public AsyncEventQueueFactory setPartitionedRegionClearSupported(boolean supported) {
+    gatewaySenderAttributes.partitionedRegionClearSupported = supported;
+    return this;
+  }
+
   @Override
   public AsyncEventQueueFactory pauseEventDispatching() {
     pauseEventsDispatching = true;
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
index df7c908..bb2ad74 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueImpl.java
@@ -201,6 +201,11 @@ public class AsyncEventQueueImpl implements InternalAsyncEventQueue {
     return sender.isForwardExpirationDestroy();
   }
 
+  // keep this method internal
+  public boolean isPartitionedRegionClearSupported() {
+    return sender.isPartitionedRegionClearSupported();
+  }
+
   public boolean waitUntilFlushed(long timeout, TimeUnit unit) throws InterruptedException {
     return sender.waitUntilFlushed(timeout, unit);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
index 938c177..52ffcaf 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
@@ -101,6 +101,8 @@ public interface GatewaySender {
 
   boolean DEFAULT_FORWARD_EXPIRATION_DESTROY = false;
 
+  boolean DEFAULT_PARTITIONED_REGION_CLEAR_SUPPORTED = false;
+
   @Immutable
   OrderPolicy DEFAULT_ORDER_POLICY = OrderPolicy.KEY;
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java
index 539f682..a9dad71 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java
@@ -17,6 +17,7 @@ package org.apache.geode.internal.cache;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
@@ -27,6 +28,8 @@ import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.OperationAbortedException;
 import org.apache.geode.cache.PartitionedRegionPartialClearException;
+import org.apache.geode.cache.asyncqueue.AsyncEventQueue;
+import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import org.apache.geode.cache.partition.PartitionRegionHelper;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.MembershipListener;
@@ -396,6 +399,17 @@ public class PartitionedRegionClear {
       // Force all primary buckets to be created before clear.
       assignAllPrimaryBuckets();
 
+      Iterator<AsyncEventQueue> allAEQsIterator =
+          partitionedRegion.getCache().getAsyncEventQueues(false).iterator();
+      while (allAEQsIterator.hasNext()) {
+        AsyncEventQueueImpl aeq = (AsyncEventQueueImpl) allAEQsIterator.next();
+        if (aeq.isPartitionedRegionClearSupported()) {
+          // this is a lucene aeq
+          throw new UnsupportedOperationException(
+              "PartitionedRegion clear is not supported on region with lucene index");
+        }
+      }
+
       // do cacheWrite
       if (cacheWrite) {
         invokeCacheWriter(regionEvent);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
index 2f69893..214abb2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
@@ -143,6 +143,8 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
 
   protected boolean forwardExpirationDestroy;
 
+  protected boolean partitionedRegionClearSupported;
+
   protected GatewayEventSubstitutionFilter substitutionFilter;
 
   protected LocatorDiscoveryCallback locatorDiscoveryCallback;
@@ -290,6 +292,7 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
     }
     this.isBucketSorted = attrs.isBucketSorted();
     this.forwardExpirationDestroy = attrs.isForwardExpirationDestroy();
+    this.partitionedRegionClearSupported = attrs.isPartitionedRegionClearSupported();
   }
 
   public GatewaySenderAdvisor getSenderAdvisor() {
@@ -387,6 +390,11 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
   }
 
   @Override
+  public boolean isPartitionedRegionClearSupported() {
+    return this.partitionedRegionClearSupported;
+  }
+
+  @Override
   public boolean isManualStart() {
     return this.manualStart;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java
index 581b576..17e696b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java
@@ -85,6 +85,9 @@ public class GatewaySenderAttributes {
 
   public boolean forwardExpirationDestroy = GatewaySender.DEFAULT_FORWARD_EXPIRATION_DESTROY;
 
+  public boolean partitionedRegionClearSupported =
+      GatewaySender.DEFAULT_PARTITIONED_REGION_CLEAR_SUPPORTED;
+
   public boolean enforceThreadsConnectSameReceiver =
       GatewaySender.DEFAULT_ENFORCE_THREADS_CONNECT_SAME_RECEIVER;
 
@@ -208,6 +211,10 @@ public class GatewaySenderAttributes {
     return this.forwardExpirationDestroy;
   }
 
+  public boolean isPartitionedRegionClearSupported() {
+    return this.partitionedRegionClearSupported;
+  }
+
   public boolean getEnforceThreadsConnectSameReceiver() {
     return this.enforceThreadsConnectSameReceiver;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
index 13e36e7..52089c8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
@@ -40,6 +40,8 @@ public interface InternalGatewaySender extends GatewaySender {
 
   boolean isForwardExpirationDestroy();
 
+  boolean isPartitionedRegionClearSupported();
+
   boolean getIsMetaQueue();
 
   InternalCache getCache();
diff --git a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
index c5f3c9e..7e1ddd0 100644
--- a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
+++ b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneIndexCreationDUnitTest.java
@@ -28,7 +28,9 @@ import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
 import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.fail;
 
 import java.io.FileInputStream;
@@ -107,6 +109,27 @@ public class LuceneIndexCreationDUnitTest extends LuceneDUnitTest {
   }
 
   @Test
+  public void verifyThrowExceptionWhenClearOnRegionWithLuceneIndex() {
+    SerializableRunnableIF createIndex = getFieldsIndexWithOneField();
+    dataStore1.invoke(() -> {
+      initDataStore(createIndex, RegionTestableType.PARTITION_REDUNDANT);
+      Region<Object, Object> region = cache.getRegion(REGION_NAME);
+      assertNotNull(region);
+      assertThrows(UnsupportedOperationException.class, () -> region.clear());
+    });
+  }
+
+  @Test
+  public void verifyNotThrowExceptionWhenClearOnRegionWithoutLuceneIndex() {
+    dataStore1.invoke(() -> {
+      initDataStore(RegionTestableType.PARTITION_REDUNDANT);
+      Region<Object, Object> region = cache.getRegion(REGION_NAME);
+      assertNotNull(region);
+      region.clear();
+    });
+  }
+
+  @Test
   public void verifyThatEmptyListIsOutputWhenThereAreNoIndexesInTheSystem() {
     dataStore1.invoke(() -> verifyIndexList(0));
     dataStore2.invoke(() -> verifyIndexList(0));
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
index 8525e88..417e909 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexImpl.java
@@ -200,6 +200,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     factory.setDiskStoreName(attributes.getDiskStoreName());
     factory.setDiskSynchronous(true);
     factory.setForwardExpirationDestroy(true);
+    factory.setPartitionedRegionClearSupported(true);
     return factory;
   }