You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2016/02/18 00:10:50 UTC

[15/33] incubator-geode git commit: GEODE-890:CompiledLike member variables are not thread safe

GEODE-890:CompiledLike member variables are not thread safe

Member variables in CompiledLike need to be thread safe.
Added the member variables into the execution context, this should
prevent threads/queries from sharing these variable values and
prevent a possible stack overflow/ infinite loop from occuring.

Added a Multithreaded test helper and test to reproduce the original issue.


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

Branch: refs/heads/feature/GEODE-831
Commit: 09c0c8dd8bc9d83392e53fbb92aee7e45317de1d
Parents: db654a7
Author: Jason Huynh <hu...@gmail.com>
Authored: Wed Feb 10 14:11:41 2016 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Fri Feb 12 10:45:29 2016 -0800

----------------------------------------------------------------------
 .../cache/query/internal/CompiledLike.java      |  55 ++++--
 .../cache/query/internal/ExecutionContext.java  |   4 +
 .../query/internal/QueryExecutionContext.java   |  12 +-
 .../cache/query/MultithreadedTester.java        |  73 +++++++
 .../functional/ParameterBindingJUnitTest.java   | 192 ++++++++++---------
 .../query/internal/QCompilerJUnitTest.java      |  49 ++---
 .../QueryExecutionContextJUnitTest.java         |  63 ++++++
 7 files changed, 310 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledLike.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledLike.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledLike.java
index 6abcd62..9560456 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledLike.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledLike.java
@@ -46,11 +46,11 @@ public class CompiledLike extends CompiledComparison
 
   final static int WILDCARD_UNDERSCORE = 1;
   
-  private int wildcardType = -1;
+  private Object wildcardTypeKey = new Object();
   
-  private int wildcardPosition = -1;
+  private Object wildcardPositionKey = new Object();
   
-  private int patternLength = 0;
+  private Object patternLengthKey = new Object();
   
   final static String LOWEST_STRING = "";
 
@@ -64,7 +64,7 @@ public class CompiledLike extends CompiledComparison
 
   private final CompiledValue var;
 
-  private boolean isIndexEvaluated = false;
+  private Object isIndexEvaluatedKey = new Object();
   
   //private final CompiledBindArgument bindArg;
   private final CompiledValue bindArg;
@@ -75,6 +75,21 @@ public class CompiledLike extends CompiledComparison
     this.bindArg = pattern;
   }
 
+  private int getWildcardPosition(ExecutionContext context) {
+    return (Integer)context.cacheGet(wildcardPositionKey, -1);
+  }
+  
+  private int getWildcardType(ExecutionContext context) {
+    return (Integer)context.cacheGet(wildcardTypeKey, -1);
+  }
+  
+  private int getPatternLength(ExecutionContext context) {
+    return (Integer) context.cacheGet(patternLengthKey, 0);
+  }
+  
+  private boolean getIsIndexEvaluated(ExecutionContext context) {
+    return (Boolean) context.cacheGet(isIndexEvaluatedKey, false);
+  }
   
   OrganizedOperands organizeOperands(ExecutionContext context,
       boolean completeExpansionNeeded, RuntimeIterator[] indpndntItrs)
@@ -88,8 +103,8 @@ public class CompiledLike extends CompiledComparison
     } else {
       // 2 or 3 conditions; create junctions
       if ((getOperator() == OQLLexerTokenTypes.TOK_NE)
-          && (wildcardPosition == patternLength - 1)
-          && (wildcardType == WILDCARD_PERCENT)) {
+          && (getWildcardPosition(context) == getPatternLength(context) - 1)
+          && (getWildcardType(context) == WILDCARD_PERCENT)) {
         // negation supported only for trailing %
         // GroupJunction is created since the boundary conditions go out of
         // range and will be evaluated as false if a RangeJunction was used
@@ -128,13 +143,13 @@ public class CompiledLike extends CompiledComparison
       QueryInvocationTargetException {
     String pattern = (String) this.bindArg.evaluate(context);
     // check if it is filter evaluatable
-    CompiledComparison[] cvs = getRangeIfSargable(this.var, pattern);
+    CompiledComparison[] cvs = getRangeIfSargable(context, this.var, pattern);
 
     for (CompiledComparison cc : cvs) {
       // negation supported only for trailing %
       if ((getOperator() == OQLLexerTokenTypes.TOK_NE)
-          && (wildcardPosition == patternLength - 1)
-          && (wildcardType == WILDCARD_PERCENT)) {
+          && (getWildcardPosition(context) == getPatternLength(context) - 1)
+          && (getWildcardType(context) == WILDCARD_PERCENT)) {
         cc.negate();
       }
       cc.computeDependencies(context);
@@ -202,13 +217,15 @@ public class CompiledLike extends CompiledComparison
    * @param pattern
    * @return The generated CompiledComparisons
    */
-  CompiledComparison[] getRangeIfSargable(CompiledValue var, String pattern) {
+  CompiledComparison[] getRangeIfSargable(ExecutionContext context, CompiledValue var, String pattern) {
     CompiledComparison[] cv = null;
     StringBuffer buffer = new StringBuffer(pattern);
     // check if the string has a % or _ anywhere
-    wildcardPosition = checkIfSargableAndRemoveEscapeChars(buffer);
-    patternLength = buffer.length();
-    isIndexEvaluated = true;
+    int wildcardPosition = checkIfSargableAndRemoveEscapeChars(context, buffer);
+    context.cachePut(wildcardPositionKey, wildcardPosition);
+    int patternLength = buffer.length();
+    context.cachePut(patternLengthKey, patternLength);
+    context.cachePut(isIndexEvaluatedKey, true);
     // if wildcardPosition is >= 0 means it is sargable
     if (wildcardPosition >= 0) {
       int len = patternLength;
@@ -249,7 +266,7 @@ public class CompiledLike extends CompiledComparison
         
         // if % is not the last char in the string.
         // or the wildchar is _ which could be anywhere
-        if (len < (patternLength - 1) || wildcardType == WILDCARD_UNDERSCORE) {
+        if (len < (patternLength - 1) || getWildcardType(context) == WILDCARD_UNDERSCORE) {
           // negation not supported if % is not the last char and also for a _
           // anywhere
           if (getOperator() == OQLLexerTokenTypes.TOK_NE) {
@@ -374,17 +391,17 @@ public class CompiledLike extends CompiledComparison
    * @param buffer
    * @return position of wildcard if sargable otherwise -1
    */
-  int checkIfSargableAndRemoveEscapeChars(StringBuffer buffer) {
+  int checkIfSargableAndRemoveEscapeChars(ExecutionContext context, StringBuffer buffer) {
     int len = buffer.length();
     int wildcardPosition = -1;
     for (int i = 0; i < len; ++i) {
       char ch = buffer.charAt(i);
       if (ch == UNDERSCORE) {
-        wildcardType = WILDCARD_UNDERSCORE;
+        context.cachePut(wildcardTypeKey, WILDCARD_UNDERSCORE);
         wildcardPosition = i; // the position of the wildcard
         break;
       } else if (ch == PERCENT) {
-        wildcardType = WILDCARD_PERCENT;
+        context.cachePut(wildcardTypeKey, WILDCARD_PERCENT);
         wildcardPosition = i; // the position of the wildcard
         break;
       } else if (ch == BACKSLASH) {
@@ -434,7 +451,7 @@ public class CompiledLike extends CompiledComparison
       NameResolutionException, QueryInvocationTargetException
   {
     //reset the isIndexEvaluated flag here since index is not being used here
-    isIndexEvaluated = false;
+    context.cachePut(isIndexEvaluatedKey, false);
 
     Pattern pattern = (Pattern)context.cacheGet(this.bindArg);
     if(pattern == null) {
@@ -484,7 +501,7 @@ public class CompiledLike extends CompiledComparison
      * re-filterevaluation of this CompiledLike.
      */
     PlanInfo result = null;
-    if (isIndexEvaluated) {
+    if (getIsIndexEvaluated(context)) {
       result = new PlanInfo();
       result.evalAsFilter = false;
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/ExecutionContext.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/ExecutionContext.java
index 08212ea..f4df8e4 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/ExecutionContext.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/ExecutionContext.java
@@ -656,6 +656,10 @@ public class ExecutionContext {
   public Object cacheGet(Object key) {
     return null;
   }
+  
+  public Object cacheGet(Object key, Object defaultValue) {
+    return defaultValue;
+  }
 
   public boolean isCqQueryContext() {
     return false;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContext.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContext.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContext.java
index b8b88b2..8f59145 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContext.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContext.java
@@ -116,6 +116,10 @@ public class QueryExecutionContext extends ExecutionContext {
   }
 
   public Object cacheGet(Object key) {
+    return cacheGet(key, null);
+  }
+  
+  public Object cacheGet(Object key, Object defaultValue) {
     //execCache can be empty in cases where we are doing adds to indexes
     //in that case, we use a default execCache
     int scopeId = -1;
@@ -123,7 +127,13 @@ public class QueryExecutionContext extends ExecutionContext {
       scopeId = (Integer) execCacheStack.peek();
     }
     Map execCache = (Map)execCaches.get(scopeId);
-    return execCache != null? execCache.get(key): null;
+    if (execCache == null) {
+      return defaultValue;
+    }
+    if (execCache.containsKey(key)) {
+      return execCache.get(key);
+    }
+    return defaultValue;
   }
 
   public void pushExecCache(int scopeNum) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/MultithreadedTester.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/MultithreadedTester.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/MultithreadedTester.java
new file mode 100644
index 0000000..3142566
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/MultithreadedTester.java
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+
+public class MultithreadedTester {
+
+  
+  public static Collection<Object> runMultithreaded(Collection<Callable> callables) throws InterruptedException {
+    final CountDownLatch allRunnablesAreSubmitted = new CountDownLatch(1);
+    final CountDownLatch callablesComplete = new CountDownLatch(callables.size());
+    final ExecutorService executor = Executors.newFixedThreadPool(callables.size());
+    final LinkedList<Future> futures = new LinkedList<>();
+    //Submit all tasks to the executor
+    callables.parallelStream().forEach(callable -> {
+      futures.add(executor.submit(() -> {
+        try {
+          allRunnablesAreSubmitted.await(60, TimeUnit.SECONDS);
+          return callable.call();
+        }
+        catch (Throwable t) {
+          return t;
+        }
+        finally {
+          callablesComplete.countDown();
+        }
+      }));
+    });
+    //Unlock all tasks
+    allRunnablesAreSubmitted.countDown();
+    //Wait until all tasks are complete
+    callablesComplete.await(60, TimeUnit.SECONDS);
+    executor.shutdown();
+    return convertFutureToResult(futures);
+  }
+
+  private static Collection<Object> convertFutureToResult(final Collection<Future> futures) {
+    List<Object> results = new LinkedList<Object>();
+    futures.forEach(future-> {
+      try {
+        results.add(future.get());
+      }
+      catch (Exception e) {
+        results.add(e);
+      }
+    });
+    return results;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/ParameterBindingJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/ParameterBindingJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/ParameterBindingJUnitTest.java
index 9b34aa9..fb42f0a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/ParameterBindingJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/ParameterBindingJUnitTest.java
@@ -22,12 +22,16 @@
  */
 package com.gemstone.gemfire.cache.query.functional;
 
-import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.stream.IntStream;
 
 import org.junit.After;
 import org.junit.Before;
@@ -36,7 +40,10 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.query.CacheUtils;
+import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.MultithreadedTester;
 import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.cache.query.QueryService;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -46,129 +53,126 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
  */
 @Category(IntegrationTest.class)
 public class ParameterBindingJUnitTest {
-  
+  String regionName = "Portfolios";
+
   @Before
   public void setUp() throws java.lang.Exception {
     CacheUtils.startCache();
-    Region region = CacheUtils.createRegion("Portfolios", Portfolio.class);
-    region.put("0",new Portfolio(0));
-    region.put("1",new Portfolio(1));
-    region.put("2",new Portfolio(2));
-    region.put("3",new Portfolio(3));
   }
-  
+
+  private Region createRegion(String regionName) {
+    return CacheUtils.createRegion(regionName, Portfolio.class);
+  }
+
+  private Index createIndex(String indexName, String indexedExpression, String regionPath) throws Exception {
+    QueryService qs = CacheUtils.getQueryService();
+    return qs.createIndex(indexName, indexedExpression, regionPath);
+  }
+
+  private void populateRegion(Region region, int numEntries) {
+    IntStream.range(0, numEntries).parallel().forEach(i -> {
+      region.put("" + i, new Portfolio(i));
+    });
+  }
+
+  private Region createAndPopulateRegion(String regionName, int numEntries) {
+    Region region = createRegion(regionName);
+    populateRegion(region, numEntries);
+    return region;
+  }
+
   @After
   public void tearDown() throws java.lang.Exception {
     CacheUtils.closeCache();
   }
-  
+
+  private void validateQueryWithBindParameter(String queryString, Object[] bindParameters, int expectedSize) throws Exception {
+    Query query = CacheUtils.getQueryService().newQuery(queryString);
+    Object result = query.execute(bindParameters);
+    assertEquals(expectedSize, ((Collection) result).size());
+  }
+
   @Test
   public void testBindCollectionInFromClause() throws Exception {
-    Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM $1 ");
-    Object params[] = new Object[1];
-    Region region = CacheUtils.getRegion("/Portfolios");
-    params[0] = region.values();
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != region.values().size())
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
+    Object params[] = new Object[] { region.values() };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM $1 ", params, numEntries);
   }
-  
+
   @Test
   public void testBindArrayInFromClause() throws Exception {
-    Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM $1 ");
-    Object params[] = new Object[1];
-    Region region = CacheUtils.getRegion("/Portfolios");
-    params[0] = region.values().toArray();
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != region.values().size())
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
+    Object params[] = new Object[] { region.values().toArray() };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM $1 ", params, numEntries);
   }
-  
+
   @Test
   public void testBindMapInFromClause() throws Exception {
-    Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM $1 ");
-    Object params[] = new Object[1];
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
     Map map = new HashMap();
-    Region region = CacheUtils.getRegion("/Portfolios");
-    Iterator iter = region.entries(false).iterator();
-    while(iter.hasNext()){
-      Region.Entry entry = (Region.Entry)iter.next();
+    Iterator iter = region.entrySet().iterator();
+    while (iter.hasNext()) {
+      Region.Entry entry = (Region.Entry) iter.next();
       map.put(entry.getKey(), entry.getValue());
     }
-    params[0] = map;
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != region.values().size())
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    Object params[] = new Object[] { map };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM $1 ", params, numEntries);
   }
-  
+
   @Test
   public void testBindRegionInFromClause() throws Exception {
-    Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM $1 ");
-    Object params[] = new Object[1];
-    Region region = CacheUtils.getRegion("/Portfolios");
-    params[0] = region;
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != region.values().size())
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
+    Object params[] = new Object[] { region };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM $1 ", params, numEntries);
   }
-  
-  
+
   @Test
-  public void testBindValueAsMethodParamter() throws Exception {
+  public void testStringBindValueAsMethodParameter() throws Exception {
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
     Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM /Portfolios where status.equals($1)");
-    Object params[] = new Object[1];
-    params[0] = "active";
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != 2)
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    Object params[] = new Object[] { "active" };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM /Portfolios where status.equals($1)", params, 2);
   }
-  
+
   @Test
-  public void testBindString() throws Exception {
+  public void testBindStringAsBindParameter() throws Exception {
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
     Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM /Portfolios where status = $1");
-    Object params[] = new Object[1];
-    params[0] = "active";
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != 2)
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    Object params[] = new Object[] { "active" };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM /Portfolios where status = $1", params, 2);
   }
-  
+
   @Test
   public void testBindInt() throws Exception {
-    Query query = CacheUtils.getQueryService().newQuery("SELECT DISTINCT * FROM /Portfolios where ID = $1");
-    Object params[] = new Object[1];
-    params[0] = new Integer(1);
-    Object result = query.execute(params);
-    if(result instanceof Collection){
-      int resultSize = ((Collection)result).size();
-      if( resultSize != 1)
-        fail("Results not as expected");
-    }else
-      fail("Invalid result");
+    int numEntries = 4;
+    Region region = createAndPopulateRegion(regionName, numEntries);
+    Object params[] = new Object[] { new Integer(1) };
+    validateQueryWithBindParameter("SELECT DISTINCT * FROM /Portfolios where ID = $1", params, 1);
+  }
+
+  @Test
+  public void testMultithreadedBindUsingSameQueryObject() throws Exception {
+    int numObjects = 10000;
+    Region region = createAndPopulateRegion("Portfolios", numObjects);
+    createIndex("Status Index", "status", "/Portfolios");
+    final Query query = CacheUtils.getQueryService().newQuery("SELECT * FROM /Portfolios where status like $1");
+    final Object[] bindParam = new Object[] { "%a%" };
+    Collection<Callable> callables = new ConcurrentLinkedQueue<>();
+    IntStream.range(0, 1000).parallel().forEach(i -> {
+      callables.add(() -> {
+        return query.execute(bindParam);
+      });
+    });
+    Collection<Object> results = MultithreadedTester.runMultithreaded(callables);
+    results.forEach(result -> {
+      assertTrue(result.getClass().getName() + " was not an expected result", result instanceof Collection);
+      assertEquals(numObjects, ((Collection)result).size());
+    });
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QCompilerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QCompilerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QCompilerJUnitTest.java
index 2510c36..dd714d3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QCompilerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QCompilerJUnitTest.java
@@ -46,82 +46,83 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 @Category(IntegrationTest.class)
 public class QCompilerJUnitTest {
 
+  QueryExecutionContext context = new QueryExecutionContext(null, null);
   @Test
   public void testStringConditioningForLike_1() {
     String s1 = "abc%";
     StringBuffer buffer = new StringBuffer(s1);
     CompiledLike cl = new CompiledLike(null, null);
-    int wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    int wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(3,wildCardPosition);
     assertEquals(s1, buffer.toString());
     
     s1 = "abc\\%abc";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(-1,wildCardPosition);
     assertEquals(buffer.toString(), "abc%abc");
     
     s1 = "abc\\\\%abc";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(4,wildCardPosition);
     assertEquals(buffer.toString(), "abc\\%abc");
 
     s1 = "abc\\\\\\%abc";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(-1,wildCardPosition);
     assertEquals(buffer.toString(), "abc\\%abc");
 
     s1 = "%";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(0,wildCardPosition);
     assertEquals(buffer.toString(), s1);
 
     s1 = "%abc";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(0,wildCardPosition);
     assertEquals(buffer.toString(), "%abc");
 
     s1 = "%%abc";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(0,wildCardPosition);
     assertEquals(buffer.toString(), "%%abc");
 
     s1 = "%\\%abc";
     buffer = new StringBuffer(s1);
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(0,wildCardPosition);
     assertEquals(buffer.toString(), "%\\%abc");
 
     s1 = "_abc";
     buffer = new StringBuffer(s1);
 
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(0,wildCardPosition);
     assertEquals(buffer.toString(), "_abc");
     
     s1 = "\\_abc";
     buffer = new StringBuffer(s1);
     
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(-1,wildCardPosition);
     assertEquals(buffer.toString(), "_abc");
     
     s1 = "ab\\%c%d";
     buffer = new StringBuffer(s1);
     
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(4,wildCardPosition);
     assertEquals(buffer.toString(), "ab%c%d");
     
     s1 = "ab\\__d";
     buffer = new StringBuffer(s1);
     
-    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(buffer);
+    wildCardPosition = cl.checkIfSargableAndRemoveEscapeChars(context, buffer);
     assertEquals(3,wildCardPosition);
     assertEquals(buffer.toString(), "ab__d");
   }
@@ -132,7 +133,7 @@ public class QCompilerJUnitTest {
     CompiledLiteral literal = new CompiledLiteral(pattern);
     CompiledID cid = new CompiledID("val");
     CompiledLike cl = new CompiledLike(cid, literal);
-    CompiledComparison[] cc = cl.getRangeIfSargable(cid, pattern);
+    CompiledComparison[] cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(1, cc.length);
    
     ArrayList cv = (ArrayList) cc[0].getChildren();
@@ -145,7 +146,7 @@ public class QCompilerJUnitTest {
     pattern = "ab%";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(2, cc.length);
     
     cv = (ArrayList) cc[0].getChildren();
@@ -165,7 +166,7 @@ public class QCompilerJUnitTest {
     pattern = "a%c";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(3, cc.length);
 
     cv = (ArrayList) cc[0].getChildren();
@@ -187,7 +188,7 @@ public class QCompilerJUnitTest {
     pattern = "%bc";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(2, cc.length);
  
     cv = (ArrayList) cc[0].getChildren();
@@ -202,7 +203,7 @@ public class QCompilerJUnitTest {
     pattern = "ab_";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(3, cc.length);
     
     cv = (ArrayList) cc[0].getChildren();
@@ -224,7 +225,7 @@ public class QCompilerJUnitTest {
     pattern = "_bc";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(2, cc.length);
 
     cv = (ArrayList) cc[0].getChildren();
@@ -239,7 +240,7 @@ public class QCompilerJUnitTest {
     pattern = "a_c";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(3, cc.length);
     
     cv = (ArrayList) cc[0].getChildren();
@@ -261,7 +262,7 @@ public class QCompilerJUnitTest {
     pattern = "_b%";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(2, cc.length);
 
     cv = (ArrayList) cc[0].getChildren();
@@ -276,7 +277,7 @@ public class QCompilerJUnitTest {
     pattern = "a\\%bc";
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(1, cc.length);
 
     cv = (ArrayList) cc[0].getChildren();
@@ -290,7 +291,7 @@ public class QCompilerJUnitTest {
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
     cl.negate();
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context,cid, pattern);
     assertEquals(2, cc.length);
     
     cv = (ArrayList) cc[0].getChildren();
@@ -311,7 +312,7 @@ public class QCompilerJUnitTest {
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
     cl.negate();
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context, cid, pattern);
     assertEquals(2, cc.length);
 
     cv = (ArrayList) cc[0].getChildren();
@@ -327,7 +328,7 @@ public class QCompilerJUnitTest {
     literal = new CompiledLiteral(pattern);
     cl = new CompiledLike(cid, literal);
     cl.negate();
-    cc = cl.getRangeIfSargable(cid, pattern);
+    cc = cl.getRangeIfSargable(context, cid, pattern);
     assertEquals(2, cc.length);
 
     cv = (ArrayList) cc[0].getChildren();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09c0c8dd/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContextJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContextJUnitTest.java
new file mode 100644
index 0000000..ed789c5
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryExecutionContextJUnitTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.internal;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import com.gemstone.gemfire.cache.query.internal.QueryExecutionContext;
+
+public class QueryExecutionContextJUnitTest {
+
+  @Test
+  public void testNullReturnedFromCacheGetWhenNoValueWasPut() {
+    Object key = new Object();
+    QueryExecutionContext context = new QueryExecutionContext(null, null);
+    assertNull(context.cacheGet(key));
+  }
+  
+  @Test
+  public void testPutValueReturnedFromCacheGet() {
+    Object key = new Object();
+    Object value = new Object();
+    QueryExecutionContext context = new QueryExecutionContext(null, null);
+    context.cachePut(key, value);
+    assertEquals(value, context.cacheGet(key));
+  }
+  
+  @Test
+  public void testDefaultReturnedFromCacheGetWhenNoValueWasPut() {
+    Object key = new Object();
+    Object value = new Object();
+    QueryExecutionContext context = new QueryExecutionContext(null, null);
+    assertEquals(value, context.cacheGet(key, value));
+  }
+  
+  @Test
+  public void testExecCachesCanBePushedAndValuesRetrievedAtTheCorrectLevel() {
+    Object key = new Object();
+    Object value = new Object();
+    QueryExecutionContext context = new QueryExecutionContext(null, null);
+    context.pushExecCache(1);
+    context.cachePut(key, value);
+    context.pushExecCache(2);
+    assertNull(context.cacheGet(key));
+    context.popExecCache();
+    assertEquals(value, context.cacheGet(key));
+  }
+  
+}