You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/04/30 00:23:57 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #6279: GEODE-9004: Align results of queries with and without map indexes

DonalEvans commented on a change in pull request #6279:
URL: https://github.com/apache/geode/pull/6279#discussion_r623497964



##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java
##########
@@ -328,44 +332,49 @@ public boolean isMapType() {
 
   @Override
   void addMapping(Object key, Object value, RegionEntry entry) throws IMQException {
-    if (key == QueryService.UNDEFINED || !(key instanceof Map)) {
-      return;
-    }
-    if (this.isAllKeys) {
-      Iterator<Map.Entry<?, ?>> entries = ((Map) key).entrySet().iterator();
-      while (entries.hasNext()) {
-        Map.Entry<?, ?> mapEntry = entries.next();
-        Object mapKey = mapEntry.getKey();
-        Object indexKey = mapEntry.getValue();
-        this.doIndexAddition(mapKey, indexKey, value, entry);
-      }
-    } else {
-      for (Object mapKey : mapKeys) {
-        Object indexKey = ((Map) key).get(mapKey);
-        if (indexKey != null) {
-          this.doIndexAddition(mapKey, indexKey, value, entry);
-        }
-      }
-    }
+    addOrSaveMapping(key, value, entry, true);
   }
 
   @Override
   void saveMapping(Object key, Object value, RegionEntry entry) throws IMQException {
-    if (key == QueryService.UNDEFINED || !(key instanceof Map)) {
+    addOrSaveMapping(key, value, entry, false);
+  }
+
+  void addOrSaveMapping(Object key, Object value, RegionEntry entry, boolean isAdd)
+      throws IMQException {
+    if (key == QueryService.UNDEFINED || (key != null && !(key instanceof Map))) {
       return;
     }
     if (this.isAllKeys) {
+      // If the key is null or it has no elements then we cannot associate it
+      // to any index key (it would apply to all). That is why
+      // this type of index does not support !=
+      // queries or queries comparing with null.
+      if (key == null) {
+        return;
+      }
       Iterator<Map.Entry<?, ?>> entries = ((Map) key).entrySet().iterator();
       while (entries.hasNext()) {
         Map.Entry<?, ?> mapEntry = entries.next();
         Object mapKey = mapEntry.getKey();
         Object indexKey = mapEntry.getValue();
-        this.saveIndexAddition(mapKey, indexKey, value, entry);
+        if (isAdd) {
+          this.doIndexAddition(mapKey, indexKey, value, entry);
+        } else {
+          this.saveIndexAddition(mapKey, indexKey, value, entry);
+        }
       }
     } else {
       for (Object mapKey : mapKeys) {
-        Object indexKey = ((Map) key).get(mapKey);
-        if (indexKey != null) {
+        Object indexKey;
+        if (key == null) {
+          indexKey = QueryService.UNDEFINED;
+        } else {
+          indexKey = ((Map) key).get(mapKey);

Review comment:
       Compiler warning here can be fixed by using `(Map<?, ?>)`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
##########
@@ -338,7 +341,302 @@ public void testNullMapValuesInIndexOnLocalRegionForCompactMap() throws Exceptio
     SelectResults result = (SelectResults) qs
         .newQuery("select * from " + SEPARATOR + "portfolio p where p.positions['SUN'] = null")
         .execute();
-    assertEquals(1, result.size());
+    assertEquals(2, result.size());
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithoutIndex() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+    testQueriesForValueInMapField(region, qs);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithCompactMapIndexWithOneKey() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 = qs.createIndex(INDEX_NAME, "positions['SUN']", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(CompactRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes in region entries for the 'SUN' key:
+    // ("nothing", "more"). null or UNDEFINED not included
+    assertThat(keys).isEqualTo(2);
+    // mapIndexKeys must be zero because the index used is a range index and not a map index
+    assertThat(mapIndexKeys).isEqualTo(0);
+    // The number of values must be equal to the number of region entries
+    assertThat(values).isEqualTo(7);
+    // The index must be used in every query
+    assertThat(uses).isEqualTo(4);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithCompactMapIndexWithSeveralKeys() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 =
+        qs.createIndex(INDEX_NAME, "positions['SUN', 'ERICSSON']", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(CompactMapRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes for the 'SUN' key ("nothing", "more") plus
+    // the number of different values the positions map takes for the 'ERICSSON' key
+    // ("hey"). null or UNDEFINED not included.
+    assertThat(keys).isEqualTo(3);
+    // The number of mapIndexKeys must be equal to the number of keys
+    // in the index that appear in region entries:
+    // 'SUN', 'ERICSSON'
+    assertThat(mapIndexKeys).isEqualTo(2);
+    // The number of values must be equal to the number of entries
+    // times the number of indexed keys in the map
+    assertThat(values).isEqualTo(14);
+    // The index must be used in all queries
+    assertThat(uses).isEqualTo(4);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithCompactMapIndexWithStar() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 = qs.createIndex(INDEX_NAME, "positions[*]", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(CompactMapRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((CompactMapRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes for each key
+    // for each entry in the region (null not included):
+    // "something", "nothing", "more", "empty", "hey", "tip"
+    assertThat(keys).isEqualTo(6);
+    // The number of mapIndexKeys must be equal to the number of different keys
+    // that appear in entries of the region:
+    // "IBM", "ERICSSON", "HP", "SUN", null
+    assertThat(mapIndexKeys).isEqualTo(5);
+    // The number of values must be equal to the number of values the
+    // positions map takes for each key
+    // for each entry in the region:
+    // "something", null, "nothing", "more", "empty", "hey", "more", "tip"
+    assertThat(values).isEqualTo(8);
+    // The index must not be used in queries with "!=" or when comparing with null
+    assertThat(uses).isEqualTo(1);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithMapIndexWithOneKey() throws Exception {
+    IndexManager.TEST_RANGEINDEX_ONLY = true;
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 = qs.createIndex(INDEX_NAME, "positions['SUN']", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(RangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((RangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((RangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((RangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((RangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes in region entries for the 'SUN' key:
+    // ("nothing", "more") excluding null.
+    assertThat(keys).isEqualTo(2);
+    // mapIndexKeys must be zero because the index used is a range index and not a map index
+    assertThat(mapIndexKeys).isEqualTo(0);
+    // The number of values must be equal to the number of region entries
+    assertThat(values).isEqualTo(7);
+    // The index must be used in every query
+    assertThat(uses).isEqualTo(4);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithMapIndexWithSeveralKeys() throws Exception {
+    IndexManager.TEST_RANGEINDEX_ONLY = true;
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 =
+        qs.createIndex(INDEX_NAME, "positions['SUN', 'ERICSSON']", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(MapRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((MapRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((MapRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((MapRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((MapRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes for the 'SUN' key ("nothing", "more") plus
+    // the number of different values the positions map takes for the 'ERICSSON' key
+    // ("hey") for each entry in the region (null values not counted).
+    assertThat(keys).isEqualTo(3);
+    // The number of mapIndexKeys must be equal to the number of keys
+    // in the index that appear in the positions map of region entries:
+    // 'SUN', 'ERICSSON'
+    assertThat(mapIndexKeys).isEqualTo(2);
+    // The number of values must be equal to the number of
+    // entries times the number of keys in the index
+    assertThat(values).isEqualTo(14);
+    // The index must be used for all queries
+    assertThat(uses).isEqualTo(4);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithMapIndexWithStar() throws Exception {
+    IndexManager.TEST_RANGEINDEX_ONLY = true;
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 = qs.createIndex(INDEX_NAME, "positions[*]", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(MapRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((MapRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((MapRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((MapRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((MapRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes for each key for each entry in the region:
+    // "something", null, "nothing", "more", "hey", "tip"
+    assertThat(keys).isEqualTo(6);
+    // The number of mapIndexKeys must be equal to the number of different keys
+    // that appear in entries of the region:
+    // "IBM", "ERICSSON", "HP", "SUN", "cannotBeNull"
+    assertThat(mapIndexKeys).isEqualTo(5);
+    // The number of values must be equal to the number of values the
+    // positions map takes for each key for each entry in the region:
+    // "something", null, "nothing", "more", "hey", "more", "tip"
+    assertThat(values).isEqualTo(8);
+    // The index must not be used in queries with "!=" or when comparing with null
+    assertThat(uses).isEqualTo(1);
+  }
+
+  public void testQueriesForValueInMapField(Region<Object, Object> region, QueryService qs)
+      throws Exception {
+
+    // Empty map
+    Portfolio p1 = new Portfolio(1, 1);
+    p1.positions = new HashMap<>();
+    region.put(1, p1);
+
+    // Map is null
+    Portfolio p2 = new Portfolio(2, 2);
+    p2.positions = null;
+    region.put(2, p2);
+
+    // Map with null value for "SUN" key
+    Portfolio p3 = new Portfolio(3, 3);
+    p3.positions = new HashMap<>();
+    p3.positions.put("IBM", "something");
+    p3.positions.put("SUN", null);
+    region.put(3, p3);
+
+    // Map with not null value for "SUN" key
+    Portfolio p4 = new Portfolio(4, 4);
+    p4.positions = new HashMap<>();
+    p4.positions.put("SUN", "nothing");
+    region.put(4, p4);
+
+    // Map with another value for the "SUN" key
+    Portfolio p5 = new Portfolio(5, 5);
+    p5.positions = new HashMap<>();
+    p5.positions.put("SUN", "more");
+    // null is not allowed as key
+    p5.positions.put("cannotBeNull", "empty");
+    region.put(5, p5);
+
+    // One more with map without the "SUN" key
+    Portfolio p6 = new Portfolio(6, 6);
+    p6.positions = new HashMap<>();
+    p6.positions.put("ERICSSON", "hey");
+    region.put(6, p6);
+
+    // Map with a repeated value for the "SUN" key
+    Portfolio p7 = new Portfolio(7, 7);
+    p7.positions = new HashMap<>();
+    p7.positions.put("SUN", "more");
+    p7.positions.put("HP", "tip");
+    region.put(7, p7);
+
+    String query;
+    query = "select * from " + SEPARATOR + "portfolio p where p.positions['SUN'] = null";
+    SelectResults<Object> result = UncheckedUtils.uncheckedCast(qs
+        .newQuery(query)
+        .execute());
+    assertThat(result.size()).isEqualTo(3);

Review comment:
       These two asserts can be combined to one by using the following:
   ```
   assertThat(result).containsExactlyInAnyOrder(p1, p3, p6);
   ```
   This can be applied for all the pairs of asserts in this method.

##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java
##########
@@ -98,10 +98,17 @@ protected void removeMapping(RegionEntry entry, int opCode) throws IMQException
 
   @Override
   void saveMapping(Object key, Object value, RegionEntry entry) throws IMQException {
-    if (key == QueryService.UNDEFINED || !(key instanceof Map)) {
+    if (key == QueryService.UNDEFINED || (key != null && !(key instanceof Map))) {
       return;
     }
     if (this.isAllKeys) {
+      // If the key is null or it has no elements then we cannot associate it
+      // to any index key (it would apply to all). That is why
+      // this type of index does not support !=
+      // queries or queries comparing with null.
+      if (key == null) {
+        return;
+      }
       Iterator<Map.Entry<?, ?>> entries = ((Map) key).entrySet().iterator();

Review comment:
       This can be simplified and the compiler warning fixed by using:
   ```
         for (Entry<?, ?> mapEntry : ((Map<?, ?>) key).entrySet()) {
           Object mapKey = mapEntry.getKey();
           Object indexKey = mapEntry.getValue();
   ```

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
##########
@@ -338,7 +341,302 @@ public void testNullMapValuesInIndexOnLocalRegionForCompactMap() throws Exceptio
     SelectResults result = (SelectResults) qs
         .newQuery("select * from " + SEPARATOR + "portfolio p where p.positions['SUN'] = null")
         .execute();
-    assertEquals(1, result.size());
+    assertEquals(2, result.size());
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithoutIndex() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+    testQueriesForValueInMapField(region, qs);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithCompactMapIndexWithOneKey() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 = qs.createIndex(INDEX_NAME, "positions['SUN']", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(CompactRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes in region entries for the 'SUN' key:
+    // ("nothing", "more"). null or UNDEFINED not included
+    assertThat(keys).isEqualTo(2);
+    // mapIndexKeys must be zero because the index used is a range index and not a map index
+    assertThat(mapIndexKeys).isEqualTo(0);
+    // The number of values must be equal to the number of region entries
+    assertThat(values).isEqualTo(7);

Review comment:
       Would it be possible to replace this hard-coded value (and other occurrences in subsequent tests) with one derived from the region directly?

##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java
##########
@@ -328,44 +332,49 @@ public boolean isMapType() {
 
   @Override
   void addMapping(Object key, Object value, RegionEntry entry) throws IMQException {
-    if (key == QueryService.UNDEFINED || !(key instanceof Map)) {
-      return;
-    }
-    if (this.isAllKeys) {
-      Iterator<Map.Entry<?, ?>> entries = ((Map) key).entrySet().iterator();
-      while (entries.hasNext()) {
-        Map.Entry<?, ?> mapEntry = entries.next();
-        Object mapKey = mapEntry.getKey();
-        Object indexKey = mapEntry.getValue();
-        this.doIndexAddition(mapKey, indexKey, value, entry);
-      }
-    } else {
-      for (Object mapKey : mapKeys) {
-        Object indexKey = ((Map) key).get(mapKey);
-        if (indexKey != null) {
-          this.doIndexAddition(mapKey, indexKey, value, entry);
-        }
-      }
-    }
+    addOrSaveMapping(key, value, entry, true);
   }
 
   @Override
   void saveMapping(Object key, Object value, RegionEntry entry) throws IMQException {
-    if (key == QueryService.UNDEFINED || !(key instanceof Map)) {
+    addOrSaveMapping(key, value, entry, false);
+  }
+
+  void addOrSaveMapping(Object key, Object value, RegionEntry entry, boolean isAdd)
+      throws IMQException {
+    if (key == QueryService.UNDEFINED || (key != null && !(key instanceof Map))) {
       return;
     }
     if (this.isAllKeys) {
+      // If the key is null or it has no elements then we cannot associate it
+      // to any index key (it would apply to all). That is why
+      // this type of index does not support !=
+      // queries or queries comparing with null.
+      if (key == null) {
+        return;
+      }
       Iterator<Map.Entry<?, ?>> entries = ((Map) key).entrySet().iterator();

Review comment:
       While I understand that it's not part of your changes, there's an opportunity to improve some code here a little. This while loop can be simplified into an enhanced for loop, and the compiler warning for unchecked assignment can be fixed by using:
   ```
         for (Map.Entry<?, ?> mapEntry : ((Map<?, ?>) key).entrySet()) {
           Object mapKey = mapEntry.getKey();
           Object indexKey = mapEntry.getValue();
   ```

##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledComparison.java
##########
@@ -651,6 +655,29 @@ public static String getSimpleClassName(Class cls) {
         indexData = QueryUtils.getAvailableIndexIfAny(path, context, this._operator);
       }
 
+      // Do not use indexes when map index with allkeys and != condition or when comparing with null
+      try {
+        if (indexData != null

Review comment:
       Would it be possible to extract the contents of this if condition to a method, and refactor it slightly to make it easier to parse and understand? As it currently is, it's very difficult to see what the desired condition is. Readability is more important than brevity here, I think.

##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java
##########
@@ -112,11 +119,13 @@ void saveMapping(Object key, Object value, RegionEntry entry) throws IMQExceptio
       removeOldMappings(((Map) key).keySet(), entry);
     } else {
       for (Object mapKey : mapKeys) {
-        Object indexKey = ((Map) key).get(mapKey);
-        if (indexKey != null) {
-          // Do not convert to IndexManager.NULL. We are only interested in specific keys
-          this.saveIndexAddition(mapKey, indexKey, value, entry);
+        Object indexKey;
+        if (key == null) {
+          indexKey = QueryService.UNDEFINED;
+        } else {
+          indexKey = ((Map) key).get(mapKey);

Review comment:
       Compiler warning here can be fixed by using `(Map<?, ?>)`

##########
File path: geode-core/src/integrationTest/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
##########
@@ -338,7 +341,302 @@ public void testNullMapValuesInIndexOnLocalRegionForCompactMap() throws Exceptio
     SelectResults result = (SelectResults) qs
         .newQuery("select * from " + SEPARATOR + "portfolio p where p.positions['SUN'] = null")
         .execute();
-    assertEquals(1, result.size());
+    assertEquals(2, result.size());
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithoutIndex() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+    testQueriesForValueInMapField(region, qs);
+  }
+
+  @Test
+  public void testQueriesForValueInMapFieldWithCompactMapIndexWithOneKey() throws Exception {
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+
+    keyIndex1 = qs.createIndex(INDEX_NAME, "positions['SUN']", SEPARATOR + "portfolio ");
+    assertThat(keyIndex1).isInstanceOf(CompactRangeIndex.class);
+    testQueriesForValueInMapField(region, qs);
+
+    long keys = ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfKeys();
+    long mapIndexKeys =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfMapIndexKeys();
+    long values =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getNumberOfValues();
+    long uses =
+        ((CompactRangeIndex) keyIndex1).internalIndexStats.getTotalUses();
+
+    // The number of keys must be equal to the number of different values the
+    // positions map takes in region entries for the 'SUN' key:
+    // ("nothing", "more"). null or UNDEFINED not included
+    assertThat(keys).isEqualTo(2);
+    // mapIndexKeys must be zero because the index used is a range index and not a map index
+    assertThat(mapIndexKeys).isEqualTo(0);
+    // The number of values must be equal to the number of region entries
+    assertThat(values).isEqualTo(7);
+    // The index must be used in every query

Review comment:
       This comment (and other similar ones in subsequent tests) is a little confusing, as there are 5 queries executed in `testQueriesForValueInMapField()`, not 4. Could it be changed to be more clear about which specific queries use the index and which don't?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org