You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by nn...@apache.org on 2017/03/16 21:14:08 UTC

geode git commit: GEODE-2655: Added DUnit tests for lucene indexes on mixed objects.

Repository: geode
Updated Branches:
  refs/heads/develop 41e14b528 -> 426c2fcd3


GEODE-2655: Added DUnit tests for lucene indexes on mixed objects.

	* Lucene must be able to index on the common field name of mixed objects
	* Lucene indexes must also work when there are mixed objects with no common field names.
	* Lucene indexes work with different objects, different data type, but same field name.

	This closes #424


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

Branch: refs/heads/develop
Commit: 426c2fcd3f27800d41ab0dfdd596fe5d1cfaf74f
Parents: 41e14b5
Author: nabarun <nn...@pivotal.io>
Authored: Tue Mar 14 00:02:51 2017 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Thu Mar 16 14:13:25 2017 -0700

----------------------------------------------------------------------
 .../lucene/LuceneQueriesIntegrationTest.java    |  56 ---
 .../cache/lucene/MixedObjectIndexDUnitTest.java | 458 +++++++++++++++++++
 .../cache/lucene/test/LuceneTestUtilities.java  |  51 +++
 3 files changed, 509 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/426c2fcd/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
index 5b85c19..9db0a5e 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -17,9 +17,6 @@ package org.apache.geode.cache.lucene;
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -39,20 +36,15 @@ import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.core.LowerCaseFilter;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.analysis.util.CharTokenizer;
-import org.apache.lucene.document.IntPoint;
-import org.apache.lucene.search.Query;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
 
-import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.test.TestObject;
-import org.apache.geode.internal.DataSerializableFixedID;
-import org.apache.geode.internal.Version;
 import org.apache.geode.pdx.JSONFormatter;
 import org.apache.geode.pdx.PdxInstance;
 import org.apache.geode.test.junit.categories.IntegrationTest;
@@ -515,52 +507,4 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
       return new TokenStreamComponents(tokenizer, filter);
     }
   }
-
-  public static class IntRangeQueryProvider
-      implements LuceneQueryProvider, DataSerializableFixedID {
-    public static final short LUCENE_INT_RANGE_QUERY_PROVIDER = 2177;
-    String fieldName;
-    int lowerValue;
-    int upperValue;
-
-    private transient Query luceneQuery;
-
-    public IntRangeQueryProvider(String fieldName, int lowerValue, int upperValue) {
-      this.fieldName = fieldName;
-      this.lowerValue = lowerValue;
-      this.upperValue = upperValue;
-    }
-
-    @Override
-    public Version[] getSerializationVersions() {
-      return null;
-    }
-
-    @Override
-    public int getDSFID() {
-      return LUCENE_INT_RANGE_QUERY_PROVIDER;
-    }
-
-    @Override
-    public void toData(DataOutput out) throws IOException {
-      DataSerializer.writeString(fieldName, out);
-      out.writeInt(lowerValue);
-      out.writeInt(upperValue);
-    }
-
-    @Override
-    public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-      fieldName = DataSerializer.readString(in);
-      lowerValue = in.readInt();
-      upperValue = in.readInt();
-    }
-
-    @Override
-    public Query getQuery(LuceneIndex index) throws LuceneQueryException {
-      if (luceneQuery == null) {
-        luceneQuery = IntPoint.newRangeQuery(fieldName, lowerValue, upperValue);
-      }
-      return luceneQuery;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/426c2fcd/geode-lucene/src/test/java/org/apache/geode/cache/lucene/MixedObjectIndexDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/MixedObjectIndexDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/MixedObjectIndexDUnitTest.java
new file mode 100644
index 0000000..affccde
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/MixedObjectIndexDUnitTest.java
@@ -0,0 +1,458 @@
+/*
+ * 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.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.test.dunit.SerializableRunnableIF;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.stream.IntStream;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+@Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class MixedObjectIndexDUnitTest extends LuceneQueriesAccessorBase {
+
+  protected RegionTestableType[] getPartitionRegionTypes() {
+    return new RegionTestableType[] {RegionTestableType.PARTITION,
+        RegionTestableType.PARTITION_PERSISTENT, RegionTestableType.PARTITION_REDUNDANT,
+        RegionTestableType.PARTITION_REDUNDANT_PERSISTENT};
+
+  }
+
+  @Test
+  @Parameters(method = "getPartitionRegionTypes")
+  public void luceneCanIndexFieldsWithSameNameButInDifferentObjects(
+      RegionTestableType regionTestType) {
+    SerializableRunnableIF createIndex = getSerializableRunnableIFCreateIndexOnFieldText();
+
+    dataStore1.invoke(() -> initDataStore(createIndex, regionTestType));
+    dataStore2.invoke(() -> initDataStore(createIndex, regionTestType));
+
+    accessor.invoke(() -> initDataStore(createIndex, regionTestType));
+
+    accessor.invoke(() -> {
+      Region region = getCache().getRegion(REGION_NAME);
+      IntStream.range(0, NUM_BUCKETS).forEach(i -> region.put(i, new TestObject("hello world")));
+      IntStream.range(NUM_BUCKETS, (2 * NUM_BUCKETS))
+          .forEach(i -> region.put(i, new TestObjectWithSameFieldName("hello world 2")));
+    });
+
+    waitForFlushBeforeExecuteTextSearch(accessor, 60000);
+
+    accessor.invoke(() -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      LuceneQuery luceneQuery = luceneService.createLuceneQueryFactory().setResultLimit(100)
+          .create(INDEX_NAME, REGION_NAME, "world", "text");
+      List resultList = luceneQuery.findResults();
+      int objectType_1_count = 0;
+      int objectType_2_count = 0;
+      for (Object luceneResultStruct : resultList) {
+        if (((LuceneResultStruct) luceneResultStruct).getValue() instanceof TestObject) {
+          objectType_1_count++;
+        } else if (((LuceneResultStruct) luceneResultStruct)
+            .getValue() instanceof TestObjectWithSameFieldName) {
+          objectType_2_count++;
+        }
+      }
+      assertEquals(NUM_BUCKETS, objectType_1_count);
+      assertEquals(NUM_BUCKETS, objectType_2_count);
+    });
+  }
+
+  @Test
+  @Parameters(method = "getPartitionRegionTypes")
+  public void luceneMustIndexFieldsWithMixedObjects(RegionTestableType regionTestableType) {
+    SerializableRunnableIF createIndexOnTextAndDataField =
+        getSerializableRunnableIFCreateIndexOnFieldData();
+
+    dataStore1.invoke(() -> initDataStore(createIndexOnTextAndDataField, regionTestableType));
+    dataStore2.invoke(() -> initDataStore(createIndexOnTextAndDataField, regionTestableType));
+
+    accessor.invoke(() -> initDataStore(createIndexOnTextAndDataField, regionTestableType));
+
+    accessor.invoke(() -> {
+      Region region = getCache().getRegion(REGION_NAME);
+      IntStream.range(0, NUM_BUCKETS).forEach(i -> region.put(i, new TestObject("hello world")));
+      IntStream.range(NUM_BUCKETS, 2 * NUM_BUCKETS)
+          .forEach(i -> region.put(i, new TestObjectWithSameFieldName("hello world")));
+      IntStream.range(2 * NUM_BUCKETS, 3 * NUM_BUCKETS)
+          .forEach(i -> region.put(i, new TestObjectWithNoCommonField("hello world")));
+    });
+
+
+    waitForFlushBeforeExecuteTextSearch(accessor, 60000);
+
+    accessor.invoke(() -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      LuceneQuery luceneQueryForTextField = luceneService.createLuceneQueryFactory()
+          .setResultLimit(100).create(INDEX_NAME, REGION_NAME, "world", "text");
+      List luceneResults = luceneQueryForTextField.findResults();
+      validateObjectResultCounts(luceneResults, TestObject.class, NUM_BUCKETS,
+          TestObjectWithSameFieldName.class, NUM_BUCKETS, TestObjectWithNoCommonField.class, 0);
+
+      luceneQueryForTextField = luceneService.createLuceneQueryFactory().setResultLimit(100)
+          .create(INDEX_NAME, REGION_NAME, "world", "data");
+      luceneResults = luceneQueryForTextField.findResults();
+      validateObjectResultCounts(luceneResults, TestObject.class, 0,
+          TestObjectWithSameFieldName.class, 0, TestObjectWithNoCommonField.class, NUM_BUCKETS);
+    });
+  }
+
+
+
+  @Test
+  @Parameters(method = "getPartitionRegionTypes")
+  public void luceneMustIndexFieldsWithTheSameNameInARegionWithMixedObjects(
+      RegionTestableType regionTestableType) {
+    SerializableRunnableIF createIndexOnTextField =
+        getSerializableRunnableIFCreateIndexOnFieldText();
+
+    dataStore1.invoke(() -> initDataStore(createIndexOnTextField, regionTestableType));
+    dataStore2.invoke(() -> initDataStore(createIndexOnTextField, regionTestableType));
+
+    accessor.invoke(() -> initDataStore(createIndexOnTextField, regionTestableType));
+
+    accessor.invoke(() -> {
+      Region region = getCache().getRegion(REGION_NAME);
+      IntStream.range(0, NUM_BUCKETS).forEach(i -> region.put(i, new TestObject("hello world")));
+      IntStream.range(NUM_BUCKETS, 2 * NUM_BUCKETS)
+          .forEach(i -> region.put(i, new TestObjectWithSameFieldName("hello world")));
+      IntStream.range(2 * NUM_BUCKETS, 3 * NUM_BUCKETS)
+          .forEach(i -> region.put(i, new TestObjectWithNoCommonField("hello world")));
+    });
+
+    waitForFlushBeforeExecuteTextSearch(accessor, 60000);
+
+    accessor.invoke(() -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      LuceneQuery luceneQueryForTextField = luceneService.createLuceneQueryFactory()
+          .setResultLimit(100).create(INDEX_NAME, REGION_NAME, "world", "text");
+      List luceneResults = luceneQueryForTextField.findResults();
+      validateObjectResultCounts(luceneResults, TestObject.class, NUM_BUCKETS,
+          TestObjectWithSameFieldName.class, NUM_BUCKETS, TestObjectWithNoCommonField.class, 0);
+    });
+
+  }
+
+  @Test
+  @Parameters(method = "getPartitionRegionTypes")
+  public void luceneMustIndexFieldsWithTheSameNameDifferentDataTypeInARegionWithMixedObjects(
+      RegionTestableType regionTestableType) {
+    SerializableRunnableIF createIndexOnTextField =
+        getSerializableRunnableIFCreateIndexOnFieldText();
+
+    dataStore1.invoke(() -> initDataStore(createIndexOnTextField, regionTestableType));
+    dataStore2.invoke(() -> initDataStore(createIndexOnTextField, regionTestableType));
+
+    accessor.invoke(() -> initDataStore(createIndexOnTextField, regionTestableType));
+
+    accessor.invoke(() -> {
+      Region region = getCache().getRegion(REGION_NAME);
+      IntStream.range(2 * NUM_BUCKETS, 3 * NUM_BUCKETS).forEach(i -> region.put(i,
+          new TestObjectSameFieldNameButDifferentDataTypeInteger(new Integer(1000))));
+      IntStream.range(0, NUM_BUCKETS).forEach(i -> region.put(i, new TestObject("hello world")));
+      IntStream.range(NUM_BUCKETS, 2 * NUM_BUCKETS).forEach(i -> region.put(i,
+          new TestObjectSameFieldNameButDifferentDataTypeFloat(new Float(999.1f))));
+    });
+
+    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
+
+    accessor.invoke(() -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+
+      LuceneQuery luceneQueryForTextField = luceneService.createLuceneQueryFactory()
+          .setResultLimit(100).create(INDEX_NAME, REGION_NAME, "world", "text");
+
+      List luceneResults = luceneQueryForTextField.findResults();
+      validateObjectResultCounts(luceneResults, TestObject.class, NUM_BUCKETS,
+          TestObjectSameFieldNameButDifferentDataTypeFloat.class, 0,
+          TestObjectSameFieldNameButDifferentDataTypeInteger.class, 0);
+
+      FloatRangeQueryProvider floatRangeQueryProvider =
+          new FloatRangeQueryProvider("text", 999.0f, 999.2f);
+      luceneQueryForTextField = luceneService.createLuceneQueryFactory().setResultLimit(100)
+          .create(INDEX_NAME, REGION_NAME, floatRangeQueryProvider);
+
+      luceneResults = luceneQueryForTextField.findResults();
+      validateObjectResultCounts(luceneResults, TestObject.class, 0,
+          TestObjectSameFieldNameButDifferentDataTypeFloat.class, NUM_BUCKETS,
+          TestObjectSameFieldNameButDifferentDataTypeInteger.class, 0);
+
+      IntRangeQueryProvider intRangeQueryProvider = new IntRangeQueryProvider("text", 1000, 1000);
+      luceneQueryForTextField = luceneService.createLuceneQueryFactory().setResultLimit(100)
+          .create(INDEX_NAME, REGION_NAME, intRangeQueryProvider);
+
+      luceneResults = luceneQueryForTextField.findResults();
+      validateObjectResultCounts(luceneResults, TestObject.class, 0,
+          TestObjectSameFieldNameButDifferentDataTypeFloat.class, 0,
+          TestObjectSameFieldNameButDifferentDataTypeInteger.class, NUM_BUCKETS);
+    });
+  }
+
+  private void validateObjectResultCounts(List luceneResults, Class objectType_1,
+      int expectedObjectType_1_count, Class objectType_2, int expectedObjectType_2_count,
+      Class objectType_3, int expectedObjectType_3_count) {
+    int actualObjectType_1_count = 0;
+    int actualObjectType_2_count = 0;
+    int actualObjectType_3_count = 0;
+
+    for (Object luceneResult : luceneResults) {
+      Object resultValue = ((LuceneResultStruct) luceneResult).getValue();
+      if (objectType_1.isInstance(resultValue)) {
+        actualObjectType_1_count++;
+      } else if (objectType_2.isInstance(resultValue)) {
+        actualObjectType_2_count++;
+      } else if (objectType_3.isInstance(resultValue)) {
+        actualObjectType_3_count++;
+      }
+    }
+
+    assertEquals(expectedObjectType_1_count, actualObjectType_1_count);
+    assertEquals(expectedObjectType_2_count, actualObjectType_2_count);
+    assertEquals(expectedObjectType_3_count, actualObjectType_3_count);
+  }
+
+  private SerializableRunnableIF getSerializableRunnableIFCreateIndexOnFieldText() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields("text").create(INDEX_NAME, REGION_NAME);
+    };
+  }
+
+  private SerializableRunnableIF getSerializableRunnableIFCreateIndexOnFieldData() {
+    return () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndexFactory().setFields("data", "text").create(INDEX_NAME, REGION_NAME);
+    };
+  }
+
+
+
+  protected static class TestObject implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String text;
+
+    public TestObject(String text) {
+      this.text = text;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((text == null) ? 0 : text.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      MixedObjectIndexDUnitTest.TestObject other = (MixedObjectIndexDUnitTest.TestObject) obj;
+      if (text == null) {
+        if (other.text != null)
+          return false;
+      } else if (!text.equals(other.text))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "TestObject[" + text + "]";
+    }
+
+
+  }
+
+  protected static class TestObjectSameFieldNameButDifferentDataTypeFloat implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private Float text;
+
+    public TestObjectSameFieldNameButDifferentDataTypeFloat(Float text) {
+      this.text = text;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((text == null) ? 0 : text.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      TestObjectSameFieldNameButDifferentDataTypeFloat other =
+          (TestObjectSameFieldNameButDifferentDataTypeFloat) obj;
+      if (text == null) {
+        if (other.text != null)
+          return false;
+      } else if (!text.equals(other.text))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "TestObjectSameFieldNameButDifferentDataTypeFloat[" + text + "]";
+    }
+  }
+
+  protected static class TestObjectSameFieldNameButDifferentDataTypeInteger
+      implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private Integer text;
+
+    public TestObjectSameFieldNameButDifferentDataTypeInteger(Integer text) {
+      this.text = text;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((text == null) ? 0 : text.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      MixedObjectIndexDUnitTest.TestObjectSameFieldNameButDifferentDataTypeInteger other =
+          (MixedObjectIndexDUnitTest.TestObjectSameFieldNameButDifferentDataTypeInteger) obj;
+      if (text == null) {
+        if (other.text != null)
+          return false;
+      } else if (!text.equals(other.text))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "TestObjectSameFieldNameButDifferentDataTypeInteger[" + text + "]";
+    }
+  }
+
+  protected static class TestObjectWithSameFieldName implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String text;
+
+    public TestObjectWithSameFieldName(String text) {
+      this.text = text;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((text == null) ? 0 : text.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      TestObjectWithSameFieldName other = (TestObjectWithSameFieldName) obj;
+      if (text == null) {
+        if (other.text != null)
+          return false;
+      } else if (!text.equals(other.text))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "TestObjectWithSameFieldName[" + text + "]";
+    }
+
+
+  }
+
+  protected static class TestObjectWithNoCommonField implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String data;
+
+    public TestObjectWithNoCommonField(String data) {
+      this.data = data;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((data == null) ? 0 : data.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      TestObjectWithNoCommonField other = (TestObjectWithNoCommonField) obj;
+      if (data == null) {
+        if (other.data != null)
+          return false;
+      } else if (!data.equals(other.data))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "TestObjectWithNoCommonField[" + data + "]";
+    }
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/426c2fcd/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
index 670e6a8..e9b9369 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
@@ -38,6 +38,7 @@ import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneIndexFactory;
 import org.apache.geode.cache.lucene.LuceneQuery;
 import org.apache.geode.cache.lucene.LuceneQueryException;
+import org.apache.geode.cache.lucene.LuceneQueryProvider;
 import org.apache.geode.cache.lucene.PageableLuceneQueryResults;
 import org.apache.geode.cache.lucene.LuceneService;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
@@ -46,6 +47,9 @@ import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.test.dunit.VM;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.search.Query;
 
 public class LuceneTestUtilities {
   public static final String INDEX_NAME = "index";
@@ -78,6 +82,53 @@ public class LuceneTestUtilities {
   public static String Quarter3 = "Q3";
   public static String Quarter4 = "Q4";
 
+  public static class IntRangeQueryProvider implements LuceneQueryProvider {
+    String fieldName;
+    int lowerValue;
+    int upperValue;
+
+    private transient Query luceneQuery;
+
+    public IntRangeQueryProvider(String fieldName, int lowerValue, int upperValue) {
+      this.fieldName = fieldName;
+      this.lowerValue = lowerValue;
+      this.upperValue = upperValue;
+    }
+
+    @Override
+    public Query getQuery(LuceneIndex index) throws LuceneQueryException {
+      if (luceneQuery == null) {
+        luceneQuery = IntPoint.newRangeQuery(fieldName, lowerValue, upperValue);
+      }
+      System.out.println("IntRangeQueryProvider, using java serializable");
+      return luceneQuery;
+    }
+  }
+
+  public static class FloatRangeQueryProvider implements LuceneQueryProvider {
+    String fieldName;
+    float lowerValue;
+    float upperValue;
+
+    private transient Query luceneQuery;
+
+    public FloatRangeQueryProvider(String fieldName, float lowerValue, float upperValue) {
+      this.fieldName = fieldName;
+      this.lowerValue = lowerValue;
+      this.upperValue = upperValue;
+    }
+
+    @Override
+    public Query getQuery(LuceneIndex index) throws LuceneQueryException {
+      if (luceneQuery == null) {
+        luceneQuery = FloatPoint.newRangeQuery(fieldName, lowerValue, upperValue);
+        // luceneQuery = DoublePoint.newRangeQuery(fieldName, lowerValue, upperValue);
+      }
+      System.out.println("IntRangeQueryProvider, using java serializable");
+      return luceneQuery;
+    }
+  }
+
   public static Region initDataStoreForFixedPR(final Cache cache) throws Exception {
     List<FixedPartitionAttributes> fpaList = new ArrayList<FixedPartitionAttributes>();
     int vmNum = VM.getCurrentVMNum();