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 2018/09/04 04:49:28 UTC

[geode] 01/01: GEODE-QueryProvider: added pointsConfigMap to support numeric query Adds in a test for int range lucene query (#2065) refactor the test cases to different package

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

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

commit c8d01af35099bd27788c5aed9c9226d0dd526076
Author: zhouxh <gz...@pivotal.io>
AuthorDate: Wed Jun 13 10:01:34 2018 -0700

    GEODE-QueryProvider: added pointsConfigMap to support numeric query
    Adds in a test for int range lucene query (#2065)
    refactor the test cases to different package
    
    Signed-off-by: Biju Kunjummen <bk...@pivotal.io>
---
 .../cache/lucene/test/SomeDomainTestObject.java    | 141 ++++++++++++
 .../cache/lucene/LuceneNumericQueryDUnitTest.java  | 240 +++++++++++++++++++++
 .../lucene/LuceneNumericQueryIntegrationTest.java  | 143 ++++++++++++
 .../geode/cache/lucene/FlatFormatSerializer.java   |  21 ++
 .../lucene/internal/LuceneIndexFactoryImpl.java    |   3 +-
 .../cache/lucene/internal/StringQueryProvider.java |  36 ++++
 .../serializer/HeterogeneousLuceneSerializer.java  |  18 ++
 .../repository/serializer/PdxLuceneSerializer.java |  21 ++
 .../serializer/ReflectionLuceneSerializer.java     |  25 +++
 9 files changed, 647 insertions(+), 1 deletion(-)

diff --git a/geode-lucene/geode-lucene-test/src/main/java/org/apache/geode/cache/lucene/test/SomeDomainTestObject.java b/geode-lucene/geode-lucene-test/src/main/java/org/apache/geode/cache/lucene/test/SomeDomainTestObject.java
new file mode 100644
index 0000000..8ccbc6d
--- /dev/null
+++ b/geode-lucene/geode-lucene-test/src/main/java/org/apache/geode/cache/lucene/test/SomeDomainTestObject.java
@@ -0,0 +1,141 @@
+package org.apache.geode.cache.lucene.test;
+
+import java.io.Serializable;
+import java.util.Date;
+import java.util.Objects;
+
+public class SomeDomainTestObject implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final String strField;
+  private final int intField;
+  private final long longField;
+  private final float floatField;
+  private final double doubleField;
+  private final Date dateField;
+
+  public SomeDomainTestObject(String strField, int intField, long longField, float floatField,
+      double doubleField, Date dateField) {
+    this.strField = strField;
+    this.intField = intField;
+    this.longField = longField;
+    this.floatField = floatField;
+    this.doubleField = doubleField;
+    this.dateField = dateField;
+  }
+
+  private SomeDomainTestObject(Builder builder) {
+    strField = builder.strField;
+    intField = builder.intField;
+    longField = builder.longField;
+    floatField = builder.floatField;
+    doubleField = builder.doubleField;
+    dateField = builder.dateField;
+  }
+
+  public String getStrField() {
+    return strField;
+  }
+
+  public int getIntField() {
+    return intField;
+  }
+
+  public long getLongField() {
+    return longField;
+  }
+
+  public float getFloatField() {
+    return floatField;
+  }
+
+  public Date getDateField() {
+    return dateField;
+  }
+
+  public double getDoubleField() {
+    return doubleField;
+  }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuffer sb = new StringBuffer("SomeDomain{");
+    sb.append("strField='").append(strField).append('\'');
+    sb.append(", height=").append(intField);
+    sb.append(", dateField=").append(dateField);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    SomeDomainTestObject that = (SomeDomainTestObject) o;
+    return intField == that.intField &&
+        longField == that.longField &&
+        Float.compare(that.floatField, floatField) == 0 &&
+        Double.compare(that.doubleField, doubleField) == 0 &&
+        Objects.equals(strField, that.strField) &&
+        Objects.equals(dateField, that.dateField);
+  }
+
+  @Override
+  public int hashCode() {
+
+    return Objects.hash(strField, intField, longField, floatField, doubleField, dateField);
+  }
+
+  public static final class Builder {
+    private String strField;
+    private int intField;
+    private long longField;
+    private float floatField;
+    private double doubleField;
+    private Date dateField;
+
+    private Builder() {}
+
+    public Builder withStrField(String val) {
+      strField = val;
+      return this;
+    }
+
+    public Builder withIntField(int val) {
+      intField = val;
+      return this;
+    }
+
+    public Builder withLongField(long val) {
+      longField = val;
+      return this;
+    }
+
+    public Builder withFloatField(float val) {
+      floatField = val;
+      return this;
+    }
+
+    public Builder withDoubleField(double val) {
+      doubleField = val;
+      return this;
+    }
+
+    public Builder withDateField(Date val) {
+      dateField = val;
+      return this;
+    }
+
+    public SomeDomainTestObject build() {
+      return new SomeDomainTestObject(this);
+    }
+  }
+}
diff --git a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneNumericQueryDUnitTest.java b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneNumericQueryDUnitTest.java
new file mode 100644
index 0000000..59d105c
--- /dev/null
+++ b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/LuceneNumericQueryDUnitTest.java
@@ -0,0 +1,240 @@
+/*
+ * 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.SERIALIZABLE_OBJECT_FILTER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Date;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.lucene.test.SomeDomainTestObject;
+import org.apache.geode.cache.query.Query;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.SelectResults;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.dunit.rules.ClientVM;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+
+@Category(LuceneDUnitTest.class)
+public class LuceneNumericQueryDUnitTest {
+
+  @ClassRule
+  public static ClusterStartupRule clusterStartupRule = new ClusterStartupRule();
+
+  private static MemberVM locator;
+  private static MemberVM server1;
+  private static ClientVM client1;
+
+  private static final SomeDomainTestObject someDomain1 =
+      SomeDomainTestObject.newBuilder()
+          .withStrField("strField1")
+          .withIntField(110)
+          .withLongField(10010L)
+          .withFloatField(101.25F)
+          .withDoubleField(101.25D)
+          .withDateField(
+              Date.from(LocalDateTime.parse("2001-01-01T00:00:00").toInstant(ZoneOffset.UTC)))
+          .build();
+
+  private static final SomeDomainTestObject someDomain2 =
+      SomeDomainTestObject.newBuilder()
+          .withStrField("strField2")
+          .withIntField(120)
+          .withLongField(10020L)
+          .withFloatField(201.25F)
+          .withDoubleField(201.25D)
+          .withDateField(
+              Date.from(LocalDateTime.parse("2002-01-01T00:00:00").toInstant(ZoneOffset.UTC)))
+          .build();
+
+  private static final SomeDomainTestObject someDomain3 =
+      SomeDomainTestObject.newBuilder()
+          .withStrField("strField3")
+          .withIntField(130)
+          .withLongField(10030L)
+          .withFloatField(301.25F)
+          .withDoubleField(301.25D)
+          .withDateField(
+              Date.from(LocalDateTime.parse("2003-01-01T00:00:00").toInstant(ZoneOffset.UTC)))
+          .build();
+
+
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    locator = clusterStartupRule.startLocatorVM(0);
+    final int locatorPort = locator.getPort();
+    server1 = clusterStartupRule
+        .startServerVM(1,
+            r -> r.withPDXPersistent()
+                .withPDXReadSerialized()
+                .withConnectionToLocator(locatorPort)
+                .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.cache.lucene.**"));
+
+
+    client1 =
+        clusterStartupRule.startClientVM(2, new Properties(),
+            (x -> x
+                .set(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.cache.lucene.**")
+                .addPoolLocator("localhost", locatorPort)));
+
+    server1.invoke(() -> {
+      InternalCache cache = ClusterStartupRule.getCache();
+      LuceneService luceneService = LuceneServiceProvider.get(cache);
+      luceneService.createIndexFactory()
+          .setFields("strField", "intField", "dateField", "longField", "floatField", "doubleField")
+          .create("idx1",
+              "/sampleregion");
+
+      cache.<String, SomeDomainTestObject>createRegionFactory(RegionShortcut.PARTITION)
+          .create("sampleregion");
+    });
+
+    client1.invoke(() -> {
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+
+      Region<String, SomeDomainTestObject> region =
+          clientCache
+              .<String, SomeDomainTestObject>createClientRegionFactory(ClientRegionShortcut.PROXY)
+              .create("sampleregion");
+      region.put(someDomain1.getStrField(), someDomain1);
+      region.put(someDomain2.getStrField(), someDomain2);
+      region.put(someDomain3.getStrField(), someDomain3);
+      LuceneService luceneService = LuceneServiceProvider.get(clientCache);
+      luceneService.waitUntilFlushed("idx1", "sampleregion", 30000, TimeUnit.MILLISECONDS);
+    });
+  }
+
+
+  @Test
+  public void testByIntegerRange() {
+    client1.invoke(() -> {
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+
+      QueryService queryService = clientCache.getQueryService();
+
+      Query query = queryService.newQuery("select * from /sampleregion");
+      SelectResults results = (SelectResults) query.execute();
+      assertThat(results).hasSize(3);
+
+      LuceneService luceneService = LuceneServiceProvider.get(clientCache);
+
+      LuceneQuery<String, SomeDomainTestObject> luceneQuery1 =
+          luceneService.createLuceneQueryFactory()
+              .create("idx1", "/sampleregion", "+strField=strField* +intField:[110 TO 120]",
+                  "strField");
+
+      assertThat(luceneQuery1.findKeys())
+          .containsExactlyInAnyOrder("strField1", "strField2");
+
+      assertThat(luceneQuery1.<SomeDomainTestObject>findValues())
+          .containsExactlyInAnyOrder(someDomain1, someDomain2);
+
+
+      LuceneQuery<String, SomeDomainTestObject> luceneQuery2 =
+          luceneService.createLuceneQueryFactory()
+              .create("idx1", "/sampleregion", "+strField=strField* +intField:[120 TO 130]",
+                  "strField");
+
+      assertThat(luceneQuery2.findKeys())
+          .containsExactlyInAnyOrder("strField2", "strField3");
+
+      assertThat(luceneQuery2.<SomeDomainTestObject>findValues())
+          .containsExactlyInAnyOrder(someDomain2, someDomain3);
+
+    });
+  }
+
+  @Test
+  public void testByLongRange() {
+    client1.invoke(() -> {
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+
+      LuceneService luceneService = LuceneServiceProvider.get(clientCache);
+
+      LuceneQuery<String, SomeDomainTestObject> luceneQuery1 =
+          luceneService.createLuceneQueryFactory()
+              .create("idx1", "/sampleregion", "+longField:[10010 TO 10025]", "strField");
+
+      assertThat(luceneQuery1.findKeys())
+          .containsExactlyInAnyOrder("strField1", "strField2");
+
+      assertThat(luceneQuery1.<SomeDomainTestObject>findValues())
+          .containsExactlyInAnyOrder(someDomain1, someDomain2);
+
+      LuceneQuery<String, SomeDomainTestObject> luceneQuery2 =
+          luceneService.createLuceneQueryFactory()
+              .create("idx1", "/sampleregion", "+longField:[10011 TO 10030]", "strField");
+
+      assertThat(luceneQuery2.findKeys())
+          .containsExactlyInAnyOrder("strField2", "strField3");
+
+      assertThat(luceneQuery2.<SomeDomainTestObject>findValues())
+          .containsExactlyInAnyOrder(someDomain2, someDomain3);
+    });
+  }
+
+  @Test
+  public void testByFloatRange() {
+    client1.invoke(() -> {
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+
+      LuceneService luceneService = LuceneServiceProvider.get(clientCache);
+
+      LuceneQuery<String, SomeDomainTestObject> luceneQuery1 =
+          luceneService.createLuceneQueryFactory()
+              .create("idx1", "/sampleregion", "+floatField:[100 TO 202]", "strField");
+
+      assertThat(luceneQuery1.findKeys())
+          .containsExactlyInAnyOrder("strField1", "strField2");
+
+      assertThat(luceneQuery1.<SomeDomainTestObject>findValues())
+          .containsExactlyInAnyOrder(someDomain1, someDomain2);
+    });
+  }
+
+  @Test
+  public void testByDoubleRange() {
+    client1.invoke(() -> {
+      ClientCache clientCache = ClusterStartupRule.getClientCache();
+
+      LuceneService luceneService = LuceneServiceProvider.get(clientCache);
+
+      LuceneQuery<String, SomeDomainTestObject> luceneQuery1 =
+          luceneService.createLuceneQueryFactory()
+              .create("idx1", "/sampleregion", "+doubleField:[100 TO 202]", "strField");
+
+      assertThat(luceneQuery1.findKeys())
+          .containsExactlyInAnyOrder("strField1", "strField2");
+
+      assertThat(luceneQuery1.<SomeDomainTestObject>findValues())
+          .containsExactlyInAnyOrder(someDomain1, someDomain2);
+    });
+  }
+}
diff --git a/geode-lucene/src/integrationTest/java/org/apache/geode/cache/lucene/LuceneNumericQueryIntegrationTest.java b/geode-lucene/src/integrationTest/java/org/apache/geode/cache/lucene/LuceneNumericQueryIntegrationTest.java
new file mode 100644
index 0000000..915047a
--- /dev/null
+++ b/geode-lucene/src/integrationTest/java/org/apache/geode/cache/lucene/LuceneNumericQueryIntegrationTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.SERIALIZABLE_OBJECT_FILTER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Date;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.lucene.test.SomeDomainTestObject;
+import org.apache.geode.cache.query.Query;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.SelectResults;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.rules.ServerStarterRule;
+
+@Category(LuceneIntegrationTest.class)
+public class LuceneNumericQueryIntegrationTest {
+
+  @Rule
+  public ServerStarterRule serverStarterRule = new ServerStarterRule()
+      .withPDXReadSerialized()
+      .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.cache.lucene.*")
+      .withAutoStart();
+
+  private InternalCache cache;
+
+
+
+  private static final SomeDomainTestObject someDomain1 =
+      SomeDomainTestObject.newBuilder()
+          .withStrField("strField1")
+          .withIntField(110)
+          .withLongField(10010L)
+          .withFloatField(101.25F)
+          .withDoubleField(101.25D)
+          .withDateField(
+              Date.from(LocalDateTime.parse("2001-01-01T00:00:00").toInstant(ZoneOffset.UTC)))
+          .build();
+
+  private static final SomeDomainTestObject someDomain2 =
+      SomeDomainTestObject.newBuilder()
+          .withStrField("strField2")
+          .withIntField(120)
+          .withLongField(10020L)
+          .withFloatField(201.25F)
+          .withDoubleField(201.25D)
+          .withDateField(
+              Date.from(LocalDateTime.parse("2002-01-01T00:00:00").toInstant(ZoneOffset.UTC)))
+          .build();
+
+  private static final SomeDomainTestObject someDomain3 =
+      SomeDomainTestObject.newBuilder()
+          .withStrField("strField3")
+          .withIntField(130)
+          .withLongField(10030L)
+          .withFloatField(301.25F)
+          .withDoubleField(301.25D)
+          .withDateField(
+              Date.from(LocalDateTime.parse("2003-01-01T00:00:00").toInstant(ZoneOffset.UTC)))
+          .build();
+
+
+  @Before
+  public void beforeTest() {
+    this.cache = serverStarterRule.getCache();
+    LuceneService luceneService = LuceneServiceProvider.get(this.cache);
+    luceneService.createIndexFactory()
+        .setFields("strField", "intField", "dateField", "longField", "floatField", "doubleField")
+        .create("idx1",
+            "/sampleregion");
+
+    this.cache.<String, SomeDomainTestObject>createRegionFactory(RegionShortcut.PARTITION)
+        .create("sampleregion");
+
+    Region<String, SomeDomainTestObject> region = this.cache.getRegion("/sampleregion");
+
+    region.put(someDomain1.getStrField(), someDomain1);
+    region.put(someDomain2.getStrField(), someDomain2);
+    region.put(someDomain3.getStrField(), someDomain3);
+  }
+
+
+  @Test
+  public void testByIntegerRange() throws Exception {
+
+    QueryService queryService = this.cache.getQueryService();
+
+    Query query = queryService.newQuery("select * from /sampleregion");
+    SelectResults results = (SelectResults) query.execute();
+    assertThat(results).hasSize(3);
+
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+
+    LuceneQuery<String, SomeDomainTestObject> luceneQuery1 =
+        luceneService.createLuceneQueryFactory()
+            .create("idx1", "/sampleregion", "+strField=strField* +intField:[110 TO 120]",
+                "strField");
+
+    assertThat(luceneQuery1.findKeys())
+        .containsExactlyInAnyOrder("strField1", "strField2");
+
+    assertThat(luceneQuery1.<SomeDomainTestObject>findValues())
+        .containsExactlyInAnyOrder(someDomain1, someDomain2);
+
+
+    LuceneQuery<String, SomeDomainTestObject> luceneQuery2 =
+        luceneService.createLuceneQueryFactory()
+            .create("idx1", "/sampleregion", "+strField=strField* +intField:[120 TO 130]",
+                "strField");
+
+    assertThat(luceneQuery2.findKeys())
+        .containsExactlyInAnyOrder("strField2", "strField3");
+
+    assertThat(luceneQuery2.<SomeDomainTestObject>findValues())
+        .containsExactlyInAnyOrder(someDomain2, someDomain3);
+
+  }
+
+
+}
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 f94f631..08cfcf4 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
@@ -16,15 +16,18 @@ package org.apache.geode.cache.lucene;
 
 import java.lang.reflect.Array;
 import java.lang.reflect.Field;
+import java.text.NumberFormat;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
 
 import org.apache.geode.cache.lucene.internal.repository.serializer.SerializerUtil;
 import org.apache.geode.internal.logging.LogService;
@@ -59,6 +62,7 @@ public class FlatFormatSerializer implements LuceneSerializer {
   private final ConcurrentMap<String, List<String>> tokenizedFieldCache = new ConcurrentHashMap<>();
 
   private static final Logger logger = LogService.getLogger();
+  private ConcurrentMap<String, PointsConfig> pointsConfigMap = new ConcurrentHashMap();
 
   /**
    * Recursively serialize each indexed field's value into a field of lucene document. The field
@@ -118,6 +122,7 @@ public class FlatFormatSerializer implements LuceneSerializer {
       Object fieldValue, List<String> tokenizedFields) {
     if (tokenizedFields.size() == 1) {
       SerializerUtil.addField(doc, indexedFieldName, fieldValue);
+      saveNumericFields(indexedFieldName, fieldValue);
     } else {
       addFieldValue(doc, indexedFieldName, fieldValue,
           tokenizedFields.subList(1, tokenizedFields.size()));
@@ -147,4 +152,20 @@ public class FlatFormatSerializer implements LuceneSerializer {
       }
     }
   }
+
+  private void saveNumericFields(String fieldName, Object fieldValue) {
+    Class<?> clazz = fieldValue.getClass();
+    if (Integer.class.equals(clazz) || Integer.TYPE.equals(clazz)
+        || Float.class.equals(clazz) || Float.TYPE.equals(clazz)
+        || Long.class.equals(clazz) || Long.TYPE.equals(clazz)
+        || Double.class.equals(clazz) || Double.TYPE.equals(clazz)) {
+      pointsConfigMap.computeIfAbsent(fieldName,
+          field -> new PointsConfig(NumberFormat.getInstance(), (Class<? extends Number>) clazz));
+    }
+  }
+
+  public Map<String, PointsConfig> getPointsConfigMap() {
+    return pointsConfigMap;
+  }
+
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
index 45d4fe1..0332954 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexFactoryImpl.java
@@ -22,11 +22,12 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
 import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.geode.cache.lucene.LuceneSerializer;
+import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 
 public class LuceneIndexFactoryImpl implements LuceneIndexFactory {
   private final LuceneServiceImpl service;
   private final Map<String, Analyzer> fields = new LinkedHashMap<String, Analyzer>();
-  private LuceneSerializer serializer;
+  private LuceneSerializer serializer = new HeterogeneousLuceneSerializer();
 
 
   public LuceneIndexFactoryImpl(final LuceneServiceImpl luceneService) {
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/StringQueryProvider.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/StringQueryProvider.java
index f0626b5..b8c0008 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/StringQueryProvider.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/StringQueryProvider.java
@@ -18,16 +18,22 @@ package org.apache.geode.cache.lucene.internal;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
 
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
 import org.apache.lucene.queryparser.flexible.standard.StandardQueryParser;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
 import org.apache.lucene.search.Query;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.lucene.FlatFormatSerializer;
 import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneQueryException;
 import org.apache.geode.cache.lucene.LuceneQueryProvider;
+import org.apache.geode.cache.lucene.LuceneSerializer;
+import org.apache.geode.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -66,6 +72,36 @@ public class StringQueryProvider implements LuceneQueryProvider, DataSerializabl
     if (luceneQuery == null) {
       LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
       StandardQueryParser parser = new StandardQueryParser(indexImpl.getAnalyzer());
+      // parser.getQueryConfigHandler().set(StandardQueryConfigHandler.ConfigurationKeys.DATE_RESOLUTION,
+      // DateTools.Resolution.MILLISECOND);
+      Map<String, PointsConfig> pointsConfigMap = null;
+      LuceneSerializer serializer = index.getLuceneSerializer();
+      if (serializer instanceof HeterogeneousLuceneSerializer) {
+        HeterogeneousLuceneSerializer heteroSerializer = (HeterogeneousLuceneSerializer) serializer;
+        pointsConfigMap = heteroSerializer.getPointsConfigMap();
+      } else if (serializer instanceof FlatFormatSerializer) {
+        FlatFormatSerializer flatFormatSerializer = (FlatFormatSerializer) serializer;
+        pointsConfigMap = flatFormatSerializer.getPointsConfigMap();
+      }
+      logger.info(serializer + ":" + Arrays.toString(indexImpl.getFieldNames()) + ":"
+          + parser.getDateResolution());
+      // pointsConfigMap.put("revenue", new PointsConfig(NumberFormat.getInstance(),
+      // Integer.class));
+      parser.setPointsConfigMap(pointsConfigMap);
+      // TODO1: is DateTools.Resolution optional?
+      // parser.setDateResolution(DateTools.Resolution.MILLISECOND);
+
+      // TODO2: HeterogeneousLuceneSerializer.getPointsConfigMap() calls putAll too often?
+      // TODO3: PdxLuceneSerializer.toDocuments().saveNumericFields's if (...) then
+      // computeIfAbsent(). Is it too expensive?
+      // TODO4: PdxLuceneSerializer and FlatFormatSerializer are using
+      // toDucuments().saveNumericFields() to get meta data, is it possible
+      // to move it to higher level for a generic method?
+      // TODO5: if a member is down, its meta data will be lost. How to recover it?
+      // TODO6: Short.class is not supported yet
+      // TODO7: Does PdxLuceneSerializer support nested field?
+      // TODO8: add example: numeric query __REGION_VALUE_FIELD:123
+      // TODO9: Can FlatFormatSerializer support pdx since it does not contain PdxLuceneSerializer
       parser.setAllowLeadingWildcard(true);
       try {
         luceneQuery = parser.parse(query, defaultField);
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
index 41281d5..74f05af 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
@@ -17,9 +17,12 @@ package org.apache.geode.cache.lucene.internal.repository.serializer;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
 
 import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneSerializer;
@@ -45,6 +48,8 @@ public class HeterogeneousLuceneSerializer implements LuceneSerializer {
   private Map<Class<?>, LuceneSerializer> mappers =
       new CopyOnWriteWeakHashMap<Class<?>, LuceneSerializer>();
 
+  private ConcurrentMap<String, PointsConfig> pointsConfigMap = new ConcurrentHashMap();
+
   private static final Logger logger = LogService.getLogger();
 
   public HeterogeneousLuceneSerializer() {
@@ -89,4 +94,17 @@ public class HeterogeneousLuceneSerializer implements LuceneSerializer {
     }
   }
 
+  public Map<String, PointsConfig> getPointsConfigMap() {
+    PdxLuceneSerializer pdxSerializer = (PdxLuceneSerializer) pdxMapper;
+    pointsConfigMap.putAll(pdxSerializer.getPointsConfigMap());
+
+    for (LuceneSerializer serializer : mappers.values()) {
+      if (serializer instanceof ReflectionLuceneSerializer) {
+        ReflectionLuceneSerializer reflectionSerializer = (ReflectionLuceneSerializer) serializer;
+        pointsConfigMap.putAll(reflectionSerializer.getPointsConfigMap());
+      }
+    }
+    return pointsConfigMap;
+  }
+
 }
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
index b644548..38bccca 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/PdxLuceneSerializer.java
@@ -15,11 +15,16 @@
 
 package org.apache.geode.cache.lucene.internal.repository.serializer;
 
+import java.text.NumberFormat;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
 
 import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneSerializer;
@@ -32,9 +37,24 @@ import org.apache.geode.pdx.PdxInstance;
 class PdxLuceneSerializer implements LuceneSerializer {
 
   private static final Logger logger = LogService.getLogger();
+  private ConcurrentMap<String, PointsConfig> pointsConfigMap = new ConcurrentHashMap();
 
   public PdxLuceneSerializer() {}
 
+  private void saveNumericFields(String fieldName, Object fieldValue) {
+    Class<?> clazz = fieldValue.getClass();
+    if (Integer.class.equals(clazz) || Integer.TYPE.equals(clazz) || Long.class.equals(clazz)
+        || Long.TYPE.equals(clazz) || Float.class.equals(clazz) || Float.TYPE.equals(clazz)
+        || Double.class.equals(clazz) || Double.TYPE.equals(clazz)) {
+      pointsConfigMap.computeIfAbsent(fieldName,
+          field -> new PointsConfig(NumberFormat.getInstance(), (Class<? extends Number>) clazz));
+    }
+  }
+
+  public Map<String, PointsConfig> getPointsConfigMap() {
+    return pointsConfigMap;
+  }
+
   @Override
   public Collection<Document> toDocuments(LuceneIndex index, Object value) {
     Document doc = new Document();
@@ -46,6 +66,7 @@ class PdxLuceneSerializer implements LuceneSerializer {
           continue;
         }
         SerializerUtil.addField(doc, field, fieldValue);
+        saveNumericFields(field, fieldValue);
       }
     }
     if (logger.isDebugEnabled()) {
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 947b2ad..50bcb9e 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
@@ -16,15 +16,20 @@
 package org.apache.geode.cache.lucene.internal.repository.serializer;
 
 import java.lang.reflect.Field;
+import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
 
 import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneSerializer;
@@ -37,6 +42,7 @@ import org.apache.geode.internal.logging.LogService;
 class ReflectionLuceneSerializer implements LuceneSerializer {
 
   private Field[] fields;
+  private ConcurrentMap<String, PointsConfig> pointsConfigMap = new ConcurrentHashMap();
 
   private static final Logger logger = LogService.getLogger();
 
@@ -54,6 +60,20 @@ class ReflectionLuceneSerializer implements LuceneSerializer {
         if (fieldSet.contains(field.getName()) && SerializerUtil.isSupported(type)) {
           field.setAccessible(true);
           foundFields.add(field);
+
+          if (Integer.class.equals(type) || Integer.TYPE.equals(type)) {
+            pointsConfigMap.put(field.getName(),
+                new PointsConfig(NumberFormat.getInstance(), Integer.class));
+          } else if (Long.class.equals(type) || Long.TYPE.equals(type)) {
+            pointsConfigMap.put(field.getName(),
+                new PointsConfig(NumberFormat.getInstance(), Long.class));
+          } else if (Float.class.equals(type) || Float.TYPE.equals(type)) {
+            pointsConfigMap.put(field.getName(),
+                new PointsConfig(NumberFormat.getInstance(), Float.class));
+          } else if (Double.class.equals(type) || Double.TYPE.equals(type)) {
+            pointsConfigMap.put(field.getName(),
+                new PointsConfig(NumberFormat.getInstance(), Double.class));
+          }
         }
       }
 
@@ -82,4 +102,9 @@ class ReflectionLuceneSerializer implements LuceneSerializer {
     }
     return Collections.singleton(doc);
   }
+
+  public Map<String, PointsConfig> getPointsConfigMap() {
+    return pointsConfigMap;
+  }
+
 }