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 2017/05/11 21:23:12 UTC

[06/50] [abbrv] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/CacheUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/CacheUtils.java b/geode-core/src/test/java/org/apache/geode/cache/query/CacheUtils.java
index da79ae0..e5fa846 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/CacheUtils.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/CacheUtils.java
@@ -12,49 +12,52 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-/*
- * Utils.java
- *
- * Created on March 8, 2005, 4:16 PM
- */
 package org.apache.geode.cache.query;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.junit.Assert.*;
+
+import java.util.Iterator;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.LogWriter;
-import org.apache.geode.cache.*;
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.CacheTransactionManager;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.query.types.CollectionType;
 import org.apache.geode.cache.query.types.ObjectType;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
 
-import java.util.Iterator;
-import java.util.Properties;
-import java.util.Set;
+import org.apache.geode.cache.TimeoutException;
+import org.apache.geode.cache.CacheWriterException;
+import org.apache.geode.cache.GatewayException;
+import org.apache.geode.cache.RegionExistsException;
 
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-/**
- * 
- */
 public class CacheUtils {
+  private static final Logger logger = LogService.getLogger();
 
-  static Properties props = new Properties();
-  static DistributedSystem ds;
-  static volatile Cache cache;
+  private static Properties props = new Properties();
+  private static DistributedSystem ds;
+  static volatile InternalCache cache;
   static QueryService qs;
   static {
-    try {
-      init();
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
+    init();
   }
 
-  static void init() throws Exception {
+  private static void init()
+      throws TimeoutException, CacheWriterException, GatewayException, RegionExistsException {
     if (GemFireCacheImpl.getInstance() == null) {
       props.setProperty(MCAST_PORT, "0");
-      cache = new CacheFactory(props).create();
+      cache = (InternalCache) new CacheFactory(props).create();
     } else {
       cache = GemFireCacheImpl.getInstance();
     }
@@ -62,19 +65,19 @@ public class CacheUtils {
     qs = cache.getQueryService();
   }
 
-  public static Cache getCache() {
+  public static InternalCache getCache() {
     return cache;
   }
 
   public static void startCache() {
     try {
       if (cache.isClosed()) {
-        cache = new CacheFactory(props).create();
+        cache = (InternalCache) new CacheFactory(props).create();
         ds = cache.getDistributedSystem();
         qs = cache.getQueryService();
       }
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
   }
 
@@ -84,7 +87,7 @@ public class CacheUtils {
         cache.close();
       }
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
   }
 
@@ -93,11 +96,11 @@ public class CacheUtils {
       if (!cache.isClosed()) {
         cache.close();
       }
-      cache = new CacheFactory(props).create();
+      cache = (InternalCache) new CacheFactory(props).create();
       ds = cache.getDistributedSystem();
       qs = cache.getQueryService();
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
   }
 
@@ -109,23 +112,20 @@ public class CacheUtils {
         attributesFactory.setScope(scope);
       }
       RegionAttributes regionAttributes = attributesFactory.create();
-      Region region = cache.createRegion(regionName, regionAttributes);
-      return region;
+      return cache.createRegion(regionName, regionAttributes);
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
-    return null;
   }
 
+  // TODO: paramter flag is unused
   public static Region createRegion(String regionName, RegionAttributes regionAttributes,
       boolean flag) {
     try {
-      Region region = cache.createRegion(regionName, regionAttributes);
-      return region;
+      return cache.createRegion(regionName, regionAttributes);
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
-    return null;
   }
 
   public static Region createRegion(String regionName, Class valueConstraint) {
@@ -139,12 +139,10 @@ public class CacheUtils {
       attributesFactory.setValueConstraint(valueConstraint);
       attributesFactory.setIndexMaintenanceSynchronous(indexMaintenanceSynchronous);
       RegionAttributes regionAttributes = attributesFactory.create();
-      Region region = cache.createRegion(regionName, regionAttributes);
-      return region;
+      return cache.createRegion(regionName, regionAttributes);
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
-    return null;
   }
 
   public static Region createRegion(Region parentRegion, String regionName, Class valueConstraint) {
@@ -153,12 +151,10 @@ public class CacheUtils {
       if (valueConstraint != null)
         attributesFactory.setValueConstraint(valueConstraint);
       RegionAttributes regionAttributes = attributesFactory.create();
-      Region region = parentRegion.createSubregion(regionName, regionAttributes);
-      return region;
+      return parentRegion.createSubregion(regionName, regionAttributes);
     } catch (Exception e) {
-      e.printStackTrace();
+      throw new AssertionError(e);
     }
-    return null;
   }
 
   public static Region getRegion(String regionPath) {
@@ -179,10 +175,7 @@ public class CacheUtils {
   }
 
   public static void log(Object message) {
-    Cache cache = GemFireCacheImpl.getInstance();
-    if (cache != null) {
-      cache.getLogger().fine(message.toString());
-    }
+    logger.debug(message);
   }
 
   public static CacheTransactionManager getCacheTranxnMgr() {
@@ -190,16 +183,19 @@ public class CacheUtils {
   }
 
   public static void compareResultsOfWithAndWithoutIndex(SelectResults[][] r, Object test) {
-    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();
+    Set set1;
+    Set set2;
+    Iterator itert1;
+    Iterator itert2;
+    ObjectType type1;
+    ObjectType type2;
+
+    for (final SelectResults[] selectResults : r) {
+      CollectionType collType1 = selectResults[0].getCollectionType();
+      CollectionType collType2 = selectResults[1].getCollectionType();
       type1 = collType1.getElementType();
       type2 = collType2.getElementType();
+
       if (collType1.getSimpleClassName().equals(collType2.getSimpleClassName())) {
         log("Both SelectResults are of the same Type i.e.--> " + collType1);
       } else {
@@ -208,6 +204,7 @@ public class CacheUtils {
             "FAILED:Select results Collection Type is different in both the cases. CollectionType1="
                 + collType1 + " CollectionType2=" + collType2);
       }
+
       if (type1.equals(type2)) {
         log("Both SelectResults have same element Type i.e.--> " + type1);
       } else {
@@ -224,15 +221,17 @@ public class CacheUtils {
         fail("FAILED:SelectResults Collection Type is different in both the cases. CollType1="
             + collType1 + " CollType2=" + collType2);
       }
-      if (r[j][0].size() == r[j][1].size()) {
-        log("Both SelectResults are of Same Size i.e.  Size= " + r[j][1].size());
+
+      if (selectResults[0].size() == selectResults[1].size()) {
+        log("Both SelectResults are of Same Size i.e.  Size= " + selectResults[1].size());
       } else {
-        fail("FAILED:SelectResults size is different in both the cases. Size1=" + r[j][0].size()
-            + " Size2 = " + r[j][1].size());
+        fail("FAILED:SelectResults size is different in both the cases. Size1="
+            + selectResults[0].size() + " Size2 = " + selectResults[1].size());
       }
-      set2 = ((r[j][1]).asSet());
-      set1 = ((r[j][0]).asSet());
-      // boolean pass = true;
+
+      set2 = selectResults[1].asSet();
+      set1 = selectResults[0].asSet();
+
       itert1 = set1.iterator();
       while (itert1.hasNext()) {
         Object p1 = itert1.next();
@@ -241,6 +240,7 @@ public class CacheUtils {
         boolean exactMatch = false;
         while (itert2.hasNext()) {
           Object p2 = itert2.next();
+
           if (p1 instanceof Struct) {
             Object[] values1 = ((Struct) p1).getFieldValues();
             Object[] values2 = ((Struct) p2).getFieldValues();
@@ -248,11 +248,11 @@ public class CacheUtils {
             boolean elementEqual = true;
             for (int i = 0; i < values1.length; ++i) {
               elementEqual =
-                  elementEqual && ((values1[i] == values2[i]) || values1[i].equals(values2[i]));
+                  elementEqual && (values1[i] == values2[i] || values1[i].equals(values2[i]));
             }
             exactMatch = elementEqual;
           } else {
-            exactMatch = (p2 == p1) || p2.equals(p1);
+            exactMatch = p2 == p1 || p2.equals(p1);
           }
           if (exactMatch) {
             break;
@@ -260,7 +260,7 @@ public class CacheUtils {
         }
         if (!exactMatch) {
           fail(
-              "Atleast one element in the pair of SelectResults supposedly identical, is not equal ");
+              "At least one element in the pair of SelectResults supposedly identical, is not equal");
         }
       }
     }
@@ -268,14 +268,17 @@ public class CacheUtils {
 
   public static boolean compareResultsOfWithAndWithoutIndex(SelectResults[][] r) {
     boolean ok = true;
-    Set set1 = null;
-    Set set2 = null;
-    Iterator itert1 = null;
-    Iterator itert2 = null;
-    ObjectType type1, type2;
-    outer: for (int j = 0; j < r.length; j++) {
-      CollectionType collType1 = r[j][0].getCollectionType();
-      CollectionType collType2 = r[j][1].getCollectionType();
+    Set set1;
+    Set set2;
+    Iterator itert1;
+    Iterator itert2;
+    ObjectType type1;
+    ObjectType type2;
+
+    // TODO: eliminate loop labels
+    outer: for (final SelectResults[] aR : r) {
+      CollectionType collType1 = aR[0].getCollectionType();
+      CollectionType collType2 = aR[1].getCollectionType();
       type1 = collType1.getElementType();
       type2 = collType2.getElementType();
 
@@ -288,6 +291,7 @@ public class CacheUtils {
         ok = false;
         break;
       }
+
       if (type1.equals(type2)) {
         log("Both SelectResults have same element Type i.e.--> " + type1);
       } else {
@@ -308,18 +312,20 @@ public class CacheUtils {
         ok = false;
         break;
       }
-      if (r[j][0].size() == r[j][1].size()) {
-        log("Both SelectResults are of Same Size i.e.  Size= " + r[j][1].size());
+
+      if (aR[0].size() == aR[1].size()) {
+        log("Both SelectResults are of Same Size i.e.  Size= " + aR[1].size());
       } else {
         // test.fail("FAILED:SelectResults size is different in both the cases. Size1=" +
         // r[j][0].size() + " Size2 = " + r[j][1].size());
         ok = false;
         break;
       }
-      set2 = (((SelectResults) r[j][1]).asSet());
-      set1 = (((SelectResults) r[j][0]).asSet());
-      boolean pass = true;
+
+      set2 = aR[1].asSet();
+      set1 = aR[0].asSet();
       itert1 = set1.iterator();
+
       while (itert1.hasNext()) {
         Object p1 = itert1.next();
         itert2 = set2.iterator();
@@ -330,7 +336,6 @@ public class CacheUtils {
           if (p1 instanceof Struct) {
             Object[] values1 = ((Struct) p1).getFieldValues();
             Object[] values2 = ((Struct) p2).getFieldValues();
-            // test.assertIndexDetailsEquals(values1.length, values2.length);
             if (values1.length != values2.length) {
               ok = false;
               break outer;
@@ -339,22 +344,20 @@ public class CacheUtils {
             for (int i = 0; i < values1.length; ++i) {
               if (values1[i] != null) {
                 elementEqual =
-                    elementEqual && ((values1[i] == values2[i]) || values1[i].equals(values2[i]));
+                    elementEqual && (values1[i] == values2[i] || values1[i].equals(values2[i]));
               } else {
-                elementEqual = elementEqual && ((values1[i] == values2[i]));
+                elementEqual = elementEqual && values1[i] == values2[i];
               }
             }
             exactMatch = elementEqual;
           } else {
-            exactMatch = (p2 == p1) || p2.equals(p1);
+            exactMatch = p2 == p1 || p2.equals(p1);
           }
           if (exactMatch) {
             break;
           }
         }
         if (!exactMatch) {
-          // test.fail("Atleast one element in the pair of SelectResults supposedly identical, is
-          // not equal ");
           ok = false;
           break outer;
         }

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/functional/FunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/functional/FunctionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/functional/FunctionJUnitTest.java
index 8bb2610..b8451bd 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/functional/FunctionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/functional/FunctionJUnitTest.java
@@ -80,14 +80,14 @@ public class FunctionJUnitTest {
     ExecutionContext context = null;
     for (int i = 0; i < 6; i++) {
       CompiledValue cf = new CompiledFunction(cvArr[i], OQLLexerTokenTypes.LITERAL_nvl);
-      StringBuffer clauseBuffer = new StringBuffer();
+      StringBuilder clauseBuffer = new StringBuilder();
       cf.generateCanonicalizedExpression(clauseBuffer, context);
       if (!clauseBuffer.toString().equals("NVL" + canonicalizedArgs[i])) {
         fail("Canonicalization not done properly");
       }
 
       cf = new CompiledFunction(cvArr[i], OQLLexerTokenTypes.LITERAL_element);
-      clauseBuffer = new StringBuffer();
+      clauseBuffer = new StringBuilder();
       cf.generateCanonicalizedExpression(clauseBuffer, context);
       if (!clauseBuffer.toString().equals("ELEMENT" + canonicalizedArgs[i])) {
         fail("Canonicalization not done properly");

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledAggregateFunctionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
index f3bcc02..5242bbe 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
@@ -22,11 +22,12 @@ import java.util.List;
 
 import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes;
 import org.jmock.Mockery;
+import org.jmock.lib.concurrent.Synchroniser;
+import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.query.Aggregator;
 import org.apache.geode.cache.query.internal.aggregate.Avg;
 import org.apache.geode.cache.query.internal.aggregate.AvgBucketNode;
@@ -42,21 +43,27 @@ import org.apache.geode.cache.query.internal.aggregate.MaxMin;
 import org.apache.geode.cache.query.internal.aggregate.Sum;
 import org.apache.geode.cache.query.internal.aggregate.SumDistinct;
 import org.apache.geode.cache.query.internal.aggregate.SumDistinctPRQueryNode;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class CompiledAggregateFunctionJUnitTest {
 
   private Mockery context;
-  private Cache cache;
+  private InternalCache cache;
   private List bucketList;
 
   @Before
   public void setUp() throws Exception {
-    context = new Mockery();
-    cache = context.mock(Cache.class);
+    context = new Mockery() {
+      {
+        setImposteriser(ClassImposteriser.INSTANCE);
+        setThreadingPolicy(new Synchroniser());
+      }
+    };
+    cache = context.mock(InternalCache.class);
     bucketList = new ArrayList();
-    bucketList.add(Integer.valueOf(1));
+    bucketList.add(1);
   }
 
   @Test
@@ -172,13 +179,13 @@ public class CompiledAggregateFunctionJUnitTest {
     Class maxMinClass = MaxMin.class;
     Field findMax = maxMinClass.getDeclaredField("findMax");
     findMax.setAccessible(true);
-    assertTrue(((Boolean) findMax.get(maxMin)).booleanValue());
+    assertTrue((Boolean) findMax.get(maxMin));
 
     CompiledAggregateFunction caf2 = new CompiledAggregateFunction(null, OQLLexerTokenTypes.MIN);
     ExecutionContext context2 = new ExecutionContext(null, cache);
     Aggregator agg1 = (Aggregator) caf2.evaluate(context1);
     assertTrue(agg1 instanceof MaxMin);
     MaxMin maxMin1 = (MaxMin) agg1;
-    assertFalse(((Boolean) findMax.get(maxMin1)).booleanValue());
+    assertFalse((Boolean) findMax.get(maxMin1));
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledJunctionInternalsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledJunctionInternalsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledJunctionInternalsJUnitTest.java
index 21745d5..504d1fe 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledJunctionInternalsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/CompiledJunctionInternalsJUnitTest.java
@@ -84,7 +84,7 @@ public class CompiledJunctionInternalsJUnitTest {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolio p, p.positions");
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     try {
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -144,7 +144,7 @@ public class CompiledJunctionInternalsJUnitTest {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolio p, p.positions");
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     try {
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -242,7 +242,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       // qs.createIndex("statusIndex",
       // IndexType.FUNCTIONAL,"status","/portfolio");
@@ -331,7 +331,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio p");
       Iterator iter = list.iterator();
       while (iter.hasNext()) {
@@ -395,7 +395,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       // qs.createIndex("statusIndex",
       // IndexType.FUNCTIONAL,"status","/portfolio");
@@ -466,7 +466,7 @@ public class CompiledJunctionInternalsJUnitTest {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolio p, p.positions");
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     try {
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -512,7 +512,7 @@ public class CompiledJunctionInternalsJUnitTest {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolio p, p.positions");
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     try {
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -580,7 +580,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -636,7 +636,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       // qs.createIndex("statusIndex",
       // IndexType.FUNCTIONAL,"status","/portfolio");
@@ -705,7 +705,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -774,7 +774,7 @@ public class CompiledJunctionInternalsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions,/employees e");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -844,7 +844,7 @@ public class CompiledJunctionInternalsJUnitTest {
       List list =
           compiler.compileFromClause("/portfolio p, p.positions,/employees e, /portfolio1 p1");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       qs.createIndex("statusIndex", IndexType.FUNCTIONAL, "status", "/portfolio");
       qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
@@ -945,7 +945,7 @@ public class CompiledJunctionInternalsJUnitTest {
       List list = compiler.compileFromClause(
           "/portfolio p, p.positions,/employees e, /employees1 e1, /portfolio p1");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
 
       // qs.createIndex("statusIndex",
       // IndexType.FUNCTIONAL,"status","/portfolio");
@@ -2861,7 +2861,7 @@ public class CompiledJunctionInternalsJUnitTest {
   private void bindIteratorsAndCreateIndex(ExecutionContext context) throws Exception {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolio p, p.positions");
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     qs.createIndex("idIndex", IndexType.FUNCTIONAL, "ID", "/portfolio");
     Iterator iter = list.iterator();
     while (iter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/ExecutionContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/ExecutionContextJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/ExecutionContextJUnitTest.java
index a15bae0..1112ee2 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/ExecutionContextJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/ExecutionContextJUnitTest.java
@@ -72,7 +72,7 @@ public class ExecutionContextJUnitTest {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolio p, p.positions");
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     try {
       Iterator iter = list.iterator();
       while (iter.hasNext()) {
@@ -100,7 +100,7 @@ public class ExecutionContextJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
       Iterator iter = list.iterator();
       int i = 0;
       while (iter.hasNext()) {
@@ -128,7 +128,7 @@ public class ExecutionContextJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions");
       ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
       Iterator iter = list.iterator();
       int i = 0;
       CompiledIteratorDef iterDef = null;
@@ -143,7 +143,7 @@ public class ExecutionContextJUnitTest {
             rIter.getIndexInternalID().equals("index_iter" + i));
       }
       Set temp = new HashSet();
-      context.computeUtlimateDependencies(iterDef, temp);
+      context.computeUltimateDependencies(iterDef, temp);
       String regionPath = context
           .getRegionPathForIndependentRuntimeIterator((RuntimeIterator) temp.iterator().next());
       if (!(regionPath != null && regionPath.equals("/portfolio"))) {
@@ -166,7 +166,7 @@ public class ExecutionContextJUnitTest {
         "/portfolio p, p.positions, p.addreses addrs, addrs.collection1 coll1, /dummy d1, d1.collection2 d2");
     RuntimeIterator indItr = null;
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
     int i = 0;
     List checkList = new ArrayList();
     try {

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/IndexManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/IndexManagerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/IndexManagerJUnitTest.java
index 3f40c97..d5b825a 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/IndexManagerJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/IndexManagerJUnitTest.java
@@ -12,8 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-/**
- */
 package org.apache.geode.cache.query.internal;
 
 import static org.junit.Assert.assertFalse;
@@ -42,18 +40,16 @@ import org.apache.geode.test.junit.categories.IntegrationTest;
 public class IndexManagerJUnitTest {
 
   @Before
-  public void setUp() throws java.lang.Exception {
+  public void setUp() throws Exception {
     CacheUtils.startCache();
     Region region = CacheUtils.createRegion("portfolios", Portfolio.class);
     for (int i = 0; i < 4; i++) {
       region.put("" + i, new Portfolio(i));
-      // CacheUtils.log(new Portfolio(i));
     }
-
   }
 
   @After
-  public void tearDown() throws java.lang.Exception {
+  public void tearDown() throws Exception {
     CacheUtils.closeCache();
   }
 
@@ -66,7 +62,7 @@ public class IndexManagerJUnitTest {
    * start time, we can assume that it needs to be reevaluated
    */
   @Test
-  public void testSafeQueryTime() {
+  public void testSafeQueryTime() throws Exception {
     IndexManager.resetIndexBufferTime();
     // fake entry update at LMT of 0 and actual time of 10
     // safe query time set in index manager is going to be 20
@@ -103,7 +99,7 @@ public class IndexManagerJUnitTest {
 
   // Let's test for negative delta's or a system that is slower than others in the cluster
   @Test
-  public void testSafeQueryTimeForASlowNode() {
+  public void testSafeQueryTimeForASlowNode() throws Exception {
     IndexManager.resetIndexBufferTime();
     // fake entry update at LMT of 0 and actual time of 10
     // safe query time set in index manager is going to be -10
@@ -130,7 +126,6 @@ public class IndexManagerJUnitTest {
     assertFalse(IndexManager.needsRecalculation(212, 210));
   }
 
-
   @Test
   public void testBestIndexPick() throws Exception {
     QueryService qs;
@@ -140,7 +135,7 @@ public class IndexManagerJUnitTest {
     QCompiler compiler = new QCompiler();
     List list = compiler.compileFromClause("/portfolios pf");
     ExecutionContext context = new QueryExecutionContext(null, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
 
     Iterator iter = list.iterator();
     while (iter.hasNext()) {
@@ -171,7 +166,6 @@ public class IndexManagerJUnitTest {
     Assert.assertEquals(id2.getMapping()[0], 1);
     Assert.assertEquals(id2.getMapping()[1], 2);
     Assert.assertEquals(id2.getMapping()[2], 0);
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
index e625afe..c18e8cb 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
@@ -162,7 +162,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     List list = compiler.compileFromClause("/pos pf");
     ExecutionContext context =
         new ExecutionContext(new Object[] {"bindkey"}, CacheUtils.getCache());
-    context.newScope(context.assosciateScopeID());
+    context.newScope(context.associateScopeID());
 
     Iterator iter = list.iterator();
     while (iter.hasNext()) {
@@ -177,7 +177,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     List args = new ArrayList();
     args.add(cl);
     CompiledOperation cop = new CompiledOperation(cp, "get", args);
-    StringBuffer sbuff = new StringBuffer();
+    StringBuilder sbuff = new StringBuilder();
     cop.generateCanonicalizedExpression(sbuff, context);
     assertEquals(sbuff.toString(), "iter1.positions.get('key1')");
 
@@ -187,7 +187,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     // args.add(cb);
     // cop = new CompiledOperation(cp, "get", args);
     //// context.setBindArguments(new Object[]{"bindkey"});
-    // sbuff = new StringBuffer();
+    // sbuff = new StringBuilder();
     // cop.generateCanonicalizedExpression(sbuff, context);
     // assertIndexDetailsEquals(sbuff.toString(),"iter1.positions.get('bindkey')");
     //
@@ -197,7 +197,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     // args = new ArrayList();
     // args.add(cb);
     // cop = new CompiledOperation(cp, "get", args);
-    // sbuff = new StringBuffer();
+    // sbuff = new StringBuilder();
     // cop.generateCanonicalizedExpression(sbuff, context);
     // assertIndexDetailsEquals(sbuff.toString(),"iter1.positions().get('bindkey')");
     //
@@ -207,7 +207,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     // args = new ArrayList();
     // args.add(cb);
     // cop = new CompiledOperation(cp, "get", args);
-    // sbuff = new StringBuffer();
+    // sbuff = new StringBuilder();
     // cop.generateCanonicalizedExpression(sbuff, context);
     // assertIndexDetailsEquals(sbuff.toString(),"iter1.positions.get('bindkey')");
 
@@ -216,7 +216,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     args = new ArrayList();
     args.add(cp1);
     cop = new CompiledOperation(cp, "get", args);
-    sbuff = new StringBuffer();
+    sbuff = new StringBuilder();
     cop.generateCanonicalizedExpression(sbuff, context);
     assertEquals(sbuff.toString(), "iter1.positions.get(iter1.pkid)");
 
@@ -224,7 +224,7 @@ public class QueryFromClauseCanonicalizationJUnitTest {
     cp = new CompiledPath(new CompiledID("pf"), "getPositions");
     cp1 = new CompiledPath(new CompiledID("pf"), "pkid");
     CompiledIndexOperation ciop = new CompiledIndexOperation(cp, cp1);
-    sbuff = new StringBuffer();
+    sbuff = new StringBuilder();
     ciop.generateCanonicalizedExpression(sbuff, context);
     assertEquals(sbuff.toString(), "iter1.positions[iter1.pkid]");
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryUtilsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryUtilsJUnitTest.java
index bb13665..bda8338 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryUtilsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/QueryUtilsJUnitTest.java
@@ -91,7 +91,7 @@ public class QueryUtilsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions");
       ExecutionContext context = new ExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
       RuntimeIterator[] indexToItrMappping = new RuntimeIterator[1];
       RuntimeIterator expand = null;
       boolean set = false;
@@ -162,7 +162,7 @@ public class QueryUtilsJUnitTest {
       QCompiler compiler = new QCompiler();
       List list = compiler.compileFromClause("/portfolio p, p.positions, /employees e");
       ExecutionContext context = new ExecutionContext(null, CacheUtils.getCache());
-      context.newScope(context.assosciateScopeID());
+      context.newScope(context.associateScopeID());
       RuntimeIterator[] indexToItrMappping = new RuntimeIterator[1];
       RuntimeIterator expand[] = new RuntimeIterator[2];
       boolean set = false;

http://git-wip-us.apache.org/repos/asf/geode/blob/153db276/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/RangeIndexAPIJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/RangeIndexAPIJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/RangeIndexAPIJUnitTest.java
index 302a633..67ab9a4 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/RangeIndexAPIJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/RangeIndexAPIJUnitTest.java
@@ -12,9 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-/**
- * 
- */
 package org.apache.geode.cache.query.internal.index;
 
 import static org.junit.Assert.assertEquals;
@@ -52,15 +49,13 @@ import org.apache.geode.cache.query.internal.RuntimeIterator;
 import org.apache.geode.pdx.PdxInstance;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
-/**
- *
- */
 @Category(IntegrationTest.class)
 public class RangeIndexAPIJUnitTest {
+
   private Region region = null;
 
   @Before
-  public void setUp() throws java.lang.Exception {
+  public void setUp() throws Exception {
     CacheUtils.startCache();
     IndexManager.ENABLE_UPDATE_IN_PROGRESS_INDEX_CALCULATION = false;
     region = CacheUtils.createRegion("portfolios", Portfolio.class);
@@ -77,7 +72,7 @@ public class RangeIndexAPIJUnitTest {
   }
 
   @After
-  public void tearDown() throws java.lang.Exception {
+  public void tearDown() throws Exception {
     IndexManager.ENABLE_UPDATE_IN_PROGRESS_INDEX_CALCULATION = true;
     CacheUtils.closeCache();
   }
@@ -132,8 +127,6 @@ public class RangeIndexAPIJUnitTest {
    * of RangeJunctionCondnEvaluator. Such null or undefined conditions are treated as separate
    * filter operands. This test checks the query method of Index which takes a set of keys which
    * need to be removed from the set
-   * 
-   * @throws Exception
    */
   @Test
   public void testQueryMethod_2() throws Exception {