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/10/09 04:36:26 UTC

[geode] branch feature/GEODE-3244 updated (90d3a4b -> 9b30794)

This is an automated email from the ASF dual-hosted git repository.

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


    omit 90d3a4b  GEODE-3244: provide a build in LuceneSerializer that flattens objects
     new 9b30794  GEODE-3244: provide a build in LuceneSerializer that flattens objects

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (90d3a4b)
            \
             N -- N -- N   refs/heads/feature/GEODE-3244 (9b30794)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../serializer/FlatFormatSerializer.java           | 46 +++++++++++-----------
 1 file changed, 24 insertions(+), 22 deletions(-)

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

[geode] 01/01: GEODE-3244: provide a build in LuceneSerializer that flattens objects

Posted by zh...@apache.org.
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 9b307944b72c60e328aec9844baca9895bdda935
Author: zhouxh <gz...@pivotal.io>
AuthorDate: Thu Oct 5 15:01:59 2017 -0700

    GEODE-3244: provide a build in LuceneSerializer that flattens objects
---
 .../geode/cache/lucene/LuceneSerializer.java       |   1 +
 .../repository/serializer/FieldsWithPrefix.java}   |  38 +--
 .../serializer/FlatFormatSerializer.java           | 257 +++++++++++++++++++++
 .../serializer/ReflectionLuceneSerializer.java     |  10 +-
 .../NestedObjectSeralizerIntegrationTest.java      | 208 +++++++++++++++++
 .../serializer/FlatFormatSerializerJUnitTest.java  |  41 ++++
 .../apache/geode/cache/lucene/test/Customer.java}  |  35 ++-
 .../org/apache/geode/cache/lucene/test/Page.java}  |  41 ++--
 .../apache/geode/cache/lucene/test/Person.java}    |  43 ++--
 9 files changed, 593 insertions(+), 81 deletions(-)

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/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FieldsWithPrefix.java
similarity index 56%
copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java
copy to geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FieldsWithPrefix.java
index 363867d..ccab403 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/internal/repository/serializer/FieldsWithPrefix.java
@@ -4,34 +4,34 @@
  * 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.internal.repository.serializer;
 
-package org.apache.geode.cache.lucene;
+import java.util.HashSet;
 
-import java.util.Collection;
+import org.apache.geode.cache.lucene.LuceneSerializer;
 
-import org.apache.geode.cache.Declarable;
-import org.apache.lucene.document.Document;
+public class FieldsWithPrefix {
+  LuceneSerializer serializer;
+  HashSet<String> topLevelFieldSet;
+  String prefix;
 
-import org.apache.geode.annotations.Experimental;
+  FieldsWithPrefix(HashSet<String> topLevelFieldSet, String prefix) {
+    // TODO initialize serializer
+    this.topLevelFieldSet = topLevelFieldSet;
+    this.prefix = prefix;
+  }
 
-/**
- * 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 "prefix is " + prefix + ", topLevelFieldSet is " + topLevelFieldSet + ":serializer is "
+        + serializer;
+  }
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializer.java
new file mode 100644
index 0000000..50c3389
--- /dev/null
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializer.java
@@ -0,0 +1,257 @@
+/*
+ * 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.internal.repository.serializer;
+
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+import org.apache.geode.cache.lucene.LuceneIndex;
+import org.apache.geode.cache.lucene.LuceneSerializer;
+import org.apache.geode.cache.lucene.LuceneService;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.util.concurrent.CopyOnWriteWeakHashMap;
+import org.apache.geode.pdx.PdxInstance;
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexableField;
+
+public class FlatFormatSerializer implements LuceneSerializer {
+
+  /**
+   * keep class to defined fields mapping for each class For example, Customer object based on
+   * indexed fields name, contact.name, contact.address, contact.homepage.title will become
+   * <Customer.class, (<name, contact>, prefix=null)>; <Person.class, (<name, address,
+   * homepage.title>, prefix=contact)>, <Page.class, (<title>, prefix=homepage)>
+   */
+  public HashMap<Class<?>, FieldsWithPrefix> classToFieldsMapping = new HashMap<>();
+
+  private static final Logger logger = LogService.getLogger();
+
+  public FlatFormatSerializer() {}
+
+  @Override
+  public Collection<Document> toDocuments(LuceneIndex index, Object value) {
+    if (value == null) {
+      return Collections.emptyList();
+    }
+
+    LuceneSerializer mapper = getFieldMapper(value, index.getFieldNames());
+
+    Collection<Document> docs = mapper.toDocuments(index, value);
+    if (logger.isDebugEnabled()) {
+      logger.debug("FlatFormatSerializer.toDocuments:" + docs);
+    }
+
+    return docs;
+  }
+
+  /**
+   * Get the field mapper based on the type of the given object.
+   */
+  public LuceneSerializer getFieldMapper(Object value, String[] indexedFields) {
+    if (value instanceof PdxInstance) {
+      throw new IllegalStateException("FlatFormatSerializer does not support PDX");
+    } else {
+      return createClassToFieldsMapping(value, indexedFields, null);
+    }
+  }
+
+  /*
+   * create value's class to its indexed fields mapping
+   */
+  private synchronized LuceneSerializer createClassToFieldsMapping(Object value,
+      String[] indexedFields, String prefix) {
+    Class<?> clazz = value.getClass();
+    FieldsWithPrefix fieldsWithPrefix = classToFieldsMapping.get(clazz);
+    if (fieldsWithPrefix == null) {
+      if (SerializerUtil.supportedPrimitiveTypes().contains(clazz)
+          && Arrays.asList(indexedFields).contains(LuceneService.REGION_VALUE_FIELD)) {
+        final PrimitiveSerializer primitiveSerializer = new PrimitiveSerializer();
+        final HashSet regionValueFields = new HashSet();
+        regionValueFields.add(LuceneService.REGION_VALUE_FIELD);
+        final FieldsWithPrefix regionValueFieldsWithPrefix =
+            new FieldsWithPrefix(regionValueFields, null);
+        regionValueFieldsWithPrefix.serializer = primitiveSerializer;
+        SerializerUtil.supportedPrimitiveTypes().stream()
+            .forEach(type -> classToFieldsMapping.put(type, regionValueFieldsWithPrefix));
+        return primitiveSerializer;
+      }
+
+      HashSet<String> topLevelFieldSet = new HashSet<String>();
+      fieldsWithPrefix = new FieldsWithPrefix(topLevelFieldSet, prefix);
+      classToFieldsMapping.put(clazz, fieldsWithPrefix);
+    } else {
+      return fieldsWithPrefix.serializer;
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug("createClassToFieldsMapping:found an undefined class, object value is " + value
+          + ":indexedFields are " + Arrays.toString(indexedFields) + ", prefix is " + prefix);
+    }
+
+    // parent field (e.g. contact) to child fields is a 1:N mapping,
+    // such as contact to <name, address, homepage.title>
+    HashMap<String, HashSet<String>> parentToChildFieldsMapping = new HashMap();
+    for (String f : indexedFields) {
+      int firstDot = f.indexOf(".");
+      if (firstDot != -1) {
+        // found a parent-child field
+        String parentFieldName = f.substring(0, firstDot);
+        String childFieldName = f.substring(firstDot + 1);
+        HashSet<String> childFieldSet = parentToChildFieldsMapping.get(parentFieldName);
+        if (childFieldSet == null) {
+          childFieldSet = new HashSet<String>();
+          parentToChildFieldsMapping.put(parentFieldName, childFieldSet);
+        }
+        childFieldSet.add(childFieldName);
+
+        fieldsWithPrefix.topLevelFieldSet.add(parentFieldName);
+      } else {
+        // only found a top level field
+        fieldsWithPrefix.topLevelFieldSet.add(f);
+      }
+    }
+    if (logger.isDebugEnabled()) {
+      logger.info("createClassToFieldsMapping:topLevelFieldSet is "
+          + fieldsWithPrefix.topLevelFieldSet + ":prefix is " + prefix
+          + ":parentToChildFieldsMap is " + parentToChildFieldsMapping);
+    }
+
+    String[] fields = fieldsWithPrefix.topLevelFieldSet
+        .toArray(new String[fieldsWithPrefix.topLevelFieldSet.size()]);
+    fieldsWithPrefix.serializer = new FlatFormatReflectionLuceneSerializer(clazz, fields, prefix);
+
+    // Now we've got a topLevelFieldSet, such as <name, contact> and a parentToChildFieldsMapping
+    // such as <contact, <name, address, homepage.title>>
+    // for each parent field, i.e. contact, recursively process with its childFieldSet, i.e. <name,
+    // address, homepage.title>
+    for (Map.Entry<String, HashSet<String>> entry : parentToChildFieldsMapping.entrySet()) {
+      String parentFieldName = entry.getKey();
+      HashSet<String> childFieldSet = entry.getValue();
+      String[] childFields = childFieldSet.toArray(new String[childFieldSet.size()]);
+
+      // parentFieldName is to get childValue
+      Object childValue = getFieldValue(value, parentFieldName);
+
+      // now we've got a child object, find class-fields mapping for its fields
+      // this time, the parameters are (personObject, <name, address, homepage.title>)
+      createClassToFieldsMapping(childValue, childFields, parentFieldName);
+    }
+
+    return fieldsWithPrefix.serializer;
+  }
+
+  private Object getFieldValue(Object parentValue, String parentFieldName) {
+    Class<?> parentClazz = parentValue.getClass();
+    try {
+      Field field = parentClazz.getDeclaredField(parentFieldName);
+      field.setAccessible(true);
+      return field.get(parentValue);
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  class FlatFormatReflectionLuceneSerializer extends ReflectionLuceneSerializer {
+
+    String prefix;
+
+    public FlatFormatReflectionLuceneSerializer(Class<? extends Object> clazz,
+        String[] indexedFields, String prefix) {
+      super(clazz, indexedFields);
+      this.prefix = prefix;
+    }
+
+    public Document toDocument(String parentPrefix, Object value) {
+      Document parentDoc = new Document();
+      String wholePrefix = getWholePrefix(parentPrefix, prefix);
+
+      for (Field field : fields) {
+        try {
+          field.setAccessible(true);
+          Object fieldValue = field.get(value);
+          if (fieldValue == null) {
+            continue;
+          }
+          Class<?> fieldType = field.getType();
+
+          if (classToFieldsMapping.containsKey(fieldType)) {
+            // for each object field value with class defined in classToFieldsMapping, serialize it
+            // to a document
+            // then add the document's indexableFields into parentDoc
+            FieldsWithPrefix fieldsWithPrefix = classToFieldsMapping.get(fieldType);
+            HashSet<String> topLevelFieldSet = fieldsWithPrefix.topLevelFieldSet;
+            String[] topLevelFieldNames =
+                topLevelFieldSet.toArray(new String[topLevelFieldSet.size()]);
+
+            FlatFormatReflectionLuceneSerializer childSerializer =
+                (FlatFormatReflectionLuceneSerializer) getFieldMapper(fieldValue,
+                    topLevelFieldNames);
+            Document childDoc = childSerializer.toDocument(wholePrefix, fieldValue);
+            for (IndexableField oneDocField : childDoc.getFields()) {
+              parentDoc.add(oneDocField);
+            }
+          } else {
+            SerializerUtil.addField(parentDoc,
+                (wholePrefix == null ? field.getName() : wholePrefix + "." + field.getName()),
+                fieldValue);
+          }
+        } catch (IllegalArgumentException | IllegalAccessException e) {
+          if (logger.isDebugEnabled()) {
+            logger.debug("FlatFormatReflectionLuceneSerializer.toDocuments encountered exception",
+                e);
+          }
+        }
+      }
+      if (logger.isDebugEnabled()) {
+        logger.debug("FlatFormatReflectionLuceneSerializer.toDocuments:" + parentDoc);
+      }
+      return parentDoc;
+    }
+
+    private String getWholePrefix(String parentPrefix, String localPrefix) {
+      if (parentPrefix != null) {
+        return parentPrefix + "." + localPrefix;
+      } else {
+        return localPrefix;
+      }
+    }
+
+    @Override
+    public Collection<Document> toDocuments(LuceneIndex index, Object value) {
+      Document parentDoc = toDocument(prefix, value);
+      if (logger.isDebugEnabled()) {
+        logger.debug("FlatFormatReflectionLuceneSerializer.toDocuments:" + parentDoc);
+      }
+      return Collections.singleton(parentDoc);
+    }
+
+    protected boolean isSupported(Class<?> type) {
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "FlatFormatReflectionLuceneSerializer:prefix is " + prefix + ", fields are "
+          + Arrays.toString(fields);
+    }
+  }
+}
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
index 5b9298b..176d4e1 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java
@@ -35,9 +35,9 @@ import org.apache.geode.internal.logging.LogService;
  */
 class ReflectionLuceneSerializer implements LuceneSerializer {
 
-  private Field[] fields;
+  protected Field[] fields;
 
-  private static final Logger logger = LogService.getLogger();
+  protected static final Logger logger = LogService.getLogger();
 
   public ReflectionLuceneSerializer(Class<? extends Object> clazz, String[] indexedFields) {
     Set<String> fieldSet = new HashSet<String>();
@@ -50,7 +50,7 @@ class ReflectionLuceneSerializer implements LuceneSerializer {
     while (clazz != Object.class) {
       for (Field field : clazz.getDeclaredFields()) {
         Class<?> type = field.getType();
-        if (fieldSet.contains(field.getName()) && SerializerUtil.isSupported(type)) {
+        if (fieldSet.contains(field.getName()) && isSupported(type)) {
           field.setAccessible(true);
           foundFields.add(field);
         }
@@ -62,6 +62,10 @@ class ReflectionLuceneSerializer implements LuceneSerializer {
     this.fields = foundFields.toArray(new Field[foundFields.size()]);
   }
 
+  protected boolean isSupported(Class<?> type) {
+    return SerializerUtil.isSupported(type);
+  }
+
   @Override
   public Collection<Document> toDocuments(LuceneIndex index, Object value) {
     Document doc = new Document();
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..61abba4
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java
@@ -0,0 +1,208 @@
+/*
+ * 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.internal.repository.serializer.FlatFormatSerializer;
+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();
+
+  @Test
+  public void queryOnNestedObject() throws InterruptedException, LuceneQueryException {
+    // define index on nested objects
+    luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer())
+        .addField("name").addField("contact.name").addField("contact.email", new KeywordAnalyzer())
+        .addField("contact.address").addField("contact.homepage.content")
+        .addField(LuceneService.REGION_VALUE_FIELD).create(INDEX_NAME, REGION_NAME);
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+
+    // insert a few records
+    region.put("object-13", new Customer("Tommy Jaskcon"));
+    region.put("object-14", new Customer("Johnny Jackson"));
+    region.put("object-15", new Customer("Johnny Jackson2"));
+    region.put("object-16", new Customer("Johnny Jackson21"));
+    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);
+
+    // query-1: contact.name
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "contact.name:jackson2*", "name");
+    PageableLuceneQueryResults<Integer, Customer> results = query.findPages();
+    assertEquals(2, results.size());
+    printResults(results);
+
+    // query-2: name
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnni Jackson\"", "contact.name");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnni Jackson\"", "name");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(0, results.size());
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnny Jackson\"", "name");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+
+    // 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);
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Johnni.Jackson2@pivotal.io", "email");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(0, results.size());
+
+    // query-4: contact.address with standard analyzer
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006",
+        "contact.address");
+    results = query.findPages();
+    assertEquals(4, results.size());
+    printResults(results);
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006",
+        "address");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(0, results.size());
+
+    // query-4: contact.homepage.title
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "contact.homepage.content:Hello", "name");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(4, results.size());
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "Hello",
+        "content");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(0, results.size());
+
+    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 indexOnNonExistFields() throws InterruptedException, LuceneQueryException {
+    // define index on nested objects
+    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);
+
+    // insert a few records
+    region.put("object-13", new Customer("Tommy Jaskcon"));
+    region.put("object-14", new Customer("Johnny Jackson"));
+    region.put("object-15", new Customer("Johnny Jackson2"));
+    region.put("object-16", new Customer("Johnny Jackson21"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, WAIT_FOR_FLUSH_TIME,
+        TimeUnit.MILLISECONDS);
+
+    // query-1: contact.name
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Jackson2*", "name");
+    PageableLuceneQueryResults<Integer, Customer> results = query.findPages();
+    assertEquals(2, results.size());
+    printResults(results);
+
+    // query-2: name
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnni Jackson\"", "contact.name");
+    results = query.findPages();
+    assertEquals(0, results.size());
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "\"Johnny Jackson\"", "name");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+
+    // query-3: contact.email with KeywordAnalyzer
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "Johnni.Jackson2@pivotal.io", "contact.email");
+    results = query.findPages();
+    assertEquals(0, results.size());
+
+    // query-4: contact.address with standard analyzer
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006",
+        "contact.address");
+    results = query.findPages();
+    assertEquals(0, results.size());
+
+    // query-4: contact.homepage.title
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
+        "contact.homepage.content:Hello", "name");
+    results = query.findPages();
+    printResults(results);
+    assertEquals(0, results.size());
+
+    // query-5: missing fields
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "*",
+        "contact.missing");
+    results = query.findPages();
+    assertEquals(0, results.size());
+    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/FlatFormatSerializerJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializerJUnitTest.java
new file mode 100644
index 0000000..acec39f
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializerJUnitTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.internal.repository.serializer;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+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"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+  }
+
+  @Test
+  public void shouldNotParseNestedObjectWithoutFields() {
+    String[] fields =
+        new String[] {"name", "contact", "contact.homepage", "contact.missing", "missing2"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+  }
+
+}
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..440e0d7 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,31 @@
  * 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) {
+    this.name = name;
 
-import org.apache.geode.annotations.Experimental;
+    // purposely use Tommi instead of Tommy, Johnni instead of Johnny in contact.name
+    this.contact = new Person(name.replace('y', 'i'));
+  }
 
-/**
- * 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..dc7b1cf 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,37 @@
  * 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) {
+    this.name = name;
+    this.email = name.replace(' ', '.') + "@pivotal.io";
+    int length = name.length();
+    this.revenue = length * 100;
+    this.address = "" + length + " NW Greenbrier PKWY, Portland OR 97006";
+    this.homepage = new Page(length);
+  }
 
-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" <co...@geode.apache.org>.