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 2017/04/27 18:47:57 UTC

[22/40] geode git commit: GEODE-2689: Modified to ignore duplicate index definition

GEODE-2689: Modified to ignore duplicate index definition


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

Branch: refs/heads/feature/GEM-1299
Commit: c8afc87b420ed55a09f131f1c69f534d89fd61c9
Parents: 809fd1a
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Tue Apr 25 12:46:49 2017 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Wed Apr 26 09:05:18 2017 -0700

----------------------------------------------------------------------
 .../geode/internal/i18n/LocalizedStrings.java   |  5 ++
 .../lucene/LuceneIndexExistsException.java      | 49 ++++++++++++++++
 .../lucene/internal/LuceneServiceImpl.java      | 11 ++--
 .../internal/xml/LuceneIndexCreation.java       | 15 ++++-
 .../LuceneIndexCreationIntegrationTest.java     |  2 +-
 .../LuceneClusterConfigurationDUnitTest.java    | 62 +++++++++++++++++++-
 6 files changed, 137 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/c8afc87b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index fb0919f..76e4acf 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -7686,6 +7686,11 @@ public class LocalizedStrings {
   public static final StringId LuceneService_MUST_DEFINE_INDEX_0_ON_REGION_1_BECAUSE_IT_IS_DEFINED_IN_ANOTHER_MEMBER =
       new StringId(6656,
           "Must create Lucene index {0} on region {1} because it is defined in another member.");
+  public static final StringId LuceneIndexExistsException_INDEX_0_ON_REGION_1_ALREADY_EXISTS =
+      new StringId(6657, "Lucene index {0} on region {1} already exists.");
+  public static final StringId LuceneIndexCreation_IGNORING_DUPLICATE_INDEX_CREATION_0_ON_REGION_1 =
+      new StringId(6658, "Ignoring duplicate index creation for Lucene index {0} on region {1}");
+
   /** Testing strings, messageId 90000-99999 **/
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/c8afc87b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexExistsException.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexExistsException.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexExistsException.java
new file mode 100644
index 0000000..ad0d349
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneIndexExistsException.java
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.geode.GemFireException;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+
+/**
+ * A LuceneIndexExistsException is thrown if a Lucene index that already exists is attempted to be
+ * created.
+ */
+public class LuceneIndexExistsException extends GemFireException {
+
+  private final String indexName;
+
+  private final String regionPath;
+
+  public LuceneIndexExistsException(String indexName, String regionPath) {
+    super();
+    this.indexName = indexName;
+    this.regionPath = regionPath;
+  }
+
+  @Override
+  public String getMessage() {
+    return LocalizedStrings.LuceneIndexExistsException_INDEX_0_ON_REGION_1_ALREADY_EXISTS
+        .toLocalizedString(new String[] {this.indexName, this.regionPath});
+  }
+
+  public String getIndexName() {
+    return this.indexName;
+  }
+
+  public String getRegionPath() {
+    return this.regionPath;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/c8afc87b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
index 935f37c..437a552 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneServiceImpl.java
@@ -18,6 +18,7 @@ package org.apache.geode.cache.lucene.internal;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.geode.cache.lucene.LuceneIndexExistsException;
 import org.apache.geode.cache.lucene.internal.distributed.LuceneQueryFunction;
 import org.apache.geode.cache.lucene.internal.management.LuceneServiceMBean;
 import org.apache.geode.cache.lucene.internal.management.ManagementIndexListener;
@@ -148,7 +149,7 @@ public class LuceneServiceImpl implements InternalLuceneService {
       regionPath = "/" + regionPath;
     }
 
-    registerDefinedIndex(LuceneServiceImpl.getUniqueIndexName(indexName, regionPath),
+    registerDefinedIndex(indexName, regionPath,
         new LuceneIndexCreationProfile(indexName, regionPath, fields, analyzer, fieldAnalyzers));
 
     Region region = cache.getRegion(regionPath);
@@ -191,10 +192,12 @@ public class LuceneServiceImpl implements InternalLuceneService {
     return luceneIndexFactory.create(indexName, regionPath, cache);
   }
 
-  private void registerDefinedIndex(final String regionAndIndex,
+  private void registerDefinedIndex(final String indexName, final String regionPath,
       final LuceneIndexCreationProfile luceneIndexCreationProfile) {
-    if (definedIndexMap.containsKey(regionAndIndex) || indexMap.containsKey(regionAndIndex))
-      throw new IllegalArgumentException("Lucene index already exists in region");
+    String regionAndIndex = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
+    if (definedIndexMap.containsKey(regionAndIndex) || indexMap.containsKey(regionAndIndex)) {
+      throw new LuceneIndexExistsException(indexName, regionPath);
+    }
     definedIndexMap.put(regionAndIndex, luceneIndexCreationProfile);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/c8afc87b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
index b878e47..06ece3b 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -18,6 +18,10 @@ package org.apache.geode.cache.lucene.internal.xml;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.geode.cache.lucene.LuceneIndexExistsException;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
@@ -38,6 +42,8 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
   private Set<String> fieldNames = new LinkedHashSet<String>();
   private Map<String, Analyzer> fieldAnalyzers;
 
+  private static Logger logger = LogService.getLogger();
+
 
   public void setRegion(Region region) {
     this.region = region;
@@ -82,7 +88,14 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
     LuceneServiceImpl service = (LuceneServiceImpl) LuceneServiceProvider.get(cache);
     Analyzer analyzer = this.fieldAnalyzers == null ? new StandardAnalyzer()
         : new PerFieldAnalyzerWrapper(new StandardAnalyzer(), this.fieldAnalyzers);
-    service.createIndex(getName(), getRegionPath(), analyzer, this.fieldAnalyzers, getFieldNames());
+    try {
+      service.createIndex(getName(), getRegionPath(), analyzer, this.fieldAnalyzers,
+          getFieldNames());
+    } catch (LuceneIndexExistsException e) {
+      logger
+          .info(LocalizedStrings.LuceneIndexCreation_IGNORING_DUPLICATE_INDEX_CREATION_0_ON_REGION_1
+              .toLocalizedString(new String[] {e.getIndexName(), e.getRegionPath()}));
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/c8afc87b/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 261b742..2fea5d5 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
@@ -229,7 +229,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
   @Test
   public void cannotCreateLuceneIndexWithExistingIndexName() {
-    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expect(LuceneIndexExistsException.class);
     createIndex("field1", "field2", "field3");
     createIndex("field4", "field5", "field6");
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/c8afc87b/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 5c18325..a0ac52f 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
@@ -35,6 +35,8 @@ import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.management.internal.cli.CommandManager;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.test.dunit.SerializableRunnableIF;
@@ -50,6 +52,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -164,6 +167,47 @@ public class LuceneClusterConfigurationDUnitTest {
     locator.invoke(verifyClusterConfiguration(false));
   }
 
+  @Test
+  public void verifyMemberWithGroupStartsAfterAlterRegion() throws Exception {
+    // Start a member with no group
+    startNodeUsingClusterConfiguration(1);
+
+    // Start a member with group
+    String group = "group1";
+    Properties properties = new Properties();
+    properties.setProperty(GROUPS, group);
+    MemberVM vm2 = startNodeUsingClusterConfiguration(2, properties);
+
+    // Connect Gfsh to locator
+    gfshConnector.connectAndVerify(locator);
+
+    // Create index and region in no group
+    createLuceneIndexUsingGfsh();
+    createRegionUsingGfsh(REGION_NAME, RegionShortcut.PARTITION, null);
+
+    // Alter region in group
+    CommandResult alterRegionResult = alterRegionUsingGfsh(group);
+    TabularResultData alterRegionResultData = (TabularResultData) alterRegionResult.getResultData();
+    List<String> alterRegionResultDataStatus = alterRegionResultData.retrieveAllValues("Status");
+
+    // Verify region is altered on only one server
+    assertEquals(1, alterRegionResultDataStatus.size());
+    assertEquals("Region \"/" + REGION_NAME + "\" altered on \"" + vm2.getName() + "\"",
+        alterRegionResultDataStatus.get(0));
+
+    // Start another member with group
+    startNodeUsingClusterConfiguration(3, properties);
+
+    // Verify all members have indexes
+    CommandResult listIndexesResult = listIndexesUsingGfsh();
+    TabularResultData listIndexesResultData = (TabularResultData) listIndexesResult.getResultData();
+    List<String> listIndexesResultDataStatus = listIndexesResultData.retrieveAllValues("Status");
+    assertEquals(3, listIndexesResultDataStatus.size());
+    for (String status : listIndexesResultDataStatus) {
+      assertEquals("Initialized", status);
+    }
+  }
+
   private void createAndAddIndexes() throws Exception {
     // Create lucene index.
     createLuceneIndexUsingGfsh(INDEX_NAME + "0");
@@ -201,7 +245,11 @@ public class LuceneClusterConfigurationDUnitTest {
   }
 
   private MemberVM startNodeUsingClusterConfiguration(int vmIndex) throws Exception {
-    Properties nodeProperties = new Properties();
+    return startNodeUsingClusterConfiguration(vmIndex, new Properties());
+  }
+
+  private MemberVM startNodeUsingClusterConfiguration(int vmIndex, Properties nodeProperties)
+      throws Exception {
     return ls.startServerVM(vmIndex, nodeProperties, ls.getMember(0).getPort());
   }
 
@@ -255,6 +303,18 @@ public class LuceneClusterConfigurationDUnitTest {
     gfshConnector.executeAndVerifyCommand(csb.toString());
   }
 
+  private CommandResult alterRegionUsingGfsh(String group) throws Exception {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_REGION);
+    csb.addOption(CliStrings.ALTER_REGION__REGION, REGION_NAME);
+    csb.addOption(CliStrings.ALTER_REGION__GROUP, group);
+    csb.addOption(CliStrings.ALTER_REGION__EVICTIONMAX, "5764");
+    return gfshConnector.executeAndVerifyCommand(csb.toString());
+  }
+
+  private CommandResult listIndexesUsingGfsh() throws Exception {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
+    return gfshConnector.executeAndVerifyCommand(csb.toString());
+  }
 
   private static void validateIndexFields(String[] indexFields, LuceneIndex index) {
     String[] indexFieldNames = index.getFieldNames();