You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ag...@apache.org on 2016/05/17 00:38:39 UTC

[30/50] [abbrv] incubator-geode git commit: GEODE-11: Integration test of lucene index with analyzers

GEODE-11: Integration test of lucene index with analyzers

Also moved the LuceneIndexCreationIngregrationTest to the public
package.


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

Branch: refs/heads/feature/GEODE-1209
Commit: f7401815fc39ec44d5390e5cf6bf34fcb3267e9d
Parents: ca89eb4
Author: Dan Smith <up...@apache.org>
Authored: Wed May 11 16:40:56 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Mon May 16 17:31:55 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneService.java     |   4 +-
 .../LuceneIndexForPartitionedRegion.java        |   4 +-
 .../LuceneIndexCreationIntegrationTest.java     | 122 +++++++++++++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |  55 +++++++++
 .../lucene/LuceneQueriesIntegrationTest.java    |  21 +---
 .../LuceneIndexCreationIntegrationTest.java     |  74 -----------
 .../lucene/internal/LuceneIntegrationTest.java  |  47 -------
 7 files changed, 181 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
index a42ab72..462aa7e 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
@@ -80,10 +80,8 @@ public interface LuceneService {
    * @param indexName index name
    * @param regionPath region name
    * @param analyzerPerField analyzer per field map
-   * @deprecated TODO This feature is not yet implemented
    */
-  @Deprecated
-  public void createIndex(String indexName, String regionPath,  
+  public void createIndex(String indexName, String regionPath,
       Map<String, Analyzer> analyzerPerField);
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index 382f61c..dd9d384 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -138,7 +138,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return createRegion(fileRegionName, regionShortCut, this.regionPath, partitionAttributes);
   }
 
-  String createFileRegionName() {
+  public String createFileRegionName() {
     return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath)+".files";
   }
 
@@ -152,7 +152,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return createRegion(chunkRegionName, regionShortCut, fileRegionName, partitionAttributes);
   }
 
-  String createChunkRegionName() {
+  public String createChunkRegionName() {
     return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
new file mode 100644
index 0000000..1dfaf51
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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 com.gemstone.gemfire.cache.lucene;
+
+import static org.junit.Assert.*;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.jayway.awaitility.Awaitility;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  @Test
+  public void shouldCreateIndexWriterWithAnalyzersWhenSettingPerFieldAnalyzers()
+    throws BucketNotFoundException, InterruptedException
+  {
+    Map<String, Analyzer> analyzers = new HashMap<>();
+
+    final RecordingAnalyzer field1Analyzer = new RecordingAnalyzer();
+    final RecordingAnalyzer field2Analyzer = new RecordingAnalyzer();
+    analyzers.put("field1", field1Analyzer);
+    analyzers.put("field2", field2Analyzer);
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, analyzers);
+    Region region = createRegion();
+    final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    region.put("key1", new TestObject());
+
+    assertEquals(analyzers, index.getFieldAnalyzers());
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
+      assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
+    });
+  }
+
+  @Test
+  public void verifyLuceneRegionInternal() {
+    createIndex("text");
+
+    // Create partitioned region
+    createRegion();
+
+    // Get index
+    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    // Verify the meta regions exist and are internal
+    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
+    assertTrue(chunkRegion.isInternalRegion());
+    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
+    assertTrue(fileRegion.isInternalRegion());
+
+    // Verify the meta regions are not contained in the root regions
+    for (Region region : cache.rootRegions()) {
+      assertNotEquals(chunkRegion.getFullPath(), region.getFullPath());
+      assertNotEquals(fileRegion.getFullPath(), region.getFullPath());
+    }
+  }
+
+  private Region createRegion() {
+    return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+  }
+
+  private void createIndex(String fieldName) {
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldName);
+  }
+
+  private static class TestObject implements Serializable {
+
+    String field1 = "a b c d";
+    String field2 = "f g h";
+  }
+
+  private static class RecordingAnalyzer extends Analyzer {
+
+    private List<String> analyzedfields = new ArrayList<String>();
+
+    @Override protected TokenStreamComponents createComponents(final String fieldName) {
+      analyzedfields.add(fieldName);
+      return new TokenStreamComponents(new KeywordTokenizer());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
new file mode 100644
index 0000000..8c6f59e
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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 com.gemstone.gemfire.cache.lucene;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+
+public class LuceneIntegrationTest {
+
+  protected Cache cache;
+  protected LuceneService luceneService;
+
+  @Rule
+  public TestName name = new TestName();
+
+  @After
+  public void tearDown() {
+    if(this.cache != null) {
+      this.cache.close();
+    }
+  }
+
+  @Before
+  public void createCache() {
+    CacheFactory cf = new CacheFactory();
+    cf.set("mcast-port", "0");
+    cf.set("locators", "");
+    this.cache = cf.create();
+    luceneService = LuceneServiceProvider.get(this.cache);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index 9009e3d..2262339 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -19,17 +19,12 @@ package com.gemstone.gemfire.cache.lucene;
 import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.*;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -39,25 +34,11 @@ import org.junit.rules.ExpectedException;
  * This class contains tests of lucene queries that can fit
  */
 @Category(IntegrationTest.class)
-public class LuceneQueriesIntegrationTest {
+public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
   private static final String INDEX_NAME = "index";
   protected static final String REGION_NAME = "index";
-  Cache cache;
-
-  @Before
-  public void createCache() {
-    cache = new CacheFactory()
-      .set("mcast-port", "0")
-      .set("locators", "")
-      .set("log-level", "warning").create();
-  }
-
-  @After
-  public void closeCache() {
-    cache.close();
-  }
 
   @Test()
   public void throwFunctionExceptionWhenGivenBadQuery() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java
deleted file mode 100644
index e04d1c7..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexCreationIntegrationTest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.cache.lucene.internal;
-
-import static org.junit.Assert.*;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(IntegrationTest.class)
-public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
-
-  @Test
-  public void verifyLuceneRegionInternal() {
-    // Create cache
-    createCache();
-
-    // Create index
-    String indexName = "index";
-    String regionName = this.name.getMethodName();
-    createIndex(indexName, regionName, "text");
-
-    // Create partitioned region
-    createRegion(regionName);
-
-    // Get index
-    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) LuceneServiceProvider.get(this.cache).getIndex(indexName, regionName);
-    assertNotNull(index);
-
-    // Verify the meta regions exist and are internal
-    LocalRegion chunkRegion = (LocalRegion) this.cache.getRegion(index.createChunkRegionName());
-    assertNotNull(chunkRegion);
-    assertTrue(chunkRegion.isInternalRegion());
-    LocalRegion fileRegion = (LocalRegion) this.cache.getRegion(index.createFileRegionName());
-    assertNotNull(fileRegion);
-    assertTrue(fileRegion.isInternalRegion());
-
-    // Verify the meta regions are not contained in the root regions
-    for (Region region : cache.rootRegions()) {
-      assertNotEquals(chunkRegion.getFullPath(), region.getFullPath());
-      assertNotEquals(fileRegion.getFullPath(), region.getFullPath());
-    }
-  }
-
-  private Region createRegion(String regionName) {
-    return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(regionName);
-  }
-
-  private void createIndex(String indexName, String regionName, String fieldName) {
-    LuceneServiceProvider.get(this.cache).createIndex(indexName, regionName, fieldName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7401815/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java
deleted file mode 100644
index 4d94042..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIntegrationTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.cache.lucene.internal;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.rules.TestName;
-
-public class LuceneIntegrationTest {
-
-  protected Cache cache;
-
-  @Rule
-  public TestName name = new TestName();
-
-  @After
-  public void tearDown() {
-    if(this.cache != null) {
-      this.cache.close();
-    }
-  }
-
-  protected void createCache() {
-    CacheFactory cf = new CacheFactory();
-    cf.set("mcast-port", "0");
-    this.cache = cf.create();
-  }
-}