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/21 00:12:49 UTC

[geode] 01/01: GEODE-3245: Flattening LuceneSerializer should support PDX values

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

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

commit f1e8e2f8129ac8c1e94650e0830e2f184e6b2fd2
Author: zhouxh <gz...@pivotal.io>
AuthorDate: Wed Oct 18 11:35:00 2017 -0700

    GEODE-3245: Flattening LuceneSerializer should support PDX values
---
 .../geode/cache/lucene/FlatFormatSerializer.java   |  37 ++-
 .../lucene/FlatFormatPdxSerializerJunitTest.java   | 169 +++++++++++
 .../lucene/FlatFormatSerializerJUnitTest.java      | 114 +++++---
 .../apache/geode/cache/lucene/LocalCacheRule.java  |  40 +++
 .../NestedObjectSeralizerIntegrationTest.java      | 324 ++++++++++++++++++++-
 5 files changed, 630 insertions(+), 54 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
index 7621356..9f3664e 100644
--- 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
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.cache.lucene;
 
+import java.lang.reflect.Array;
 import java.lang.reflect.Field;
 import java.util.Arrays;
 import java.util.Collection;
@@ -24,6 +25,7 @@ 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.geode.pdx.PdxInstance;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
 
@@ -97,8 +99,8 @@ public class FlatFormatSerializer implements LuceneSerializer {
     }
 
     if (fieldValue.getClass().isArray()) {
-      Object[] array = (Object[]) fieldValue;
-      for (Object item : array) {
+      for (int i = 0; i < Array.getLength(fieldValue); i++) {
+        Object item = Array.get(fieldValue, i);
         addFieldValueForNonCollectionObject(doc, indexedFieldName, item, tokenizedFields);
       }
     } else if (fieldValue instanceof Collection) {
@@ -122,17 +124,26 @@ public class FlatFormatSerializer implements LuceneSerializer {
   }
 
   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;
+    if (value instanceof PdxInstance) {
+      PdxInstance pdx = (PdxInstance) value;
+      Object fieldValue = null;
+      if (pdx.hasField(fieldName)) {
+        fieldValue = pdx.getField(fieldName);
+      }
+      return fieldValue;
+    } else {
+      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/test/java/org/apache/geode/cache/lucene/FlatFormatPdxSerializerJunitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/FlatFormatPdxSerializerJunitTest.java
new file mode 100644
index 0000000..d6a98cd
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/FlatFormatPdxSerializerJunitTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+
+import org.apache.geode.cache.lucene.internal.repository.serializer.SerializerTestHelper;
+import org.apache.geode.pdx.PdxInstance;
+import org.apache.geode.pdx.internal.PdxInstanceFactoryImpl;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexableField;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class FlatFormatPdxSerializerJunitTest {
+
+  private PdxInstance createPdxInstance() {
+    HashSet positions = new HashSet();
+
+    PdxInstanceFactoryImpl outForPosition1 =
+        (PdxInstanceFactoryImpl) PdxInstanceFactoryImpl.newCreator("dummy.PositionPdx", false);
+    outForPosition1.writeString("country", "USA");
+    outForPosition1.writeString("secId", "DELL");
+    outForPosition1.writeDouble("sharesOutstanding", 3000);
+    outForPosition1.writeInt("pid", 13);
+    outForPosition1.writeInt("portfolioId", 3);
+    // Identity Field.
+    outForPosition1.markIdentityField("secId");
+    PdxInstance position1_pdx = outForPosition1.create();
+
+    PdxInstanceFactoryImpl outForPositions1 =
+        (PdxInstanceFactoryImpl) PdxInstanceFactoryImpl.newCreator("dummy.PositionPdx", false);
+    outForPositions1.writeString("country", "USA");
+    outForPositions1.writeString("secId", "AAPL");
+    outForPositions1.writeDouble("sharesOutstanding", 5000);
+    outForPositions1.writeInt("pid", 15);
+    outForPositions1.writeInt("portfolioId", 3);
+    // Identity Field.
+    outForPositions1.markIdentityField("secId");
+    PdxInstance positions1_pdx = outForPositions1.create();
+
+    PdxInstanceFactoryImpl outForPositions2 =
+        (PdxInstanceFactoryImpl) PdxInstanceFactoryImpl.newCreator("dummy.PositionPdx", false);
+    outForPositions2.writeString("country", "USA");
+    outForPositions2.writeString("secId", "IBM");
+    outForPositions2.writeDouble("sharesOutstanding", 4000);
+    outForPositions2.writeInt("pid", 14);
+    outForPositions2.writeInt("portfolioId", 3);
+    // Identity Field.
+    outForPositions2.markIdentityField("secId");
+    PdxInstance positions2_pdx = outForPositions2.create();
+
+    positions.add(positions1_pdx);
+    positions.add(positions2_pdx);
+
+    PdxInstanceFactoryImpl out =
+        (PdxInstanceFactoryImpl) PdxInstanceFactoryImpl.newCreator("dummy.PortfolioPdx", false);
+    out.writeInt("ID", 3);
+    out.writeObject("position1", position1_pdx);
+    out.writeObject("positions", positions);
+    out.writeString("status", "active");
+    out.writeStringArray("names", new String[] {"aaa", "bbb", "ccc", "ddd"});
+    out.writeString("description", "John Denver");
+    out.writeLong("createTime", 0);
+    out.writeIntArray("intArr", new int[] {2001, 2017});
+    // Identity Field.
+    out.markIdentityField("ID");
+    PdxInstance pdx = out.create();
+    return pdx;
+  }
+
+  @Rule
+  public LocalCacheRule localCacheRule = new LocalCacheRule();
+
+  @Test
+  public void shouldParseTopLevelPdxIntArray() {
+    String[] fields = new String[] {"description", "status", "names", "intArr", "position1.country",
+        "position1.sharesOutstanding", "position1.secId", "positions.country",
+        "positions.sharesOutstanding", "positions.secId"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    PdxInstance pdx = createPdxInstance();
+
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, pdx, fields);
+    assertEquals(17, doc1.getFields().size());
+
+    IndexableField[] fieldsInDoc = doc1.getFields("intArr");
+    Collection<Object> results = getResultCollection(fieldsInDoc, true);
+    assertEquals(2, results.size());
+    assertTrue(results.contains(2001));
+    assertTrue(results.contains(2017));
+  }
+
+  @Test
+  public void shouldParseTopLevelPdxStringField() {
+    String[] fields = new String[] {"status"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    PdxInstance pdx = createPdxInstance();
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, pdx, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("status");
+    Collection<Object> results = getResultCollection(fieldsInDoc, false);
+    assertEquals(1, results.size());
+    assertTrue(results.contains("active"));
+  }
+
+  @Test
+  public void shouldParseSecondTopLevelPdxStringField() {
+    String[] fields = new String[] {"positions.secId"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    PdxInstance pdx = createPdxInstance();
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, pdx, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("positions.secId");
+    Collection<Object> results = getResultCollection(fieldsInDoc, false);
+    assertEquals(2, results.size());
+    assertTrue(results.contains("IBM"));
+    assertTrue(results.contains("AAPL"));
+  }
+
+  @Test
+  public void shouldParseSecondTopLevelPdxDoubleField() {
+    String[] fields = new String[] {"positions.sharesOutstanding"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+    PdxInstance pdx = createPdxInstance();
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, pdx, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("positions.sharesOutstanding");
+    Collection<Object> results = getResultCollection(fieldsInDoc, true);
+    assertEquals(2, results.size());
+    assertTrue(results.contains(5000.0));
+    assertTrue(results.contains(4000.0));
+  }
+
+  private Collection<Object> getResultCollection(IndexableField[] fieldsInDoc, boolean isNumeric) {
+    Collection<Object> results = new LinkedHashSet();
+    for (IndexableField field : fieldsInDoc) {
+      if (isNumeric) {
+        results.add((Object) field.numericValue());
+      } else {
+        results.add(field.stringValue());
+      }
+    }
+    return results;
+  }
+}
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
index 7c45879..2c0155d 100644
--- 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
@@ -17,7 +17,6 @@ package org.apache.geode.cache.lucene;
 import static org.junit.Assert.*;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -36,8 +35,17 @@ import org.junit.experimental.categories.Category;
 @Category(UnitTest.class)
 public class FlatFormatSerializerJUnitTest {
 
+  private HashSet<Person> createCollectionObjectContacts() {
+    Person contact1 = new Person("Tommi Jackson", new String[] {"5036330001", "5036330002"}, 1);
+    Person contact2 = new Person("Tommi2 Skywalker", new String[] {"5036330003", "5036330004"}, 2);
+    HashSet<Person> contacts1 = new HashSet();
+    contacts1.add(contact1);
+    contacts1.add(contact2);
+    return contacts1;
+  }
+
   @Test
-  public void shouldParseMultiLayerNestObject() {
+  public void verifyFieldCountsInDocument() {
     String[] fields = new String[] {"name", "phoneNumbers", "myHomePages.content", "contacts.name",
         "contacts.email", "contacts.phoneNumbers", "contacts.address", "contacts.revenue",
         "contacts.homepage.id", "contacts.homepage.title", "contacts.homepage.content",
@@ -58,48 +66,91 @@ public class FlatFormatSerializerJUnitTest {
     Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
     assertEquals(23, doc1.getFields().size());
     assertEquals("Tommy Jackson", doc1.getField("name").stringValue());
+  }
+
+  @Test
+  public void shouldQueryOnFieldInArrayObject() {
+    String[] fields = new String[] {"myHomePages.content"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+
+    Page[] myHomePages1 = new Page[] {new Page(131), new Page(132)};
+    Customer customer = new Customer("Tommy Jackson", null, null, myHomePages1);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
 
     IndexableField[] fieldsInDoc = doc1.getFields("myHomePages.content");
-    Collection<String> results = getStringResultCollection(fieldsInDoc);
+    Collection<Object> results = getResultCollection(fieldsInDoc, false);
     assertEquals(2, results.size());
     assertTrue(results.contains("Hello world no 131"));
     assertTrue(results.contains("Hello world no 132"));
+  }
+
+  @Test
+  public void shouldQueryOnFieldInCollectionObject() {
+    String[] fields = new String[] {"contacts.name"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
 
-    fieldsInDoc = doc1.getFields("contacts.name");
-    results = getStringResultCollection(fieldsInDoc);
+    HashSet<Person> contacts1 = createCollectionObjectContacts();
+    Customer customer = new Customer("Tommy Jackson", null, contacts1, null);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("contacts.name");
+    Collection<Object> results = getResultCollection(fieldsInDoc, false);
     assertEquals(2, results.size());
     assertTrue(results.contains("Tommi Jackson"));
     assertTrue(results.contains("Tommi2 Skywalker"));
+  }
 
-    fieldsInDoc = doc1.getFields("contacts.email");
-    results = getStringResultCollection(fieldsInDoc);
+  @Test
+  public void shouldQueryOnFieldWithAnalyzerInCollectionObject() {
+    String[] fields = new String[] {"contacts.email"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+
+    HashSet<Person> contacts1 = createCollectionObjectContacts();
+    Customer customer = new Customer("Tommy Jackson", null, contacts1, null);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("contacts.email");
+    Collection<Object> results = getResultCollection(fieldsInDoc, false);
     assertEquals(2, results.size());
     assertTrue(results.contains("Tommi.Jackson@pivotal.io"));
     assertTrue(results.contains("Tommi2.Skywalker@pivotal.io"));
+  }
 
-    fieldsInDoc = doc1.getFields("contacts.revenue");
-    Collection<Integer> intResults = getIntResultCollection(fieldsInDoc);
+  @Test
+  public void shouldQueryOnIntFieldInCollectionObject() {
+    String[] fields = new String[] {"contacts.revenue"};
+
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+
+    HashSet<Person> contacts1 = createCollectionObjectContacts();
+    Customer customer = new Customer("Tommy Jackson", null, contacts1, null);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("contacts.revenue");
+    Collection<Object> intResults = getResultCollection(fieldsInDoc, true);
     assertEquals(2, intResults.size());
     assertTrue(intResults.contains(100));
     assertTrue(intResults.contains(200));
+  }
 
-    fieldsInDoc = doc1.getFields("contacts.address");
-    results = getStringResultCollection(fieldsInDoc);
-    assertEquals(2, results.size());
-    assertTrue(results.contains("1 NW Greenbrier PKWY, Portland OR 97006"));
-    assertTrue(results.contains("2 NW Greenbrier PKWY, Portland OR 97006"));
+  @Test
+  public void shouldQueryOnFieldInThirdLevelObject() {
+    String[] fields = new String[] {"contacts.homepage.title"};
 
-    fieldsInDoc = doc1.getFields("contacts.homepage.title");
-    results = getStringResultCollection(fieldsInDoc);
+    FlatFormatSerializer serializer = new FlatFormatSerializer();
+
+    HashSet<Person> contacts1 = createCollectionObjectContacts();
+    Customer customer = new Customer("Tommy Jackson", null, contacts1, null);
+    Document doc1 = SerializerTestHelper.invokeSerializer(serializer, customer, fields);
+
+    IndexableField[] fieldsInDoc = doc1.getFields("contacts.homepage.title");
+    Collection<Object> results = getResultCollection(fieldsInDoc, false);
     assertEquals(2, results.size());
     assertTrue(results.contains("developer"));
     assertTrue(results.contains("manager"));
-
-    fieldsInDoc = doc1.getFields("contacts.homepage.content");
-    results = getStringResultCollection(fieldsInDoc);
-    assertEquals(2, results.size());
-    assertTrue(results.contains("Hello world no 1"));
-    assertTrue(results.contains("Hello world no 1"));
   }
 
   @Test
@@ -149,20 +200,15 @@ public class FlatFormatSerializerJUnitTest {
     assertEquals("Tommy Jackson", doc1.getField("name").stringValue());
   }
 
-  private Collection<String> getStringResultCollection(IndexableField[] fieldsInDoc) {
-    Collection<String> results = new LinkedHashSet();
+  private Collection<Object> getResultCollection(IndexableField[] fieldsInDoc, boolean isNumeric) {
+    Collection<Object> results = new LinkedHashSet();
     for (IndexableField field : fieldsInDoc) {
-      results.add(field.stringValue());
+      if (isNumeric) {
+        results.add((Object) field.numericValue());
+      } else {
+        results.add(field.stringValue());
+      }
     }
     return results;
   }
-
-  private Collection<Integer> getIntResultCollection(IndexableField[] fieldsInDoc) {
-    Collection<Integer> results = new LinkedHashSet();
-    for (IndexableField field : fieldsInDoc) {
-      results.add((Integer) field.numericValue());
-    }
-    return results;
-  }
-
 }
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LocalCacheRule.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LocalCacheRule.java
new file mode 100644
index 0000000..bc438a2
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LocalCacheRule.java
@@ -0,0 +1,40 @@
+/*
+ * 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.distributed.ConfigurationProperties.MCAST_PORT;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.internal.cache.InternalCache;
+import org.junit.rules.ExternalResource;
+
+public class LocalCacheRule extends ExternalResource {
+  private InternalCache cache;
+
+  @Override
+  protected void before() {
+    this.cache = (InternalCache) new CacheFactory().set(MCAST_PORT, "0").create();
+  }
+
+  @Override
+  protected void after() {
+    this.cache.close();
+  }
+
+  public InternalCache getCache() {
+    return cache;
+  }
+}
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
index 4893ca6..a5801d0 100644
--- 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
@@ -18,9 +18,12 @@ 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.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.logging.log4j.Logger;
@@ -30,6 +33,9 @@ import org.apache.geode.cache.lucene.test.Customer;
 import org.apache.geode.cache.lucene.test.Page;
 import org.apache.geode.cache.lucene.test.Person;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.pdx.PdxReader;
+import org.apache.geode.pdx.PdxSerializable;
+import org.apache.geode.pdx.PdxWriter;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.junit.Test;
@@ -40,8 +46,8 @@ 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;
+  LuceneQuery<Integer, Object> query;
+  PageableLuceneQueryResults<Integer, Object> results;
 
   private Region createRegionAndIndex() {
     luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer())
@@ -120,7 +126,6 @@ public class NestedObjectSeralizerIntegrationTest extends LuceneIntegrationTest
     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);
   }
@@ -337,7 +342,7 @@ public class NestedObjectSeralizerIntegrationTest extends LuceneIntegrationTest
 
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
         "Jackson2*", "name");
-    PageableLuceneQueryResults<Integer, Customer> results = query.findPages();
+    PageableLuceneQueryResults<Integer, Object> results = query.findPages();
     assertEquals(2, results.size());
     printResults(results);
   }
@@ -427,14 +432,319 @@ public class NestedObjectSeralizerIntegrationTest extends LuceneIntegrationTest
     assertEquals(0, results.size());
   }
 
-  private void printResults(PageableLuceneQueryResults<Integer, Customer> results) {
+  private void printResults(PageableLuceneQueryResults<Integer, Object> results) {
     if (results.size() > 0) {
       while (results.hasNext()) {
         results.next().stream().forEach(struct -> {
-          logger.info("Result is:" + struct.getValue());
+          if (logger.isDebugEnabled()) {
+            logger.debug("Result is:" + struct.getValue());
+          }
         });
       }
     }
   }
 
+  private Region createRegionAndIndexForPdxObject() {
+    luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer())
+        .addField("ID").addField("description").addField("status").addField("names")
+        .addField("position1.country").addField("position1.secId").addField("positions.secId")
+        .addField("positions.country").create(INDEX_NAME, REGION_NAME);
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    return region;
+  }
+
+  private void feedSomePdxObjects(Region region) throws InterruptedException {
+    SimplePortfolioPdx.resetCounter();
+    SimplePositionPdx.resetCounter();
+    for (int i = 1; i < 10; i++) {
+      SimplePortfolioPdx pdx = new SimplePortfolioPdx(i);
+      region.put("object-" + i, pdx);
+    }
+
+    luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, WAIT_FOR_FLUSH_TIME,
+        TimeUnit.MILLISECONDS);
+  }
+
+  @Test
+  public void queryOnTopLevelPdxField() throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexForPdxObject();
+    feedSomePdxObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "active",
+        "status");
+    results = query.findPages();
+    // even id number: active status; odd id number: inactive status
+    assertEquals(4, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnTopLevelPdxArrayField() throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexForPdxObject();
+    feedSomePdxObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "bbb AND ccc",
+        "names");
+    results = query.findPages();
+    // all the entries should be found
+    assertEquals(9, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnSecondLevelPdxCollectionField()
+      throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexForPdxObject();
+    feedSomePdxObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "NOVL",
+        "positions.secId");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+  }
+
+  @Test
+  public void queryOnSecondLevelPdxField() throws InterruptedException, LuceneQueryException {
+    Region region = createRegionAndIndexForPdxObject();
+    feedSomePdxObjects(region);
+
+    query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "DELL",
+        "position1.secId");
+    results = query.findPages();
+    assertEquals(1, results.size());
+    printResults(results);
+  }
+
+  public static class SimplePortfolioPdx implements Serializable, PdxSerializable {
+    private int ID;
+    public String description;
+    public long createTime;
+    public String status;
+    public String[] names = {"aaa", "bbb", "ccc", "ddd"};
+    public int[] intArr = {2001, 2017};
+
+    public SimplePositionPdx position1;
+    public HashSet positions = new HashSet();
+
+    public static int numInstance = 0;
+
+    /*
+     * public String getStatus(){ return status;
+     */
+    public int getID() {
+      return ID;
+    }
+
+    public long getCreateTime() {
+      return this.createTime;
+    }
+
+    public void setCreateTime(long time) {
+      this.createTime = time;
+    }
+
+    public HashSet getPositions() {
+      return positions;
+    }
+
+    public SimplePositionPdx getP1() {
+      return position1;
+    }
+
+    public boolean isActive() {
+      return status.equals("active");
+    }
+
+    public static String secIds[] = {"SUN", "IBM", "YHOO", "GOOG", "MSFT", "AOL", "APPL", "ORCL",
+        "SAP", "DELL", "RHAT", "NOVL", "HP"};
+
+    /* public no-arg constructor required for Deserializable */
+    public SimplePortfolioPdx() {
+      this.numInstance++;
+    }
+
+    public SimplePortfolioPdx(int i) {
+      this.numInstance++;
+      ID = i;
+      if (i % 2 == 0) {
+        description = "YYYY";
+      } else {
+        description = "XXXX";
+      }
+      status = i % 2 == 0 ? "active" : "inactive";
+      position1 = new SimplePositionPdx(secIds[SimplePositionPdx.cnt % secIds.length],
+          SimplePositionPdx.cnt * 1000);
+
+      positions.add(new SimplePositionPdx(secIds[SimplePositionPdx.cnt % secIds.length],
+          SimplePositionPdx.cnt * 1000));
+      positions.add(new SimplePositionPdx(secIds[SimplePositionPdx.cnt % secIds.length],
+          SimplePositionPdx.cnt * 1000));
+    }
+
+    public SimplePortfolioPdx(int i, int j) {
+      this(i);
+      this.position1.portfolioId = j;
+    }
+
+    public static void resetCounter() {
+      numInstance = 0;
+    }
+
+    private boolean eq(Object o1, Object o2) {
+      return o1 == null ? o2 == null : o1.equals(o2);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof SimplePortfolioPdx)) {
+        return false;
+      }
+      SimplePortfolioPdx p2 = (SimplePortfolioPdx) o;
+      return this.ID == p2.getID();
+    }
+
+    @Override
+    public int hashCode() {
+      return this.ID;
+    }
+
+
+    public String toString() {
+      String out = "SimplePortfolioPdx [ID=" + ID + " status=" + status + "\n ";
+      Iterator iter = positions.iterator();
+      while (iter.hasNext()) {
+        out += (SimplePositionPdx) (iter.next()) + ", ";
+      }
+      out += "\n P1:" + position1;
+      return out + "\n]";
+    }
+
+    /**
+     * Getter for property type.S
+     * 
+     * @return Value of property type.
+     */
+    public boolean boolFunction(String strArg) {
+      return "active".equals(strArg);
+    }
+
+    public int intFunction(int j) {
+      return j;
+    }
+
+    public String funcReturnSecId(Object o) {
+      return ((SimplePositionPdx) o).getSecId();
+    }
+
+    public long longFunction(long j) {
+      return j;
+    }
+
+    public void fromData(PdxReader in) {
+      this.ID = in.readInt("ID");
+      this.position1 = (SimplePositionPdx) in.readObject("position1");
+      this.positions = (HashSet) in.readObject("positions");
+      this.status = in.readString("status");
+      this.names = in.readStringArray("names");
+      this.description = in.readString("description");
+      this.createTime = in.readLong("createTime");
+      this.intArr = in.readIntArray("intArr");
+    }
+
+    public void toData(PdxWriter out) {
+      out.writeInt("ID", this.ID);
+      out.writeObject("position1", this.position1);
+      out.writeObject("positions", this.positions);
+      out.writeString("status", this.status);
+      out.writeStringArray("names", this.names);
+      out.writeString("description", this.description);
+      out.writeLong("createTime", this.createTime);
+      out.writeIntArray("intArr", this.intArr);
+      // Identity Field.
+      out.markIdentityField("ID");
+    }
+
+  }
+
+  public static class SimplePositionPdx implements Serializable, PdxSerializable, Comparable {
+    public String secId;
+    private String country = "USA";
+    private int pid;
+    private double sharesOutstanding;
+    public int portfolioId = 0;
+    public static int cnt = 0;
+
+    public static int numInstance = 0;
+
+    public SimplePositionPdx() {
+      this.numInstance++;
+    }
+
+    public SimplePositionPdx(String id, double out) {
+      secId = id;
+      sharesOutstanding = out;
+      pid = cnt++;
+
+      this.numInstance++;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof SimplePositionPdx))
+        return false;
+      return this.secId.equals(((SimplePositionPdx) o).secId);
+    }
+
+    @Override
+    public int hashCode() {
+      return this.secId.hashCode();
+    }
+
+    public String getSecId() {
+      return secId;
+    }
+
+    public static void resetCounter() {
+      cnt = 0;
+    }
+
+    public double getSharesOutstanding() {
+      return sharesOutstanding;
+    }
+
+    public String toString() {
+      return "SimplePositionPdx [secId=" + this.secId + " pid=" + this.pid + " out="
+          + this.sharesOutstanding + "]";
+    }
+
+    public void fromData(PdxReader in) {
+      this.country = in.readString("country");
+      this.secId = in.readString("secId");
+      this.sharesOutstanding = in.readDouble("sharesOutstanding");
+      this.pid = in.readInt("pid");
+      this.portfolioId = in.readInt("portfolioId");
+    }
+
+    public void toData(PdxWriter out) {
+      out.writeString("country", this.country);
+      out.writeString("secId", this.secId);
+      out.writeDouble("sharesOutstanding", this.sharesOutstanding);
+      out.writeInt("pid", this.pid);
+      out.writeInt("portfolioId", this.portfolioId);
+      // Identity Field.
+      out.markIdentityField("secId");
+    }
+
+
+    public int compareTo(Object o) {
+      if (o == this || ((SimplePositionPdx) o).secId.equals(this.secId)) {
+        return 0;
+      } else {
+        return this.pid < ((SimplePositionPdx) o).pid ? -1 : 1;
+      }
+
+    }
+  }
 }

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