You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kh...@apache.org on 2017/03/17 20:11:04 UTC

[29/49] geode git commit: GEODE-2635: Creating DUnit tests to test eviction in lucene

GEODE-2635: Creating DUnit tests to test eviction in lucene

	* DUnit tests for eviction with local destroy and overflow
	* Refactored the integration tests for eviction

	This closes #420


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

Branch: refs/heads/feature/GEODE-2420
Commit: 16bf532a7bf92ef2ef666d3c823ab86a69f8cec8
Parents: 6c66e65
Author: nabarunnag <na...@cs.wisc.edu>
Authored: Wed Mar 8 16:16:52 2017 -0800
Committer: Ken Howe <kh...@pivotal.io>
Committed: Fri Mar 17 13:09:45 2017 -0700

----------------------------------------------------------------------
 .../geode/cache/lucene/EvictionDUnitTest.java   | 152 +++++++++++++++++++
 .../geode/cache/lucene/LuceneDUnitTest.java     |  22 ++-
 .../LuceneIndexCreationIntegrationTest.java     |  18 ++-
 3 files changed, 189 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/16bf532a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
new file mode 100644
index 0000000..0fb0d9d
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
@@ -0,0 +1,152 @@
+/*
+ * 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.lucene;
+
+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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
+import org.apache.geode.test.dunit.SerializableRunnableIF;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.awaitility.Awaitility;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+
+@Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class EvictionDUnitTest extends LuceneQueriesAccessorBase {
+
+  protected final static float INITIAL_EVICTION_HEAP_PERCENTAGE = 50.9f;
+  protected final static float EVICTION_HEAP_PERCENTAGE_FAKE_NOTIFICATION = 85.0f;
+  protected final static int TEST_MAX_MEMORY = 100;
+  protected final static int MEMORY_USED_FAKE_NOTIFICATION = 90;
+
+  protected RegionTestableType[] getPartitionRedundantOverflowEvictionRegionType() {
+    return new RegionTestableType[] {
+        RegionTestableType.PARTITION_PERSISTENT_REDUNDANT_EVICTION_OVERFLOW};
+  }
+
+  protected RegionTestableType[] getPartitionRedundantLocalDestroyEvictionRegionType() {
+    return new RegionTestableType[] {RegionTestableType.PARTITION_REDUNDANT_EVICTION_LOCAL_DESTROY,
+        RegionTestableType.PARTITION_REDUNDANT_PERSISTENT_EVICTION_LOCAL_DESTROY,
+        RegionTestableType.PARTITION_EVICTION_LOCAL_DESTROY,
+        RegionTestableType.PARTITION_PERSISTENT_EVICTION_LOCAL_DESTROY};
+  }
+
+  @Test
+  @Parameters(method = "getPartitionRedundantLocalDestroyEvictionRegionType")
+  public void regionWithEvictionWithLocalDestroyMustNotbeAbleToCreateLuceneIndexes(
+      RegionTestableType regionTestType) {
+    SerializableRunnableIF createIndex = getSerializableRunnableIFCreateIndex();
+
+    dataStore1.invoke(() -> {
+      try {
+        initDataStore(createIndex, regionTestType);
+      } catch (UnsupportedOperationException e) {
+        assertEquals(
+            "Lucene indexes on regions with eviction and action local destroy are not supported",
+            e.getMessage());
+        assertNull(getCache().getRegion(REGION_NAME));
+      }
+    });
+
+  }
+
+  private SerializableRunnableIF getSerializableRunnableIFCreateIndex() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
+    };
+  }
+
+  @Test
+  @Parameters(method = "getPartitionRedundantOverflowEvictionRegionType")
+  public void regionsWithEvictionWithOverflowMustBeAbleToCreateLuceneIndexes(
+      RegionTestableType regionTestType) {
+    SerializableRunnableIF createIndex = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
+    };
+
+    dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
+
+    accessor.invoke(() -> initDataStore(createIndex, regionTestType));
+
+    accessor.invoke(() -> {
+      Cache cache = getCache();
+      Region region = cache.getRegion(REGION_NAME);
+      IntStream.range(0, NUM_BUCKETS).forEach(i -> region.put(i, new TestObject("hello world")));
+    });
+
+    dataStore1.invoke(() -> {
+      try {
+        getCache().getResourceManager().setEvictionHeapPercentage(INITIAL_EVICTION_HEAP_PERCENTAGE);
+        final PartitionedRegion partitionedRegion = (PartitionedRegion) getRootRegion(REGION_NAME);
+        raiseFakeNotification();
+        Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+          assertTrue(partitionedRegion.getDiskRegionStats().getNumOverflowOnDisk() > 0);
+        });
+      } finally {
+        cleanUpAfterFakeNotification();
+      }
+    });
+
+    accessor.invoke(() -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      LuceneQuery<Integer, TestObject> query = luceneService.createLuceneQueryFactory()
+          .setResultLimit(100).create(INDEX_NAME, REGION_NAME, "world", "text");
+      List<LuceneResultStruct<Integer, TestObject>> resultList = query.findResults();
+      assertEquals(NUM_BUCKETS, resultList.size());
+    });
+
+  }
+
+  protected void raiseFakeNotification() {
+    ((GemFireCacheImpl) getCache()).getHeapEvictor().testAbortAfterLoopCount = 1;
+    HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
+    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", "0");
+
+    getCache().getResourceManager()
+        .setEvictionHeapPercentage(EVICTION_HEAP_PERCENTAGE_FAKE_NOTIFICATION);
+    HeapMemoryMonitor heapMemoryMonitor =
+        ((GemFireCacheImpl) getCache()).getResourceManager().getHeapMonitor();
+    heapMemoryMonitor.setTestMaxMemoryBytes(TEST_MAX_MEMORY);
+
+    heapMemoryMonitor.updateStateAndSendEvent(MEMORY_USED_FAKE_NOTIFICATION);
+  }
+
+  protected void cleanUpAfterFakeNotification() {
+    ((GemFireCacheImpl) getCache()).getHeapEvictor().testAbortAfterLoopCount = Integer.MAX_VALUE;
+    HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
+    System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/16bf532a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneDUnitTest.java
index db48294..fa3c470 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneDUnitTest.java
@@ -36,7 +36,7 @@ public abstract class LuceneDUnitTest extends JUnit4CacheTestCase {
   protected VM dataStore2;
 
   protected static int NUM_BUCKETS = 10;
-
+  protected static int MAX_ENTRIES_FOR_EVICTION = 20;
 
   @Override
   public void postSetUp() throws Exception {
@@ -83,7 +83,25 @@ public abstract class LuceneDUnitTest extends JUnit4CacheTestCase {
     PARTITION_OVERFLOW_TO_DISK(RegionShortcut.PARTITION_PROXY, RegionShortcut.PARTITION_OVERFLOW,
         EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK)),
     FIXED_PARTITION(RegionShortcut.PARTITION, RegionShortcut.PARTITION),
-    PARTITION_WITH_CLIENT(RegionShortcut.PARTITION_PROXY, RegionShortcut.PARTITION);
+    PARTITION_WITH_CLIENT(RegionShortcut.PARTITION_PROXY, RegionShortcut.PARTITION),
+    PARTITION_PERSISTENT_REDUNDANT_EVICTION_OVERFLOW(RegionShortcut.PARTITION_PROXY_REDUNDANT,
+        RegionShortcut.PARTITION_REDUNDANT_PERSISTENT,
+        EvictionAttributes.createLRUHeapAttributes(null, EvictionAction.OVERFLOW_TO_DISK)),
+    PARTITION_REDUNDANT_EVICTION_LOCAL_DESTROY(RegionShortcut.PARTITION_PROXY_REDUNDANT,
+        RegionShortcut.PARTITION_REDUNDANT, EvictionAttributes
+            .createLRUEntryAttributes(MAX_ENTRIES_FOR_EVICTION, EvictionAction.LOCAL_DESTROY)),
+    PARTITION_REDUNDANT_PERSISTENT_EVICTION_LOCAL_DESTROY(RegionShortcut.PARTITION_PROXY_REDUNDANT,
+        RegionShortcut.PARTITION_REDUNDANT_PERSISTENT, EvictionAttributes
+            .createLRUEntryAttributes(MAX_ENTRIES_FOR_EVICTION, EvictionAction.LOCAL_DESTROY)),
+    PARTITION_EVICTION_LOCAL_DESTROY(RegionShortcut.PARTITION_PROXY, RegionShortcut.PARTITION,
+        EvictionAttributes.createLRUEntryAttributes(MAX_ENTRIES_FOR_EVICTION,
+            EvictionAction.LOCAL_DESTROY)),
+    PARTITION_PERSISTENT_EVICTION_LOCAL_DESTROY(RegionShortcut.PARTITION_PROXY,
+        RegionShortcut.PARTITION_PERSISTENT, EvictionAttributes
+            .createLRUEntryAttributes(MAX_ENTRIES_FOR_EVICTION, EvictionAction.LOCAL_DESTROY))
+
+
+    ;
 
 
     EvictionAttributes evictionAttributes = null;

http://git-wip-us.apache.org/repos/asf/geode/blob/16bf532a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 80e77e9..5a86ccb 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -195,7 +195,8 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void cannotCreateLuceneIndexForRegionWithEviction() throws IOException, ParseException {
+  public void cannotCreateLuceneIndexForRegionWithEvictionWithLocalDestroy()
+      throws IOException, ParseException {
     try {
       createIndex("field1", "field2", "field3");
       RegionFactory regionFactory = this.cache.createRegionFactory(RegionShortcut.PARTITION);
@@ -211,6 +212,21 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
+  public void canCreateLuceneIndexForRegionWithEvictionWithOverflowToDisk()
+      throws IOException, ParseException {
+    try {
+      createIndex("field1", "field2", "field3");
+      RegionFactory regionFactory = this.cache.createRegionFactory(RegionShortcut.PARTITION);
+      regionFactory.setEvictionAttributes(
+          EvictionAttributes.createLRUHeapAttributes(null, EvictionAction.OVERFLOW_TO_DISK));
+      regionFactory.create(REGION_NAME);
+    } catch (Exception e) {
+      fail("Should have been able to create Lucene Index");
+      e.printStackTrace();
+    }
+  }
+
+  @Test
   public void cannotCreateLuceneIndexWithExistingIndexName() {
     expectedException.expect(IllegalArgumentException.class);
     createIndex("field1", "field2", "field3");