You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ja...@apache.org on 2016/03/18 21:21:12 UTC

incubator-geode git commit: GEODE-1044: InitialImagePut executes BEFORE_UPDATE_OP on indexes for updates

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 8c74d600b -> 54931a52e


GEODE-1044: InitialImagePut executes BEFORE_UPDATE_OP on indexes for updates

When entries are recovered from persistence, the values are added to the index,
however a gii can then occur where the initialImagePut needs to properly
alert the index that an update is occuring.  The BEFORE_UPDATE_OP allows the index
to retreive the oldKey and notifies the index that an update is occuring.

Refactor QueryTestUtils and removed unused code
Minor modification for QueryTestUtils to be more generic and not be specific to Portfolios


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

Branch: refs/heads/develop
Commit: 54931a52ed3f2c3d72156f830edc637f00b7a8b1
Parents: 8c74d60
Author: Jason Huynh <hu...@gmail.com>
Authored: Wed Mar 9 13:32:55 2016 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Fri Mar 18 13:24:38 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/AbstractRegionMap.java       |  17 +-
 .../gemfire/cache/query/QueryTestUtils.java     | 335 ++++---------------
 .../cache/query/QueryTestUtilsJUnitTest.java    |  31 +-
 .../query/dunit/CompactRangeIndexDUnitTest.java |  33 +-
 .../cache/query/dunit/HashIndexDUnitTest.java   |  29 +-
 ...itionedRegionCompactRangeIndexDUnitTest.java | 130 +++++++
 .../index/CompactRangeIndexJUnitTest.java       |  17 +-
 .../dunit/PersistentPartitionWithIndex.xml      |  17 +
 8 files changed, 314 insertions(+), 295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
index 9058984..9497131 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
@@ -954,6 +954,11 @@ public abstract class AbstractRegionMap implements RegionMap {
                     }
                   }
                   if (owner.getIndexManager() != null) {
+                    // Due to having no reverse map, we need to be able to generate the oldkey before doing an update
+                    // Without the BEFORE_UPDATE_OP, we would see duplicate entries in the index as the update could not locate the old key
+                    if (!oldRe.isRemoved()) {
+                      owner.getIndexManager().updateIndexes(oldRe, IndexManager.REMOVE_ENTRY, IndexProtocol.BEFORE_UPDATE_OP);
+                    }
                     owner.getIndexManager().updateIndexes(oldRe, oldRe.isRemoved() ? IndexManager.ADD_ENTRY : IndexManager.UPDATE_ENTRY, 
                         oldRe.isRemoved() ? IndexProtocol.OTHER_OP : IndexProtocol.AFTER_UPDATE_OP);
                   }
@@ -992,10 +997,16 @@ public abstract class AbstractRegionMap implements RegionMap {
                 }
                 incEntryCount(1);
               }
+              
               //Update local indexes
-              if (owner.getIndexManager() != null) {
-                owner.getIndexManager().updateIndexes(newRe, newRe.isRemoved() ? IndexManager.REMOVE_ENTRY : IndexManager.UPDATE_ENTRY, 
-                    newRe.isRemoved() ? IndexProtocol.OTHER_OP : IndexProtocol.AFTER_UPDATE_OP);
+                if (owner.getIndexManager() != null) {
+                  // Due to having no reverse map, we need to be able to generate the oldkey before doing an update
+                  // Without the BEFORE_UPDATE_OP, we would see duplicate entries in the index as the update could not locate the old key
+                  if (oldRe != null && !oldRe.isRemoved()) {
+                    owner.getIndexManager().updateIndexes(oldRe, IndexManager.REMOVE_ENTRY, IndexProtocol.BEFORE_UPDATE_OP);
+                  }
+                  owner.getIndexManager().updateIndexes(newRe, newRe.isRemoved() ? IndexManager.REMOVE_ENTRY : IndexManager.UPDATE_ENTRY,
+                      newRe.isRemoved() ? IndexProtocol.OTHER_OP : IndexProtocol.AFTER_UPDATE_OP);
               }
               done = true;
             } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtils.java
index ce7c9fd..edfe6b3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtils.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.query;
 
+import java.io.File;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.HashMap;
@@ -56,14 +58,16 @@ public class QueryTestUtils implements Serializable {
 
   private static Cache cache;
 
-  private static QueryService qs;
-
-  public static final String KEY = "key-";
-
   public HashMap<String, String> queries;
 
   public HashMap<String, String> bindQueries;
 
+  private static QueryTestUtils instance = new QueryTestUtils();
+  
+  public static QueryTestUtils getInstance() {
+    return instance;
+  }
+  
   public QueryTestUtils() {
     queries = new HashMap<String, String>();
     bindQueries = new HashMap<String, String>();
@@ -1036,6 +1040,10 @@ public class QueryTestUtils implements Serializable {
     bindQueries.put("81", "select p from /exampleRegion.values p where p like $1");
     bindQueries.put("82", "select p.positions.get('acc') from $1 p");
   }
+  
+  public static QueryService getQueryService() {
+    return cache.getQueryService();
+  }
 
   public void createServer(VM server, final Properties prop) {
     SerializableRunnable createCacheServer = new CacheSerializableRunnable(
@@ -1061,27 +1069,6 @@ public class QueryTestUtils implements Serializable {
   public static void setCache(Cache cache) {
     QueryTestUtils.cache = cache;
   }
-  
-  public void createClient(VM client, final Properties prop) {
-    SerializableRunnable createCacheClient = new CacheSerializableRunnable(
-        "Create Cache Client") {
-      private static final long serialVersionUID = 1L;
-
-      public void run2() throws CacheException {
-        createClientCache(prop);
-      }
-    };
-    client.invoke(createCacheClient);
-  }
-
-  public void createClientCache(Properties prop) {
-    if (null != prop && !prop.isEmpty()) {
-      cache = (Cache) new ClientCacheFactory(prop).create();
-    }
-    else{
-      cache = (Cache) new ClientCacheFactory().create();
-    }
- }
 
   public void createPartitionRegion(final String name, final Class constraint, VM vm) {
     vm.invoke(new CacheSerializableRunnable("Create Partition region") {
@@ -1159,7 +1146,7 @@ public class QueryTestUtils implements Serializable {
   
   public void removeIndex(String name, String region){
     try {
-      qs = cache.getQueryService();
+      QueryService qs = getQueryService();
       qs.removeIndex(qs.getIndex(cache.getRegion(region), name));
     } catch (Exception e) {
       e.printStackTrace();
@@ -1219,99 +1206,6 @@ public class QueryTestUtils implements Serializable {
       cache.close();
     }
   }
-
-  /**
-   * Places new values in the given region with Object ({@link Portfolio}) Keys
-   * @param vm 
-   * @param regionName
-   * @param size
-   */
-  public void createValuesObjKeys(VM vm, final String regionName, final int size) {
-    vm.invoke(new SerializableRunnable() {
-      public void run() {
-        createValuesObjKeys(regionName, size);
-      }
-    });
-  }
-
-  /**
-   * Places new values in the given region with {@link String} Keys
-   * @param vm 
-   * @param regionName
-   * @param size
-   */
-  public void createValuesStringKeys(VM vm, final String regionName, final int size) {
-    vm.invoke(new SerializableRunnable() {
-      public void run() {
-        createValuesStringKeys(regionName, size);
-      }
-    });
-  }
-
-  /**
-   * Places new values in the given region with {@link String} Keys 
-   * @param regionName
-   * @param size
-   */
-  public void createValuesStringKeys(String regionName, final int size) {
-    Region region = cache.getRegion(regionName);
-    for (int i = 1; i <= size; i++) {
-      region.put("KEY-"+ i, new Portfolio(i));
-    }
-  }
-  
-  /**
-   * Used to update the values in the given region having {@link String} Keys
-   * with modifed values. Should preferably use after the call to createValuesStringKeys
-   * @param regionName
-   * @param size
-   */
-  public void createDiffValuesStringKeys(String regionName, final int size) {
-    Region region = cache.getRegion(regionName);
-    for (int i = 1; i <= size; i++) {
-      region.put("KEY-"+ i, new Portfolio(i+1));
-    }
-  }
-  /**
-   * Places new values in the given region with Object ({@link Portfolio}) Keys 
-   * @param regionName
-   * @param size
-   */
-  public void createValuesObjKeys(String regionName, final int size) {
-    Region region = cache.getRegion(regionName);
-    for (int i = 1; i <= size; i++) {
-      region.put(new Portfolio(i), new Portfolio(i));
-    }
-  }
-  
-  /**
-   * Places new values in the given region with {@link String} Keys 
-   * @param regionName
-   * @param size
-   */
-  public void createNumericValuesStringKeys(String regionName, final int size) {
-    Region region = cache.getRegion(regionName);
-    for (int i = 1; i <= size; i++) {
-      region.put("KEY-"+ i, new Numbers(i));
-    }
-  }
-
-  /**
-   * Destroys entries from the region
-   * @param regionName
-   * @param size
-   * @throws Exception 
-   */
-  public void destroyRegion(String regionName, int size) throws Exception{
-    Region region = cache.getRegion(regionName);
-    for (int i = 1; i <= size; i++) {
-      try {
-        region.destroy("KEY-"+ i);
-      } catch (Exception e) {
-        throw new Exception(e);
-      }
-    }
-  }
  
   /**
    * Executes queries corresponding to the keys passed using array<br>  
@@ -1348,7 +1242,7 @@ public class QueryTestUtils implements Serializable {
    *  Object array containing SelectResults
    */
   public Object[] executeQueries(String qarr[]) throws Exception{
-    qs = cache.getQueryService();
+    QueryService qs = cache.getQueryService();
     Object[] result = new Object[qarr.length];
     String query = null;
     int j = 0;
@@ -1377,7 +1271,7 @@ public class QueryTestUtils implements Serializable {
    * @param qarr
    */
   public Object[] executeQueriesWithoutDistinct(String qarr[]) {
-    qs = cache.getQueryService();
+    QueryService qs = cache.getQueryService();
     Object[] result = new Object[qarr.length];
     String query = null;
     int j = 0;
@@ -1416,7 +1310,7 @@ public class QueryTestUtils implements Serializable {
    * @param qarr
    */
   public Object[] executeQueriesWithDistinct(String qarr[]) {
-    qs = cache.getQueryService();
+    QueryService qs = cache.getQueryService();
     Object[] result = new Object[qarr.length];
     String query = null;
     int j = 0;
@@ -1438,40 +1332,6 @@ public class QueryTestUtils implements Serializable {
   }
 
   /**
-   * Execute all the queries in the map
-   * @param vm
-   */
-  public void executeAllQueries(VM vm) {
-    vm.invoke(new CacheSerializableRunnable("Execute all queries") {
-      public void run2() {
-        executeAllQueries();
-      }
-    });
-  }
-
-  /**
-   * Execute all the queries in the map
-   */
-  public Object[] executeAllQueries() {
-    qs = cache.getQueryService();
-    Object[] result = new Object[queries.size()];
-
-    int j = 0;
-    for (Map.Entry<String, String> entry : queries.entrySet()) {
-      String queryId = entry.getKey();
-      String query = entry.getValue();
-      System.out.println("\nExecuting query: " + query);
-      try {
-        result[j++] = qs.newQuery(query).execute();
-        formatQueryResult(result);
-      } catch (Exception e) {
-        throw new RuntimeException("Query: " + queryId + "-" + query + " failed: ", e);
-      }
-    }
-    return result;
-  }
-
-  /**
    * Executes a particular bind query corresponding to the ID passed using array<br>  
    *    <P><code>
    *    String[] arr = {"1", "2", "3"};<br>
@@ -1499,7 +1359,7 @@ public class QueryTestUtils implements Serializable {
   
   public Object executeBindQuery(final String queryId, final Object[] params) {
     Object result = null;
-    qs = cache.getQueryService();
+    QueryService qs = cache.getQueryService();
     String query = bindQueries.get(queryId);
     try {
       getLogger().fine("\nExecuting query: " + query);
@@ -1510,120 +1370,8 @@ public class QueryTestUtils implements Serializable {
     return result;
   }
 
-  public void compareResults(SelectResults[][] r) {
-    Set set1 = null;
-    Set set2 = null;
-    Iterator itert1 = null;
-    Iterator itert2 = null;
-    ObjectType type1, type2;
-
-    for (int j = 0; j < r.length; j++) {
-      CollectionType collType1 = r[j][0].getCollectionType();
-      CollectionType collType2 = r[j][1].getCollectionType();
-      type1 = collType1.getElementType();
-      type2 = collType2.getElementType();
-
-      if (collType1.getSimpleClassName().equals(collType2.getSimpleClassName())) {
-        CacheUtils.log("Both SelectResults are of the same Type i.e.--> "
-            + collType1);
-      }
-      else {
-        CacheUtils.log("Collection type are : " + collType1 + "and  "
-            + collType2);
-        Assert
-            .fail("FAILED:Select results Collection Type is different in both the cases. CollectionType1="
-                + collType1 + " CollectionType2=" + collType2);
-      }
-      if (type1.equals(type2)) {
-        CacheUtils.log("Both SelectResults have same element Type i.e.--> "
-            + type1);
-      }
-      else {
-        CacheUtils.log("Classes are :  type1=" + type1.getSimpleClassName()
-            + " type2= " + type2.getSimpleClassName());
-        Assert
-            .fail("FAILED:SelectResult Element Type is different in both the cases. Type1="
-                + type1 + " Type2=" + type2);
-      }
-
-      if (collType1.equals(collType2)) {
-        CacheUtils.log("Both SelectResults are of the same Type i.e.--> "
-            + collType1);
-      }
-      else {
-        CacheUtils.log("Collections are : " + collType1 + " " + collType2);
-        Assert
-            .fail("FAILED:SelectResults Collection Type is different in both the cases. CollType1="
-                + collType1 + " CollType2=" + collType2);
-      }
-      if (r[j][0].size() == r[j][1].size()) {
-        CacheUtils.log("Both SelectResults are of Same Size i.e.  Size= "
-            + r[j][1].size());
-      }
-      else {
-        Assert
-            .fail("FAILED:SelectResults size is different in both the cases. Size1="
-                + r[j][0].size() + " Size2 = " + r[j][1].size());
-      }
-      set2 = ((r[j][1]).asSet());
-      set1 = ((r[j][0]).asSet());
-      // boolean pass = true;
-      itert1 = set1.iterator();
-      while (itert1.hasNext()) {
-        Object p1 = itert1.next();
-        itert2 = set2.iterator();
-
-        boolean exactMatch = false;
-        while (itert2.hasNext()) {
-          Object p2 = itert2.next();
-          if (p1 instanceof Struct) {
-            Object[] values1 = ((Struct)p1).getFieldValues();
-            Object[] values2 = ((Struct)p2).getFieldValues();
-            Assert.assertEquals(values1.length, values2.length);
-            boolean elementEqual = true;
-            for (int i = 0; i < values1.length; ++i) {
-              elementEqual = elementEqual
-                  && ((values1[i] == values2[i]) || values1[i]
-                      .equals(values2[i]));
-            }
-            exactMatch = elementEqual;
-          }
-          else {
-            exactMatch = (p2 == p1) || p2.equals(p1);
-          }
-          if (exactMatch) {
-            break;
-          }
-        }
-        if (!exactMatch) {
-          Assert
-              .fail("Atleast one element in the pair of SelectResults supposedly identical, is not equal ");
-        }
-      }
-    }
-  }
-
-  public void formatQueryResult(Object[] results) {
-    log("\nResults: ");
-    for (Object result : results) {
-      if (result == null) {
-        log("NULL");
-      } else if (result == QueryService.UNDEFINED) {
-        log("UNDEFINED");
-      } else if (result instanceof SelectResults) {
-        Collection<?> collection = ((SelectResults<?>) result).asList();
-        StringBuffer sb = new StringBuffer();
-        for (Object e : collection) {
-          sb.append(e + "\n\t");
-        }
-        log(sb.toString());
-      } else {
-        log("Unknown");
-      }
-    }
-  }
-
-  public LogWriter getLogger() {
+ 
+  private LogWriter getLogger() {
     if (cache == null) {
       return null;
     }
@@ -1633,8 +1381,47 @@ public class QueryTestUtils implements Serializable {
   public Cache getCache() {
     return cache;
   }
-
-  public void log(Object message) {
-    CacheUtils.log(message);
+  
+  public static void createCacheInVM(VM vm, Properties props) {
+    vm.invoke(() -> {
+      getInstance().createCache(props);
+    });
+  }
+  
+  public static void closeCacheInVM(VM vm) {
+    vm.invoke(() -> {
+      getInstance().cache.close();
+    });
+  }
+  
+  public Region getRegion(String regionName) {
+    return cache.getRegion(regionName);
+  }
+  
+  public void populateRegion(String regionName, Map<?, ?> entries) {
+    Region r = cache.getRegion("/" + regionName);
+    entries.entrySet().forEach(e -> {
+      r.put(e.getKey(), e.getValue());
+    });
+  }
+  
+  public static void populateRegion(VM vm, String regionName, Map<?, ?> entries) {
+    vm.invoke(() -> {
+      getInstance().populateRegion(regionName, entries);
+    });
+  }
+  
+  public static File createTestRootDiskStore(String testName) throws IOException {
+    File diskDir = new File(testName).getAbsoluteFile();
+    com.gemstone.gemfire.internal.FileUtil.delete(diskDir);
+    diskDir.mkdir();
+    diskDir.deleteOnExit();
+    return diskDir;
+  }
+  
+  public static File createRootDiskStoreInVM(VM vm, String rootDiskStoreName) {
+    return vm.invoke(() -> {
+      return createTestRootDiskStore(rootDiskStoreName);
+    });
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtilsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtilsJUnitTest.java
index 0358aad..c5d40bd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtilsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryTestUtilsJUnitTest.java
@@ -30,6 +30,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.query.data.Numbers;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -49,14 +51,35 @@ public class QueryTestUtilsJUnitTest {
   @Before
   public void setUp() throws Exception {
 
-    utils = new QueryTestUtils();
+    utils = QueryTestUtils.getInstance();
     utils.createCache(null);
     // create regions
     utils.createReplicateRegion("exampleRegion");
     utils.createReplicateRegion("numericRegion");
     // put entries in the region
-    utils.createValuesStringKeys("exampleRegion", 10);
-    utils.createNumericValuesStringKeys("numericRegion", 10);
+    populatePortfolioValuesInRegion("exampleRegion", 10);
+    populateNumericValuesInRegion("numericRegion", 10);
+  }
+  
+  private void populatePortfolioValuesInRegion(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Portfolio(i));
+    }
+  }
+  
+  private void populateOffsetPortfolioValuesInRegion(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Portfolio(i + 1));
+    }
+  }
+  
+  private void populateNumericValuesInRegion(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Numbers(i));
+    }
   }
 
   @Test
@@ -85,7 +108,7 @@ public class QueryTestUtilsJUnitTest {
   
   @Test
   public void testQueriesWithoutDistinct() throws Exception{
-    utils.createDiffValuesStringKeys("exampleRegion", 2);
+    this.populateOffsetPortfolioValuesInRegion("exampleRegion", 2);
     String[] queries = { "181" };
     int results = 0;
     for (Object result :  utils.executeQueriesWithoutDistinct(queries)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CompactRangeIndexDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CompactRangeIndexDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CompactRangeIndexDUnitTest.java
index 81b545f..df655a2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CompactRangeIndexDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CompactRangeIndexDUnitTest.java
@@ -19,6 +19,7 @@ package com.gemstone.gemfire.cache.query.dunit;
 import java.util.Properties;
 
 import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.query.QueryTestUtils;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
@@ -70,7 +71,7 @@ public class CompactRangeIndexDUnitTest extends DistributedTestCase{
 
     vm0.invoke(new CacheSerializableRunnable("Putting values") {
       public void run2() {
-        utils.createValuesStringKeys("examplePartitionedRegion", 100);
+        putPortfolios("examplePartitionedRegion", 100);
       }
     });
     try {
@@ -112,7 +113,7 @@ public class CompactRangeIndexDUnitTest extends DistributedTestCase{
   public void doPut(final int entries) {
      vm0.invokeAsync(new CacheSerializableRunnable("Putting values") {
       public void run2() {
-        utils.createValuesStringKeys("exampleRegion", entries);
+        putPortfolios("exampleRegion", entries);
       }
     });
   }
@@ -120,7 +121,7 @@ public class CompactRangeIndexDUnitTest extends DistributedTestCase{
   public void doPutSync(final int entries) {
     vm0.invoke(new CacheSerializableRunnable("Putting values") {
      public void run2() {
-       utils.createValuesStringKeys("exampleRegion", entries);
+       putPortfolios("exampleRegion", entries);
      }
    });
  }
@@ -128,7 +129,7 @@ public class CompactRangeIndexDUnitTest extends DistributedTestCase{
   public void doUpdate(final int entries) {
     vm0.invokeAsync(new CacheSerializableRunnable("Updating values") {
      public void run2() {
-       utils.createDiffValuesStringKeys("exampleRegion", entries);
+       putOffsetPortfolios("exampleRegion", entries);
      }
    });
  }
@@ -159,7 +160,15 @@ public class CompactRangeIndexDUnitTest extends DistributedTestCase{
       public void run2() {
         try {
           Thread.sleep(500);
-          utils.destroyRegion("exampleRegion", entries);
+          //destroy entries
+          Region region = utils.getRegion("exampleRegion");
+          for (int i = 1; i <= entries; i++) {
+            try {
+              region.destroy("KEY-"+ i);
+            } catch (Exception e) {
+              throw new Exception(e);
+            }
+          }
         } catch (Exception e) {
           fail("Destroy failed.");
         }
@@ -191,6 +200,20 @@ public class CompactRangeIndexDUnitTest extends DistributedTestCase{
     });
   }
   
+  private void putPortfolios(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Portfolio(i));
+    }
+  }
+  
+  private void putOffsetPortfolios(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Portfolio(i + 1));
+    }
+  }
+  
   
   private static class CompactRangeIndexTestHook implements TestHook{
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/HashIndexDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/HashIndexDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/HashIndexDUnitTest.java
index 38721be..76f850d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/HashIndexDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/HashIndexDUnitTest.java
@@ -19,7 +19,9 @@ package com.gemstone.gemfire.cache.query.dunit;
 import java.util.Properties;
 
 import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.query.QueryTestUtils;
+import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
@@ -76,7 +78,7 @@ public class HashIndexDUnitTest extends DistributedTestCase{
   public void doPut(final int entries) {
      vm0.invokeAsync(new CacheSerializableRunnable("Putting values") {
       public void run2() {
-        utils.createValuesStringKeys("exampleRegion", entries);
+        putPortfolios("exampleRegion", entries);
       }
     });
   }
@@ -84,7 +86,7 @@ public class HashIndexDUnitTest extends DistributedTestCase{
   public void doUpdate(final int entries) {
     vm0.invokeAsync(new CacheSerializableRunnable("Updating values") {
      public void run2() {
-       utils.createDiffValuesStringKeys("exampleRegion", entries);
+       putOffsetPortfolios("exampleRegion", entries);
      }
    });
  }
@@ -115,7 +117,14 @@ public class HashIndexDUnitTest extends DistributedTestCase{
           e.printStackTrace();
         }
         try {
-         utils.destroyRegion("exampleRegion", entries);
+         Region region = utils.getRegion("exampleRegion");
+         for (int i = 1; i <= entries; i++) {
+           try {
+             region.destroy("KEY-"+ i);
+           } catch (Exception e) {
+             throw new Exception(e);
+           }
+         }
         }
         catch (Exception e) {
           throw new CacheException(e){};
@@ -129,4 +138,18 @@ public class HashIndexDUnitTest extends DistributedTestCase{
     Thread.sleep(5000);
     utils.closeServer(vm0);
   }
+  
+  private void putPortfolios(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Portfolio(i));
+    }
+  }
+  
+  private void putOffsetPortfolios(String regionName, int size) {
+    Region region = utils.getRegion(regionName);
+    for (int i = 1; i <= size; i++) {
+      region.put("KEY-"+ i, new Portfolio(i + 1));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PartitionedRegionCompactRangeIndexDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PartitionedRegionCompactRangeIndexDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PartitionedRegionCompactRangeIndexDUnitTest.java
new file mode 100644
index 0000000..8be017c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PartitionedRegionCompactRangeIndexDUnitTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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 com.gemstone.gemfire.cache.query.dunit;
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.QueryTestUtils;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.query.data.Portfolio;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+
+@Category(DistributedTest.class)
+public class PartitionedRegionCompactRangeIndexDUnitTest extends DistributedTestCase {
+
+  public PartitionedRegionCompactRangeIndexDUnitTest(String name) {
+    super(name);
+  }
+
+  private Properties getSystemProperties(String cacheXML) {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+DistributedTestUtils.getDUnitLocatorPort()+"]");
+    props.setProperty("cache-xml-file", TestUtil.getResourcePath(getClass(), cacheXML));
+    return props;
+  }
+  
+  public void setUp() throws Exception {
+    this.disconnectAllFromDS();
+  }
+  
+  @Override
+  protected void preTearDown() throws Exception {
+    Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    VM vm1 = host.getVM(1);
+    VM vm2 = host.getVM(2);
+    QueryTestUtils.closeCacheInVM(vm0);
+    QueryTestUtils.closeCacheInVM(vm1);
+    QueryTestUtils.closeCacheInVM(vm2);
+  }
+  
+  @Test
+  public void testGIIUpdateWithIndexDoesNotDuplicateEntryInIndexWhenAlreadyRecoveredFromPersistence() throws Exception {
+    Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    VM vm1 = host.getVM(1);
+    VM vm2 = host.getVM(2);
+    //Adding due to known race condition for creation of partitioned indexes via cache.xml
+    IgnoredException.addIgnoredException("IndexNameConflictException");
+    
+    String regionName = "persistentTestRegion"; //this region is created via cache.xml
+    String idQuery = "select * from /" + regionName + " p where p.ID = 1";
+    int idQueryExpectedSize = 1;
+    int numEntries = 100;
+    Map<String, Portfolio> entries = new HashMap<>();
+    IntStream.range(0, numEntries).forEach(i -> entries.put("key-" + i, new Portfolio(i)));
+    Map<String, Portfolio> newEntries = new HashMap<>();
+    IntStream.range(0, numEntries).forEach(i -> newEntries.put("key-" + i,  new Portfolio(i + 1)));
+
+    File rootDiskStore1 = QueryTestUtils.createRootDiskStoreInVM(vm0, "diskDir-PersistentPartitionWithIndexDiskStore");
+    File rootDiskStore2 = QueryTestUtils.createRootDiskStoreInVM(vm1, "diskDir-PersistentPartitionWithIndexDiskStore");
+    File rootDiskStore3 = QueryTestUtils.createRootDiskStoreInVM(vm2, "diskDir-PersistentPartitionWithIndexDiskStore");
+
+    QueryTestUtils.createCacheInVM(vm0, getSystemProperties("PersistentPartitionWithIndex.xml"));
+    QueryTestUtils.createCacheInVM(vm1, getSystemProperties("PersistentPartitionWithIndex.xml"));
+    QueryTestUtils.createCacheInVM(vm2, getSystemProperties("PersistentPartitionWithIndex.xml"));
+    QueryTestUtils.populateRegion(vm2, regionName, entries);
+    
+    vm1.invoke(verifyQueryResultsSize(idQuery, idQueryExpectedSize));
+    QueryTestUtils.closeCacheInVM(vm1);
+    
+    //update entries
+    QueryTestUtils.populateRegion(vm0, regionName, entries);
+    QueryTestUtils.closeCacheInVM(vm0);
+
+    //restart 2 nodes
+    QueryTestUtils.createCacheInVM(vm0, getSystemProperties("PersistentPartitionWithIndex.xml"));
+    QueryTestUtils.createCacheInVM(vm1, getSystemProperties("PersistentPartitionWithIndex.xml"));
+
+    vm2.invoke(verifyQueryResultsSize(idQuery, idQueryExpectedSize));
+    vm1.invoke(verifyQueryResultsSize(idQuery, idQueryExpectedSize));
+    vm0.invoke(verifyQueryResultsSize(idQuery, idQueryExpectedSize));
+  }
+  
+  private SerializableRunnable verifyQueryResultsSize(String query, int expectedSize) {
+    return new SerializableRunnable() {
+      public void run() {
+        try {
+          QueryService qs = QueryTestUtils.getInstance().getQueryService();
+          Query q = qs.newQuery(query);
+          SelectResults sr = (SelectResults) q.execute();
+          assertEquals(expectedSize, sr.size());
+        }
+        catch (Exception e) {
+          fail("Exception occurred when executing verifyQueryResultsSize for query:" + query);
+        }
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndexJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndexJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndexJUnitTest.java
index 68a946c..ba84fc8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndexJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CompactRangeIndexJUnitTest.java
@@ -77,7 +77,7 @@ public class CompactRangeIndexJUnitTest  {
     isUsingConcurrentHashSet("type1");
     utils.removeIndex("type", "/exampleRegion");
     executeQueryWithAndWithoutIndex(4);
-    updateValues(2);
+    putOffsetValues(2);
     executeQueryWithCount();
     executeQueryWithAndWithoutIndex(3);
     executeRangeQueryWithDistinct(8);
@@ -377,13 +377,18 @@ public class CompactRangeIndexJUnitTest  {
 
   
  
-  public void putValues(int num) {
-    long start = System.currentTimeMillis();
-    utils.createValuesStringKeys("exampleRegion", num);
+  private void putValues(int num) {
+    Region region = utils.getRegion("exampleRegion");
+    for (int i = 1; i <= num; i++) {
+      region.put("KEY-"+ i, new Portfolio(i));
+    }
   }
   
-  private void updateValues(int num){
-    utils.createDiffValuesStringKeys("exampleRegion", num);
+  private void putOffsetValues(int num) {
+    Region region = utils.getRegion("exampleRegion");
+    for (int i = 1; i <= num; i++) {
+      region.put("KEY-"+ i, new Portfolio(i + 1));
+    }
   }
  
   public void executeQueryWithCount() throws Exception{

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/54931a52/geode-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/PersistentPartitionWithIndex.xml
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/PersistentPartitionWithIndex.xml b/geode-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/PersistentPartitionWithIndex.xml
new file mode 100644
index 0000000..8d9fd97
--- /dev/null
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/PersistentPartitionWithIndex.xml
@@ -0,0 +1,17 @@
+<?xml version="1.0"?>
+<!DOCTYPE cache PUBLIC
+        "-//GemStone Systems, Inc.//GemFire Declarative Caching 8.0//EN"
+        "http://www.gemstone.com/dtd/cache8_0.dtd">
+<cache>
+<disk-store name="diskStore">
+        <disk-dirs>
+            <disk-dir>diskDir-PersistentPartitionWithIndexDiskStore</disk-dir>
+        </disk-dirs>
+    </disk-store>
+
+<region name="persistentTestRegion">
+<region-attributes refid="PARTITION_REDUNDANT_PERSISTENT" disk-store-name="diskStore" >
+</region-attributes>
+<index name="persistentTestRegion_ID" from-clause="/persistentTestRegion" expression="ID" type="range"/>
+</region>
+</cache>