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/28 16:46:47 UTC

[19/35] geode git commit: GEODE-2404: Added gfsh support for destroying a lucene index

http://git-wip-us.apache.org/repos/asf/geode/blob/834235a5/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
index f86f4a1..1d2d773 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/functions/LuceneDestroyIndexFunctionJUnitTest.java
@@ -17,6 +17,8 @@ package org.apache.geode.cache.lucene.internal.cli.functions;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.ResultSender;
 import org.apache.geode.cache.lucene.internal.InternalLuceneService;
+import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
+import org.apache.geode.cache.lucene.internal.cli.LuceneDestroyIndexInfo;
 import org.apache.geode.cache.lucene.internal.cli.LuceneIndexInfo;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
@@ -34,7 +36,7 @@ import static org.mockito.Mockito.*;
 @Category(UnitTest.class)
 public class LuceneDestroyIndexFunctionJUnitTest {
 
-  private InternalLuceneService service;
+  private LuceneServiceImpl service;
   private GemFireCacheImpl cache;
   private String member;
   private FunctionContext context;
@@ -44,7 +46,7 @@ public class LuceneDestroyIndexFunctionJUnitTest {
   public void prepare() {
     this.cache = Fakes.cache();
     this.member = Fakes.distributedSystem().getDistributedMember().getId();
-    this.service = mock(InternalLuceneService.class);
+    this.service = mock(LuceneServiceImpl.class);
     when(this.cache.getService(InternalLuceneService.class)).thenReturn(this.service);
     this.context = mock(FunctionContext.class);
     this.resultSender = mock(ResultSender.class);
@@ -53,41 +55,127 @@ public class LuceneDestroyIndexFunctionJUnitTest {
 
   @Test
   @SuppressWarnings("unchecked")
-  public void testExecuteWithRegionAndIndex() throws Throwable {
-    LuceneIndexInfo indexInfo = new LuceneIndexInfo("index1", "/region1");
+  public void testDestroyIndex() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo("index1", "/region1", false);
     when(this.context.getArguments()).thenReturn(indexInfo);
     LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
     function = spy(function);
     doReturn(this.cache).when(function).getCache();
-    doReturn(mock(XmlEntity.class)).when(function).getXmlEntity(any());
     function.execute(this.context);
     verify(this.service).destroyIndex(eq("index1"), eq("/region1"));
+    verify(function).getXmlEntity(eq("index1"), eq("/region1"));
+    verify(this.service, never()).destroyDefinedIndex(eq("index1"), eq("/region1"));
     verify(this.service, never()).destroyIndexes(eq("/region1"));
-    ArgumentCaptor<CliFunctionResult> resultCaptor =
-        ArgumentCaptor.forClass(CliFunctionResult.class);
-    verify(resultSender).lastResult(resultCaptor.capture());
-    CliFunctionResult result = resultCaptor.getValue();
-    assertEquals(this.member, result.getMemberIdOrName());
-    assertEquals(true, result.isSuccessful());
+    verifyFunctionResult(true);
   }
 
   @Test
   @SuppressWarnings("unchecked")
-  public void testExecuteWithRegion() throws Throwable {
-    LuceneIndexInfo indexInfo = new LuceneIndexInfo(null, "/region1");
+  public void testDestroyIndexFailure() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo("index1", "/region1", false);
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    doThrow(new IllegalStateException()).when(this.service).destroyIndex(eq("index1"),
+        eq("/region1"));
+    function.execute(this.context);
+    verifyFunctionResult(false);
+  }
+
+  @Test
+  public void testDestroyDefinedIndex() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo("index1", "/region1", true);
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    function.execute(this.context);
+    verify(this.service).destroyDefinedIndex(eq("index1"), eq("/region1"));
+    verify(this.service, never()).destroyIndex(eq("index1"), eq("/region1"));
+    verify(this.service, never()).destroyIndexes(eq("/region1"));
+    verify(function, never()).getXmlEntity(eq("index1"), eq("/region1"));
+    verifyFunctionResult(true);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDestroyDefinedIndexFailure() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo("index1", "/region1", true);
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    doThrow(new IllegalStateException()).when(this.service).destroyDefinedIndex(eq("index1"),
+        eq("/region1"));
+    function.execute(this.context);
+    verifyFunctionResult(false);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDestroyIndexes() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo(null, "/region1", false);
     when(this.context.getArguments()).thenReturn(indexInfo);
     LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
     function = spy(function);
     doReturn(this.cache).when(function).getCache();
-    doReturn(mock(XmlEntity.class)).when(function).getXmlEntity(any());
     function.execute(this.context);
     verify(this.service).destroyIndexes(eq("/region1"));
+    verify(function).getXmlEntity(eq(null), eq("/region1"));
+    verify(this.service, never()).destroyDefinedIndexes(eq("/region1"));
     verify(this.service, never()).destroyIndex(any(), eq("/region1"));
+    verifyFunctionResult(true);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDestroyIndexesFailure() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo(null, "/region1", false);
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    doThrow(new IllegalStateException()).when(this.service).destroyIndexes(eq("/region1"));
+    function.execute(this.context);
+    verifyFunctionResult(false);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDestroyDefinedIndexes() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo(null, "/region1", true);
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    function.execute(this.context);
+    verify(this.service).destroyDefinedIndexes(eq("/region1"));
+    verify(this.service, never()).destroyIndexes(eq("/region1"));
+    verify(this.service, never()).destroyIndex(any(), eq("/region1"));
+    verify(function, never()).getXmlEntity(eq("index1"), eq("/region1"));
+    verifyFunctionResult(true);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDestroyDefinedIndexesFailure() throws Throwable {
+    LuceneDestroyIndexInfo indexInfo = new LuceneDestroyIndexInfo(null, "/region1", true);
+    when(this.context.getArguments()).thenReturn(indexInfo);
+    LuceneDestroyIndexFunction function = new LuceneDestroyIndexFunction();
+    function = spy(function);
+    doReturn(this.cache).when(function).getCache();
+    doThrow(new IllegalStateException()).when(this.service).destroyDefinedIndexes(eq("/region1"));
+    function.execute(this.context);
+    verifyFunctionResult(false);
+  }
+
+  private void verifyFunctionResult(boolean result) {
     ArgumentCaptor<CliFunctionResult> resultCaptor =
         ArgumentCaptor.forClass(CliFunctionResult.class);
     verify(resultSender).lastResult(resultCaptor.capture());
-    CliFunctionResult result = resultCaptor.getValue();
-    assertEquals(this.member, result.getMemberIdOrName());
-    assertEquals(true, result.isSuccessful());
+    CliFunctionResult functionResult = resultCaptor.getValue();
+    assertEquals(this.member, functionResult.getMemberIdOrName());
+    assertEquals(result, functionResult.isSuccessful());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/834235a5/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
index 58f1870..5c18325 100755
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
@@ -57,8 +57,6 @@ import java.util.Properties;
 @Category(DistributedTest.class)
 public class LuceneClusterConfigurationDUnitTest {
 
-  private String groupName = "Lucene";
-
   @Rule
   public LocatorServerStartupRule ls = new LocatorServerStartupRule();
 
@@ -74,19 +72,19 @@ public class LuceneClusterConfigurationDUnitTest {
 
   @Test
   public void indexGetsCreatedUsingClusterConfiguration() throws Exception {
-    Member vm1 = startNodeUsingClusterConfiguration(1, false);
+    Member vm1 = startNodeUsingClusterConfiguration(1);
 
     // Connect Gfsh to locator.
     gfshConnector.connectAndVerify(locator);
 
     // Create lucene index.
-    createLuceneIndexUsingGfsh(false);
+    createLuceneIndexUsingGfsh();
 
     createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, null);
 
     // Start vm2. This should have lucene index created using cluster
     // configuration.
-    MemberVM vm2 = startNodeUsingClusterConfiguration(2, false);
+    MemberVM vm2 = startNodeUsingClusterConfiguration(2);
     vm2.invoke(() -> {
       LuceneService luceneService =
           LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
@@ -99,20 +97,20 @@ public class LuceneClusterConfigurationDUnitTest {
 
   @Test
   public void indexWithAnalyzerGetsCreatedUsingClusterConfiguration() throws Exception {
-    startNodeUsingClusterConfiguration(1, false);
+    startNodeUsingClusterConfiguration(1);
 
     // Connect Gfsh to locator.
     gfshConnector.connectAndVerify(locator);
 
     // Create lucene index.
-    // createLuceneIndexUsingGfsh(false);
+    // createLuceneIndexUsingGfsh();
     createLuceneIndexWithAnalyzerUsingGfsh(false);
 
     createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, null);
 
     // Start vm2. This should have lucene index created using cluster
     // configuration.
-    MemberVM vm2 = startNodeUsingClusterConfiguration(2, false);
+    MemberVM vm2 = startNodeUsingClusterConfiguration(2);
     vm2.invoke(() -> {
       LuceneService luceneService =
           LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
@@ -130,117 +128,8 @@ public class LuceneClusterConfigurationDUnitTest {
   }
 
   @Test
-  public void indexGetsCreatedOnGroupOfNodes() throws Exception {
-
-    // Start vm1, vm2 in group
-    MemberVM vm1 = startNodeUsingClusterConfiguration(1, true);
-    MemberVM vm2 = startNodeUsingClusterConfiguration(2, true);
-
-    // Start vm3 outside the group. The Lucene index should not be present here.
-    MemberVM vm3 = startNodeUsingClusterConfiguration(3, true);
-
-    // Connect Gfsh to locator.
-    gfshConnector.connectAndVerify(locator);
-
-    // Create lucene index on group.
-    createLuceneIndexUsingGfsh(true);
-
-    // Create region.
-    createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, groupName);
-
-    // VM2 should have lucene index created using gfsh execution.
-    vm2.invoke(() -> {
-      LuceneService luceneService =
-          LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
-      final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
-      assertNotNull(index);
-      validateIndexFields(new String[] {"field1", "field2", "field3"}, index);
-    });
-
-    // The Lucene index is present in vm3.
-    vm3.invoke(() -> {
-      LuceneService luceneService =
-          LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
-      final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
-      assertNotNull(index);
-    });
-  }
-
-  @Test
-  public void indexNotCreatedOnNodeOutSideTheGroup() throws Exception {
-    // Start vm1, vm2 in group
-    MemberVM vm1 = startNodeUsingClusterConfiguration(1, true);
-    MemberVM vm2 = startNodeUsingClusterConfiguration(2, true);
-
-    // Start vm3 outside the group. The Lucene index should not be present here.
-    MemberVM vm3 = startNodeUsingClusterConfiguration(3, false);
-
-    // Connect Gfsh to locator.
-    gfshConnector.connectAndVerify(locator);
-
-    // Create lucene index on group.
-    createLuceneIndexUsingGfsh(true);
-
-    // Create region.
-    createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, groupName);
-
-    // VM2 should have lucene index created using gfsh execution
-    vm2.invoke(() -> {
-      LuceneService luceneService =
-          LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
-      final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
-      assertNotNull(index);
-      validateIndexFields(new String[] {"field1", "field2", "field3"}, index);
-    });
-
-    // The Lucene index should not be present in vm3.
-    vm3.invoke(() -> {
-      LuceneService luceneService =
-          LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
-      final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
-      assertNull(index);
-    });
-  }
-
-  @Test
-  public void indexAreCreatedInValidGroupOfNodesJoiningLater() throws Exception {
-    // Start vm1 in group
-    startNodeUsingClusterConfiguration(1, true);
-    // Connect Gfsh to locator.
-    gfshConnector.connectAndVerify(locator);
-
-    // Create lucene index on group.
-    createLuceneIndexUsingGfsh(true);
-
-    createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, groupName);
-
-    // Start vm2 in group
-    MemberVM vm2 = startNodeUsingClusterConfiguration(2, true);
-
-    // Start vm3 outside the group. The Lucene index should not be present here.
-    MemberVM vm3 = startNodeUsingClusterConfiguration(3, false);
-
-    // VM2 should have lucene index created using gfsh execution
-    vm2.invoke(() -> {
-      LuceneService luceneService =
-          LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
-      final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
-      assertNotNull(index);
-      validateIndexFields(new String[] {"field1", "field2", "field3"}, index);
-    });
-
-    // The Lucene index should not be present in vm3.
-    vm3.invoke(() -> {
-      LuceneService luceneService =
-          LuceneServiceProvider.get(LocatorServerStartupRule.serverStarter.getCache());
-      final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
-      assertNull(index);
-    });
-  }
-
-  @Test
   public void verifyClusterConfigurationAfterDestroyIndex() throws Exception {
-    Member vm1 = startNodeUsingClusterConfiguration(1, false);
+    Member vm1 = startNodeUsingClusterConfiguration(1);
 
     // Connect Gfsh to locator.
     gfshConnector.connectAndVerify(locator);
@@ -260,7 +149,7 @@ public class LuceneClusterConfigurationDUnitTest {
 
   @Test
   public void verifyClusterConfigurationAfterDestroyIndexes() throws Exception {
-    Member vm1 = startNodeUsingClusterConfiguration(1, false);
+    Member vm1 = startNodeUsingClusterConfiguration(1);
 
     // Connect Gfsh to locator.
     gfshConnector.connectAndVerify(locator);
@@ -277,10 +166,10 @@ public class LuceneClusterConfigurationDUnitTest {
 
   private void createAndAddIndexes() throws Exception {
     // Create lucene index.
-    createLuceneIndexUsingGfsh(INDEX_NAME + "0", false);
+    createLuceneIndexUsingGfsh(INDEX_NAME + "0");
 
     // Create another lucene index.
-    createLuceneIndexUsingGfsh(INDEX_NAME + "1", false);
+    createLuceneIndexUsingGfsh(INDEX_NAME + "1");
 
     // Create region
     createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, null);
@@ -311,28 +200,21 @@ public class LuceneClusterConfigurationDUnitTest {
     };
   }
 
-  private MemberVM startNodeUsingClusterConfiguration(int vmIndex, boolean addGroup)
-      throws Exception {
+  private MemberVM startNodeUsingClusterConfiguration(int vmIndex) throws Exception {
     Properties nodeProperties = new Properties();
-    if (addGroup) {
-      nodeProperties.setProperty(GROUPS, groupName);
-    }
     return ls.startServerVM(vmIndex, nodeProperties, ls.getMember(0).getPort());
   }
 
-  private void createLuceneIndexUsingGfsh(boolean addGroup) throws Exception {
-    createLuceneIndexUsingGfsh(INDEX_NAME, addGroup);
+  private void createLuceneIndexUsingGfsh() throws Exception {
+    createLuceneIndexUsingGfsh(INDEX_NAME);
   }
 
-  private void createLuceneIndexUsingGfsh(String indexName, boolean addGroup) throws Exception {
+  private void createLuceneIndexUsingGfsh(String indexName) throws Exception {
     // Execute Gfsh command to create lucene index.
     CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, indexName);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
-    if (addGroup) {
-      csb.addOption(LuceneCliStrings.LUCENE_CREATE_INDEX__GROUP, groupName);
-    }
     csb.addOption(LuceneCliStrings.LUCENE_CREATE_INDEX__FIELD, "field1,field2,field3");
     gfshConnector.executeAndVerifyCommand(csb.toString());
   }
@@ -349,9 +231,6 @@ public class LuceneClusterConfigurationDUnitTest {
             + "org.apache.lucene.analysis.standard.StandardAnalyzer,"
             + "org.apache.lucene.analysis.standard.StandardAnalyzer");
 
-    if (addGroup) {
-      csb.addOption(LuceneCliStrings.LUCENE_CREATE_INDEX__GROUP, groupName);
-    }
     // Execute Gfsh command.
     gfshConnector.executeAndVerifyCommand(csb.toString());
   }