geode-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From zho...@apache.org
Subject [geode] 01/01: GEODE-3244: provide a build in LuceneSerializer that flattens objects
Date Tue, 10 Oct 2017 04:02:31 GMT
This is an automated email from the ASF dual-hosted git repository.

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

commit 1ae128e2783de16482002803a04c2f1e83ff3795
Author: zhouxh <gzhou@pivotal.io>
AuthorDate: Thu Oct 5 15:01:59 2017 -0700

    GEODE-3244: provide a build in LuceneSerializer that flattens objects
---
 .../geode/cache/lucene/FlatFormatSerializer.java   | 119 ++++++++
 .../geode/cache/lucene/LuceneSerializer.java       |   1 +
 .../lucene/FlatFormatSerializerJUnitTest.java      |  90 ++++++
 .../NestedObjectSeralizerIntegrationTest.java      | 314 +++++++++++++++++++++
 .../serializer/SerializerTestHelper.java           |   2 +-
 .../apache/geode/cache/lucene/test/Customer.java}  |  34 +--
 .../org/apache/geode/cache/lucene/test/Page.java}  |  41 +--
 .../apache/geode/cache/lucene/test/Person.java}    |  42 +--
 8 files changed, 583 insertions(+), 60 deletions(-)

diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/FlatFormatSerializer.java
b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/FlatFormatSerializer.java
new file mode 100644
index 0000000..99690b2
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/FlatFormatSerializer.java
@@ -0,0 +1,119 @@
+/*
+ * 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 java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.geode.cache.lucene.internal.repository.serializer.SerializerUtil;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.document.Document;
+
+/**
+ * A built-in {@link LuceneSerializer} to parse user's nested object into a flat format,
i.e. a
+ * single document. Each nested object will become a set of fields, with field name in format
of
+ * contact.name, contact.homepage.title.
+ * 
+ * Here is a example of usage:
+ * 
+ * User needs to explicitly setLuceneSerializer with an instance of this class, and specify
nested
+ * objects' indexed fields in following format:
+ * 
+ * luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer())
+ * .addField("name").addField("contact.name").addField("contact.email", new KeywordAnalyzer())
+ * .addField("contact.address").addField("contact.homepage.content") .create(INDEX_NAME,
+ * REGION_NAME);
+ * 
+ * Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+ * 
+ * When querying, use the same dot-separated index field name, such as contact.homepage.content
+ * 
+ * LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+ * "contact.homepage.content:Hello*", "name"); results = query.findPages();
+ */
+public class FlatFormatSerializer implements LuceneSerializer {
+
+  private final ConcurrentMap<String, List<String>> tokenizedFieldCache = new
ConcurrentHashMap<>();
+
+  private static final Logger logger = LogService.getLogger();
+
+  /**
+   * Recursively serialize each indexed field's value into a field of lucene document. The
field
+   * name will be in the same format as its indexed, such as contact.homepage.content
+   * 
+   * @param index lucene index
+   * @param value user object to be serialized into index
+   */
+  @Override
+  public Collection<Document> toDocuments(LuceneIndex index, Object value) {
+    String[] fields = index.getFieldNames();
+
+    Document doc = new Document();
+    for (String indexedFieldName : fields) {
+      List<String> tokenizedFields = tokenizeField(indexedFieldName);
+      addFieldValue(doc, indexedFieldName, value, tokenizedFields);
+    }
+
+    if (logger.isDebugEnabled()) {
+      logger.debug("FlatFormatSerializer.toDocuments: " + doc);
+    }
+    return Collections.singleton(doc);
+  }
+
+  private List<String> tokenizeField(String indexedFieldName) {
+    List<String> tokenizedFields = tokenizedFieldCache.computeIfAbsent(indexedFieldName,
+        field -> Arrays.asList(indexedFieldName.split("\\.")));
+    return tokenizedFields;
+  }
+
+  private void addFieldValue(Document doc, String indexedFieldName, Object value,
+      List<String> tokenizedFields) {
+    String currentLevelField = tokenizedFields.get(0);
+
+    Object fieldValue = getFieldValue(value, currentLevelField);
+
+    if (fieldValue == null) {
+      return;
+    }
+
+    if (tokenizedFields.size() == 1) {
+      SerializerUtil.addField(doc, indexedFieldName, fieldValue);
+    } else {
+      addFieldValue(doc, indexedFieldName, fieldValue,
+          tokenizedFields.subList(1, tokenizedFields.size()));
+    }
+  }
+
+  private Object getFieldValue(Object value, String fieldName) {
+    Class<?> clazz = value.getClass();
+    if (fieldName.equals(LuceneService.REGION_VALUE_FIELD)
+        && SerializerUtil.supportedPrimitiveTypes().contains(clazz)) {
+      return value;
+    }
+    try {
+      Field field = clazz.getDeclaredField(fieldName);
+      field.setAccessible(true);
+      return field.get(value);
+    } catch (Exception e) {
+      return null;
+    }
+  }
+}
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
index 363867d..ceebc65 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
@@ -32,6 +32,7 @@ public interface LuceneSerializer extends Declarable {
    * Add the fields of the given value to a set of documents
    * 
    * @param index lucene index
+   * @param value user object to be serialized into index
    */
   Collection<Document> toDocuments(LuceneIndex index, Object value);
 }
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/FlatFormatSerializerJUnitTest.java
b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/FlatFormatSerializerJUnitTest.java
new file mode 100644
index 0000000..7106472
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/FlatFormatSerializerJUnitTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+
+import org.apache.geode.cache.lucene.FlatFormatSerializer;
+import org.apache.geode.cache.lucene.internal.repository.serializer.SerializerTestHelper;
+import org.apache.geode.cache.lucene.test.Customer;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.apache.lucene.document.Document;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class FlatFormatSerializerJUnitTest {
+
+  @Test
+  public void shouldParseMultiLayerNestObject() {
+    String[] fields = new String[] {"name", "contact.name", "contact.email", "contact.revenue",
+        "contact.address", "contact.homepage.id", "contact.homepage.title",
+        "contact.homepage.content", LuceneService.REGION_VALUE_FIELD};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    Customer customer = new Customer("Tommy Jackson", "Tommi Jackson", 13);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
+    assertEquals(fields.length - 1, doc1.getFields().size());
+    assertEquals("Tommy Jackson", doc1.getField("name").stringValue());
+    assertEquals("Tommi Jackson", doc1.getField("contact.name").stringValue());
+    assertEquals("Tommi.Jackson@pivotal.io", doc1.getField("contact.email").stringValue());
+    assertEquals(1300, doc1.getField("contact.revenue").numericValue());
+    assertEquals("13 NW Greenbrier PKWY, Portland OR 97006",
+        doc1.getField("contact.address").stringValue());
+    assertEquals("developer", doc1.getField("contact.homepage.title").stringValue());
+    assertEquals("Hello world no 13", doc1.getField("contact.homepage.content").stringValue());
+  }
+
+  @Test
+  public void shouldParseRegionValueFieldForString() {
+    String[] fields = new String[] {"name", "contact.name", "contact.email", "contact.revenue",
+        "contact.address", "contact.homepage.id", "contact.homepage.title",
+        "contact.homepage.content", LuceneService.REGION_VALUE_FIELD};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    String str = new String("Hello world");
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, str, fields);
+    assertEquals(1, doc1.getFields().size());
+    assertEquals("Hello world", doc1.getField(LuceneService.REGION_VALUE_FIELD).stringValue());
+  }
+
+  @Test
+  public void shouldParseRegionValueFieldForInteger() {
+    String[] fields = new String[] {"name", "contact.name", "contact.email", "contact.revenue",
+        "contact.address", "contact.homepage.id", "contact.homepage.title",
+        "contact.homepage.content", LuceneService.REGION_VALUE_FIELD};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    Integer integer = 15;
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, integer, fields);
+    assertEquals(1, doc1.getFields().size());
+    assertEquals(15, doc1.getField(LuceneService.REGION_VALUE_FIELD).numericValue());
+  }
+
+  @Test
+  public void shouldNotParseNestedObjectWithoutFields() {
+    String[] fields =
+        new String[] {"name", "contact", "contact.homepage", "contact.missing", "missing2"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    Customer customer = new Customer("Tommy Jackson", "Tommi Jackson", 13);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
+    assertEquals(1, doc1.getFields().size());
+    assertEquals("Tommy Jackson", doc1.getField("name").stringValue());
+  }
+
+}
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java
b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java
new file mode 100644
index 0000000..542d5be
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java
@@ -0,0 +1,314 @@
+/*
+ * 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 java.util.concurrent.TimeUnit;
+
+import org.apache.logging.log4j.Logger;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.lucene.test.Customer;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class NestedObjectSeralizerIntegrationTest extends LuceneIntegrationTest {
+
+  private static int WAIT_FOR_FLUSH_TIME = 10000;
+  private static final Logger logger = LogService.getLogger();
+  LuceneQuery<Integer, Customer> query;
+  PageableLuceneQueryResults<Integer, Customer> results;
+
+  private Region createRegionAndIndex() {
+    luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer())
+        .addField("name").addField("contact.name").addField("contact.email", new KeywordAnalyzer())
+        .addField("contact.address").addField("contact.homepage.content")
+        .addField("contact.homepage.id").addField(LuceneService.REGION_VALUE_FIELD)
+        .create(INDEX_NAME, REGION_NAME);
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    return region;
+  }
+
+  private Region createRegionAndIndexOnInvalidFields() {
+    luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer())
+        .addField("name").addField("contact").addField("contact.page").addField("contact.missing")
+        .addField("missing2").create(INDEX_NAME, REGION_NAME);
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    return region;
+  }
+
+  private void feedSomeNestedObjects(Region region) throws InterruptedException {
+    region.put("object-13", new Customer("Tommy Jackson", "Tommi Jackson", 13));
+    region.put("object-14", new Customer("Johnny Jackson", "Johnni Jackson", 14));
+    region.put("object-15", new Customer("Johnny Jackson2", "Johnni Jackson2", 15));
+    region.put("object-16", new Customer("Johnny Jackson21", "Johnni Jackson21", 16));
+    region.put("key-1", "region value 1");
+    region.put("key-2", "region value 2");
+    region.put("key-3", "region value 3");
+    region.put("key-4", "region value 4");
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, WAIT_FOR_FLUSH_TIME,
+        TimeUnit.MILLISECONDS);
+  }
+
+  @Test
+  public void queryOnContactNameWithExpression() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "contact.name:jackson2*", "name");
+    results = query.findPages();
+    assertEquals(2, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnContactNameWithExactMath() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnni Jackson\"", "contact.name");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnNameWithWrongValue() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnni Jackson\"", "name");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnNameWithExactMatch() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnny Jackson\"", "name");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnContactEmailWithAnalyzer() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    // query-3: contact.email with KeywordAnalyzer
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Johnni.Jackson2@pivotal.io", "contact.email");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnNonExistEmailField() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Johnni.Jackson2@pivotal.io", "email");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnContactAddressWithStandardAnalyzer()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006",
+        "contact.address");
+    results = query.findPages();
+    assertEquals(4, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnNonExistAddressField() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006",
+        "address");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnThreeLayerField() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "contact.homepage.content:Hello", "name");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(4, results.size());
+  }
+
+  @Test
+  public void queryOnThirdLayerFieldDirectlyShouldNotGetResult()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "Hello",
+        "content");
+    results = query.findPages();
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnRegionValueField() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndex();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "region",
+        LuceneService.REGION_VALUE_FIELD);
+    results = query.findPages();
+    printResults(results);
+    assertEquals(4, results.size());
+  }
+
+  @Test
+  public void nonExistFieldsShouldBeIgnored() throws InterruptedException, LuceneQueryException
{
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Jackson2*", "name");
+    PageableLuceneQueryResults<Integer, Customer> results = query.findPages();
+    assertEquals(2, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnNotIndexedFieldShouldReturnNothing()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnni Jackson\"", "contact.name");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryWithExactMatchWhileIndexOnSomeWrongFields()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnny Jackson\"", "name");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnNotIndexedFieldWithAnalyzerShouldReturnNothing()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Johnni.Jackson2@pivotal.io", "contact.email");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnNotIndexedContactAddressFieldShouldReturnNothing()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006",
+        "contact.address");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnNotIndexedThreeLayerFieldShouldReturnNothing()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "contact.homepage.content:Hello", "name");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnNotExistSecondLevelFieldShouldReturnNothing()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "*",
+        "contact.missing");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  @Test
+  public void queryOnNotExistTopLevelFieldShouldReturnNothing()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexOnInvalidFields();
+    feedSomeNestedObjects(region);
+
+    query =
+        luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "*", "missing2");
+    results = query.findPages();
+    assertEquals(0, results.size());
+  }
+
+  private void printResults(PageableLuceneQueryResults<Integer, Customer> results)
{
+    if (results.size() > 0) {
+      while (results.hasNext()) {
+        results.next().stream().forEach(struct -> {
+          logger.info("Result is:" + struct.getValue());
+        });
+      }
+    }
+  }
+
+}
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/SerializerTestHelper.java
b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/SerializerTestHelper.java
index 4ef06f4..e475e71 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/SerializerTestHelper.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/SerializerTestHelper.java
@@ -24,7 +24,7 @@ import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneSerializer;
 
 public class SerializerTestHelper {
-  static Document invokeSerializer(LuceneSerializer mapper, Object object, String[] fields)
{
+  public static Document invokeSerializer(LuceneSerializer mapper, Object object, String[]
fields) {
     LuceneIndex index = Mockito.mock(LuceneIndex.class);
     Mockito.when(index.getFieldNames()).thenReturn(fields);
     Collection<Document> docs = mapper.toDocuments(index, object);
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Customer.java
similarity index 59%
copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
copy to geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Customer.java
index 363867d..a75a4b5 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Customer.java
@@ -4,34 +4,30 @@
  * 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.test;
 
-package org.apache.geode.cache.lucene;
+import java.io.Serializable;
 
-import java.util.Collection;
+public class Customer implements Serializable {
+  private String name;
+  private Person contact; // search nested object
 
-import org.apache.geode.cache.Declarable;
-import org.apache.lucene.document.Document;
+  public Customer(String name, String contactName, int pageId) {
+    this.name = name;
 
-import org.apache.geode.annotations.Experimental;
+    this.contact = new Person(contactName, pageId);
+  }
 
-/**
- * An interface for writing the fields of an object into a lucene document
- */
-@Experimental
-public interface LuceneSerializer extends Declarable {
-
-  /**
-   * Add the fields of the given value to a set of documents
-   * 
-   * @param index lucene index
-   */
-  Collection<Document> toDocuments(LuceneIndex index, Object value);
+  @Override
+  public String toString() {
+    return "Customer[name=" + name + ",contact=" + contact + "]";
+  }
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Page.java
similarity index 59%
copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
copy to geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Page.java
index 363867d..bd0dc57 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Page.java
@@ -4,34 +4,35 @@
  * 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.test;
 
-package org.apache.geode.cache.lucene;
+import java.io.Serializable;
 
-import java.util.Collection;
+public class Page implements Serializable {
+  private int id;
+  private String title;
+  private String content;
 
-import org.apache.geode.cache.Declarable;
-import org.apache.lucene.document.Document;
+  public Page(int id) {
+    this.id = id;
+    if (id % 2 == 0) {
+      this.title = "manager";
+    } else {
+      this.title = "developer";
+    }
+    this.content = "Hello world no " + id;
+  }
 
-import org.apache.geode.annotations.Experimental;
-
-/**
- * An interface for writing the fields of an object into a lucene document
- */
-@Experimental
-public interface LuceneSerializer extends Declarable {
-
-  /**
-   * Add the fields of the given value to a set of documents
-   * 
-   * @param index lucene index
-   */
-  Collection<Document> toDocuments(LuceneIndex index, Object value);
+  @Override
+  public String toString() {
+    return "Page[id=" + id + ",title=" + title + ",content=" + content + "]";
+  }
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Person.java
similarity index 52%
copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
copy to geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Person.java
index 363867d..7034177 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Person.java
@@ -4,34 +4,36 @@
  * 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.test;
 
-package org.apache.geode.cache.lucene;
+import java.io.Serializable;
 
-import java.util.Collection;
+public class Person implements Serializable {
+  private String name;
+  private String email;
+  private int revenue;
+  private String address;
+  private Page homepage;
 
-import org.apache.geode.cache.Declarable;
-import org.apache.lucene.document.Document;
+  public Person(String name, int pageId) {
+    this.name = name;
+    this.email = name.replace(' ', '.') + "@pivotal.io";
+    this.revenue = pageId * 100;
+    this.address = "" + pageId + " NW Greenbrier PKWY, Portland OR 97006";
+    this.homepage = new Page(pageId);
+  }
 
-import org.apache.geode.annotations.Experimental;
-
-/**
- * An interface for writing the fields of an object into a lucene document
- */
-@Experimental
-public interface LuceneSerializer extends Declarable {
-
-  /**
-   * Add the fields of the given value to a set of documents
-   * 
-   * @param index lucene index
-   */
-  Collection<Document> toDocuments(LuceneIndex index, Object value);
+  @Override
+  public String toString() {
+    return "Person[name=" + name + ",email=" + email + ",revenue=" + revenue + ",address="
+ address
+        + ",homepage=" + homepage + "]";
+  }
 }

-- 
To stop receiving notification emails like this one, please contact
"commits@geode.apache.org" <commits@geode.apache.org>.

Mime
View raw message