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

[06/16] geode git commit: GEODE-2185: Index not used with parameterized query

GEODE-2185: Index not used with parameterized query


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

Branch: refs/heads/feature/GEODE-1027
Commit: 7fb0e68b130317ac3cce0fa1ca8f39fb0006720f
Parents: c2b1c8b
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Dec 6 10:21:01 2016 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Wed Dec 7 09:52:15 2016 -0800

----------------------------------------------------------------------
 .../query/internal/CompiledBindArgument.java    | 18 ++++--
 .../internal/IndexTrackingQueryObserver.java    | 33 ++++++-----
 .../geode/cache/query/data/Instrument.java      | 60 ++++++++++++++++++++
 .../geode/cache/query/data/TradingLine.java     | 41 +++++++++++++
 .../MapRangeIndexMaintenanceJUnitTest.java      | 49 ++++++++++++++++
 5 files changed, 182 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/7fb0e68b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
index cb9218b..106d389 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
@@ -46,13 +46,19 @@ public class CompiledBindArgument extends AbstractCompiledValue {
   public void generateCanonicalizedExpression(StringBuffer clauseBuffer, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
 
-    Object bindArg;
-    if (context.isBindArgsSet()
-        && (bindArg = context.getBindArgument(this.index)) instanceof Region) {
-      clauseBuffer.insert(0, ((Region) bindArg).getFullPath());
-    } else {
-      clauseBuffer.insert(0, "$" + this.index);
+    // When compiling a new query, a context is created where there are no bind arguments at this
+    // point
+    if (context.isBindArgsSet()) {
+      Object bindArgumentValue = context.getBindArgument(this.index);
+      if (bindArgumentValue instanceof Region) {
+        clauseBuffer.insert(0, ((Region) bindArgumentValue).getFullPath());
+      } else if (bindArgumentValue instanceof String) {
+        clauseBuffer.insert(0, '\'').insert(0, bindArgumentValue).insert(0, '\'');
+      } else {
+        super.generateCanonicalizedExpression(clauseBuffer, context);
+      }
     }
+
   }
 
   public Object evaluate(ExecutionContext context) {

http://git-wip-us.apache.org/repos/asf/geode/blob/7fb0e68b/geode-core/src/main/java/org/apache/geode/cache/query/internal/IndexTrackingQueryObserver.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/IndexTrackingQueryObserver.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/IndexTrackingQueryObserver.java
index a643b59..da223d5 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/IndexTrackingQueryObserver.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/IndexTrackingQueryObserver.java
@@ -40,6 +40,7 @@ import org.apache.geode.internal.cache.PartitionedRegionQueryEvaluator.TestHook;
 public class IndexTrackingQueryObserver extends QueryObserverAdapter {
 
   private static final ThreadLocal indexInfo = new ThreadLocal();
+  private static final ThreadLocal lastKeyUsed = new ThreadLocal();
   private static final ThreadLocal lastIndexUsed = new ThreadLocal();
   private volatile TestHook th;
 
@@ -50,15 +51,7 @@ public class IndexTrackingQueryObserver extends QueryObserverAdapter {
       this.indexInfo.set(indexMap);
     }
     IndexInfo iInfo;
-    String indexName;
-    // Dont create new IndexInfo if one is already there in map for aggregation
-    // of results later for whole partition region on this node.
-    if ((index instanceof MapRangeIndex || index instanceof CompactMapRangeIndex)
-        && key instanceof Object[]) {
-      indexName = index.getName() + "-" + ((Object[]) key)[1];
-    } else {
-      indexName = index.getName();
-    }
+    String indexName = getIndexName(index, key);
     if (indexMap.containsKey(indexName)) {
       iInfo = indexMap.get(indexName);
     } else {
@@ -67,6 +60,7 @@ public class IndexTrackingQueryObserver extends QueryObserverAdapter {
     iInfo.addRegionId(index.getRegion().getFullPath());
     indexMap.put(indexName, iInfo);
     this.lastIndexUsed.set(index);
+    this.lastKeyUsed.set(key);
     if (th != null) {
       th.hook(1);
     }
@@ -108,19 +102,32 @@ public class IndexTrackingQueryObserver extends QueryObserverAdapter {
     // append the size of the lookup results (and bucket id if its an Index on bucket)
     // to IndexInfo results Map.
     Map indexMap = (Map) this.indexInfo.get();
-    if (lastIndexUsed.get() != null) {
-      IndexInfo indexInfo = (IndexInfo) indexMap.get(((Index) this.lastIndexUsed.get()).getName());
+    Index index = (Index) lastIndexUsed.get();
+    if (index != null) {
+      IndexInfo indexInfo = (IndexInfo) indexMap.get(getIndexName(index, this.lastKeyUsed.get()));
       if (indexInfo != null) {
-        indexInfo.getResults().put(((Index) this.lastIndexUsed.get()).getRegion().getFullPath(),
-            new Integer(results.size()));
+        indexInfo.getResults().put(index.getRegion().getFullPath(), new Integer(results.size()));
       }
     }
     this.lastIndexUsed.set(null);
+    this.lastKeyUsed.set(null);
     if (th != null) {
       th.hook(3);
     }
   }
 
+  private String getIndexName(Index index, Object key) {
+    String indexName;
+    if ((index instanceof MapRangeIndex || index instanceof CompactMapRangeIndex)
+        && key instanceof Object[]) {
+      indexName = index.getName() + "-" + ((Object[]) key)[1];
+    } else {
+      indexName = index.getName();
+    }
+    return indexName;
+  }
+
+
   /**
    * This should be called only when one query execution on one gemfire node is done. NOT for each
    * buckets.

http://git-wip-us.apache.org/repos/asf/geode/blob/7fb0e68b/geode-core/src/test/java/org/apache/geode/cache/query/data/Instrument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/data/Instrument.java b/geode-core/src/test/java/org/apache/geode/cache/query/data/Instrument.java
new file mode 100644
index 0000000..131efa6
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/data/Instrument.java
@@ -0,0 +1,60 @@
+/*
+ * 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.query.data;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class Instrument implements Serializable {
+
+  private String id;
+
+  private List<TradingLine> tradingLines;
+
+  public Instrument(String id) {
+    this.id = id;
+    this.tradingLines = new ArrayList<>();
+  }
+
+  public String getId() {
+    return this.id;
+  }
+
+  public void addTradingLine(TradingLine tl) {
+    this.tradingLines.add(tl);
+  }
+
+  // This method is needed for the query
+  public List<TradingLine> getTradingLines() {
+    return this.tradingLines;
+  }
+
+  public String toString() {
+    return new StringBuilder().append(getClass().getSimpleName()).append("[").append("id=")
+        .append(this.id).append("; tradingLines=").append(this.tradingLines).append("]").toString();
+  }
+
+  public static Instrument getInstrument(String id) {
+    Instrument inst = new Instrument(id);
+    for (int i = 0; i < 5; i++) {
+      TradingLine tl = new TradingLine();
+      tl.addAlternateReference("SOME_KEY", "SOME_VALUE");
+      tl.addAlternateReference("SOME_OTHER_KEY", "SOME_OTHER_VALUE");
+      inst.addTradingLine(tl);
+    }
+    return inst;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7fb0e68b/geode-core/src/test/java/org/apache/geode/cache/query/data/TradingLine.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/data/TradingLine.java b/geode-core/src/test/java/org/apache/geode/cache/query/data/TradingLine.java
new file mode 100644
index 0000000..55a5f43
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/data/TradingLine.java
@@ -0,0 +1,41 @@
+/*
+ * 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.query.data;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class TradingLine {
+
+  private Map<String, String> alternateReferences;
+
+  public TradingLine() {
+    this.alternateReferences = new HashMap<String, String>();
+  }
+
+  public void addAlternateReference(String key, String value) {
+    this.alternateReferences.put(key, value);
+  }
+
+  // This method is needed for the query
+  public Map<String, String> getAlternateReferences() {
+    return this.alternateReferences;
+  }
+
+  public String toString() {
+    return new StringBuilder().append(getClass().getSimpleName()).append("[")
+        .append("; alternateReferences=").append(this.alternateReferences).append("]").toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7fb0e68b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
index 5159f89..988defb 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
@@ -21,8 +21,13 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.Collection;
 import java.util.HashMap;
 
+import org.apache.geode.cache.query.Query;
+import org.apache.geode.cache.query.data.Instrument;
+import org.apache.geode.cache.query.internal.IndexTrackingQueryObserver;
+import org.apache.geode.cache.query.internal.QueryObserverHolder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -65,6 +70,50 @@ public class MapRangeIndexMaintenanceJUnitTest {
   }
 
   @Test
+  public void testMapIndexIsUsedWithBindKeyParameter() throws Exception {
+    // Create Region
+    region =
+        CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("instrument");
+
+    // Initialize observer
+    MyQueryObserverAdapter observer = new MyQueryObserverAdapter();
+    QueryObserverHolder.setInstance(observer);
+
+    // Create map index
+    qs = CacheUtils.getQueryService();
+    qs.createIndex(INDEX_NAME, "tl.alternateReferences['SOME_KEY', 'SOME_OTHER_KEY']",
+        "/instrument i, i.tradingLines tl");
+
+    // Add instruments
+    int numInstruments = 20;
+    for (int i = 0; i < numInstruments; i++) {
+      String key = String.valueOf(i);
+      Object value = Instrument.getInstrument(key);
+      region.put(key, value);
+    }
+
+    // Execute query
+    Query query = qs.newQuery(
+        "<trace> select distinct i from /instrument i, i.tradingLines t where t.alternateReferences[$1]='SOME_VALUE'");
+    SelectResults results = (SelectResults) query.execute(new Object[] {"SOME_KEY"});
+
+    // Verify index was used
+    assertTrue(observer.indexUsed);
+
+    // Verify the results size
+    assertEquals(numInstruments, results.size());
+  }
+
+  private static class MyQueryObserverAdapter extends IndexTrackingQueryObserver {
+    public boolean indexUsed = false;
+
+    public void afterIndexLookup(Collection results) {
+      super.afterIndexLookup(results);
+      indexUsed = true;
+    }
+  }
+
+  @Test
   public void testNullMapKeysInIndexOnLocalRegionForCompactMap() throws Exception {
 
     // Create Partition Region