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/04/27 01:17:13 UTC

[01/16] incubator-geode git commit: Updating and fixing tests

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1276 [created] 74c625edb


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index 0d257e3..28ff543 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -27,12 +27,11 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 
 /* wrapper of IndexWriter */
@@ -88,7 +87,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       }
 
       // we will create RegionDirectories on the fly when data comes in
-      HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(getFieldNames());
+      HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(getFieldNames());
       repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, mapper, analyzer);
       
       // create AEQ, AEQ listener and specify the listener to repositoryManager

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
new file mode 100644
index 0000000..a0319f4
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializer.java
@@ -0,0 +1,82 @@
+/*
+ * 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.lucene.internal.repository.serializer;
+
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+
+import com.gemstone.gemfire.internal.util.concurrent.CopyOnWriteWeakHashMap;
+import com.gemstone.gemfire.pdx.PdxInstance;
+
+/**
+ * An implementation of LuceneSerializer that reads the fields
+ * of a given object using reflection or from a PDX instance and
+ * writes them to a lucene document.
+ */
+public class HeterogeneousLuceneSerializer implements LuceneSerializer {
+  /**
+   * The set of indexed fields for this mapper
+   */
+  private String[] indexedFields;
+  
+  /**
+   * A mapper for converting a PDX object into a document
+   */
+  private LuceneSerializer pdxMapper;
+  
+  /**
+   * Mappers for each individual class type that this class has seen.
+   * 
+   * Weak so that entry will be removed if a class is garbage collected.
+   */
+  private Map<Class<?>, LuceneSerializer> mappers = new CopyOnWriteWeakHashMap<Class<?>, LuceneSerializer>();
+  
+  public HeterogeneousLuceneSerializer(String[] indexedFields) {
+    this.indexedFields = indexedFields;
+    pdxMapper = new PdxLuceneSerializer(indexedFields);
+  }
+  
+  @Override
+  public void toDocument(Object value, Document doc) {
+    
+    LuceneSerializer mapper = getFieldMapper(value);
+    
+    mapper.toDocument(value, doc);
+  }
+
+  /**
+   * Get the field mapper based on the type of the given object.
+   */
+  private LuceneSerializer getFieldMapper(Object value) {
+    if(value instanceof PdxInstance) {
+      return pdxMapper;
+    } else {
+      Class<?> clazz = value.getClass();
+      LuceneSerializer mapper = mappers.get(clazz);
+      if(mapper == null) {
+        mapper = new ReflectionLuceneSerializer(clazz, indexedFields);
+        mappers.put(clazz, mapper);
+      }
+      return mapper;
+    }
+  }
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializer.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializer.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializer.java
deleted file mode 100644
index 7cb25bb..0000000
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializer.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.lucene.internal.repository.serializer;
-
-import java.util.Map;
-
-import org.apache.lucene.document.Document;
-
-import com.gemstone.gemfire.internal.util.concurrent.CopyOnWriteWeakHashMap;
-import com.gemstone.gemfire.pdx.PdxInstance;
-
-/**
- * An implementation of LuceneSerializer that reads the fields
- * of a given object using reflection or from a PDX instance and
- * writes them to a lucene document.
- */
-public class HeterogenousLuceneSerializer implements LuceneSerializer {
-  /**
-   * The set of indexed fiels for this mapper
-   */
-  private String[] indexedFields;
-  
-  /**
-   * A mapper for converting a PDX object into a document
-   */
-  private LuceneSerializer pdxMapper;
-  
-  /**
-   * Mappers for each individual class type that this class has seen.
-   * 
-   * Weak so that entry will be removed if a class is garbage collected.
-   */
-  private Map<Class<?>, LuceneSerializer> mappers = new CopyOnWriteWeakHashMap<Class<?>, LuceneSerializer>();
-  
-  public HeterogenousLuceneSerializer(String[] indexedFields) {
-    this.indexedFields = indexedFields;
-    pdxMapper = new PdxLuceneSerializer(indexedFields);
-  }
-  
-  @Override
-  public void toDocument(Object value, Document doc) {
-    
-    LuceneSerializer mapper = getFieldMapper(value);
-    
-    mapper.toDocument(value, doc);
-  }
-
-  /**
-   * Get the field mapper based on the type of the given object.
-   */
-  private LuceneSerializer getFieldMapper(Object value) {
-    if(value instanceof PdxInstance) {
-      return pdxMapper;
-    } else {
-      Class<?> clazz = value.getClass();
-      LuceneSerializer mapper = mappers.get(clazz);
-      if(mapper == null) {
-        mapper = new ReflectionLuceneSerializer(clazz, indexedFields);
-        mappers.put(clazz, mapper);
-      }
-      return mapper;
-    }
-  }
-  
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
index 85a5333..b0e3cc0 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.mockito.Matchers.any;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
index 2aa3241..ebde9d4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
@@ -21,7 +21,6 @@ package com.gemstone.gemfire.cache.lucene.internal;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -48,7 +47,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.Type1;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.EvictionAttributesImpl;
@@ -64,7 +63,7 @@ public class LuceneIndexRecoveryHAJUnitTest {
   private static final String INDEX = "index";
   private static final String REGION = "indexedRegion";
   String[] indexedFields = new String[] { "txt" };
-  HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
+  HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(indexedFields);
   Analyzer analyzer = new StandardAnalyzer();
 
   Cache cache;
@@ -72,7 +71,7 @@ public class LuceneIndexRecoveryHAJUnitTest {
   @Before
   public void setup() {
     indexedFields = new String[] { "txt" };
-    mapper = new HeterogenousLuceneSerializer(indexedFields);
+    mapper = new HeterogeneousLuceneSerializer(indexedFields);
     analyzer = new StandardAnalyzer();
     LuceneServiceImpl.registerDataSerializables();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
index 91556a9..0398de1 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
@@ -76,7 +75,6 @@ public class LuceneQueryResultsImplJUnitTest {
   
   @Test
   public void testMaxStore() {
-
     hits.set(5, new EntryScore("key_5", 502));
     
     LuceneQueryResultsImpl<String, String> results = new LuceneQueryResultsImpl<String, String>(hits, null, 5);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneResultStructImpJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneResultStructImpJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneResultStructImpJUnitTest.java
index 7893940..a41c7da 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneResultStructImpJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneResultStructImpJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
@@ -31,7 +30,6 @@ public class LuceneResultStructImpJUnitTest {
 
   @Test
   public void hashCodeAndEquals() {
-    
     //Create 2 equal structs
     LuceneResultStructImpl<String, String> result1 = new LuceneResultStructImpl<String, String>("key1", "value1", 5);
     LuceneResultStructImpl<String, String> result2 = new LuceneResultStructImpl<String, String>("key1", "value1", 5);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index 1beea0f..dec4572 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
@@ -36,7 +35,6 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.After;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -44,32 +42,30 @@ import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class LuceneServiceImplJUnitTest {
 
   Cache cache;
   ClientCache clientCache;
   private LuceneIndexImpl repo;
-  private HeterogenousLuceneSerializer mapper;
+  private HeterogeneousLuceneSerializer mapper;
   private StandardAnalyzer analyzer = new StandardAnalyzer();
   private IndexWriter writer;
   LuceneServiceImpl service = null;
@@ -242,7 +238,7 @@ public class LuceneServiceImplJUnitTest {
     }
   }
 
-  public void canCreateLuceneIndexForPRType(RegionShortcut regionShortcut) {
+  private void canCreateLuceneIndexForPRType(RegionShortcut regionShortcut) {
     getService();
     service.createIndex("index1", "PR1", "field1", "field2", "field3");
     Region region = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
index bf09f99..d6abca0 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.assertEquals;
@@ -43,11 +42,10 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepositoryImpl;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.LuceneSerializer;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
@@ -60,7 +58,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class PartitionedRepositoryManagerJUnitTest {
 
-  
   private PartitionedRegion userRegion;
   private PartitionedRegion fileRegion;
   private PartitionedRegion chunkRegion;
@@ -84,7 +81,7 @@ public class PartitionedRepositoryManagerJUnitTest {
     chunkRegion = Mockito.mock(PartitionedRegion.class);
     chunkDataStore = Mockito.mock(PartitionedRegionDataStore.class);
     Mockito.when(chunkRegion.getDataStore()).thenReturn(chunkDataStore);
-    serializer = new HeterogenousLuceneSerializer(new String[] {"a", "b"} );  
+    serializer = new HeterogeneousLuceneSerializer(new String[] {"a", "b"} );
   }
   
   @Test
@@ -163,7 +160,6 @@ public class PartitionedRepositoryManagerJUnitTest {
   
   @Test
   public void getByRegion() throws BucketNotFoundException {
-
     PartitionedRepositoryManager repoManager = new PartitionedRepositoryManager(userRegion, fileRegion, chunkRegion, serializer, new StandardAnalyzer());
     
     setUpMockBucket(0);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
index a16b019..a223c6e 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.assertEquals;
@@ -36,7 +35,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class StringQueryProviderJUnitTest {
-  static final String INDEXED_REGION = "indexedRegion";
 
   LuceneIndex mockIndex;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
index d20b052..92a822b 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.directory;
 
 import java.io.IOException;
@@ -42,6 +41,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  */
 @Category(UnitTest.class)
 public class RegionDirectoryJUnitTest extends BaseDirectoryTestCase {
+
   @Rule
   public SystemPropertiesRestoreRule restoreProps = new SystemPropertiesRestoreRule();
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
index bec2da8..c8b1747 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import java.io.IOException;
@@ -41,21 +40,14 @@ import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepositoryImpl;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class DistributedScoringJUnitTest {
-  class TestType {
-    String txt;
-
-    public TestType(String txt) {
-      this.txt = txt;
-    }
-  }
 
   String[] indexedFields = new String[] { "txt" };
-  HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
+  HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(indexedFields);
 
   final StandardAnalyzer analyzer = new StandardAnalyzer();
   Region<String, String> region;
@@ -152,4 +144,13 @@ public class DistributedScoringJUnitTest {
 
     return new IndexRepositoryImpl(region, writer, mapper);
   }
+
+  private static class TestType {
+
+    String txt;
+
+    public TestType(String txt) {
+      this.txt = txt;
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
index abdf8ec..93d09c8 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import org.junit.Assert;
@@ -29,6 +28,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class EntryScoreJUnitTest {
+
   @Test
   public void testSerialization() {
     LuceneServiceImpl.registerDataSerializables();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContextJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContextJUnitTest.java
index 46ea67f..eea8f3c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContextJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionContextJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.assertEquals;
@@ -38,6 +37,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class LuceneFunctionContextJUnitTest {
+
   @Test
   public void testLuceneFunctionArgsDefaults() {
     LuceneFunctionContext<IndexResultCollector> context = new LuceneFunctionContext<>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
index 750ec0f..0e0d28a 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.assertEquals;
@@ -57,6 +56,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class LuceneFunctionJUnitTest {
+
   Mockery mocker;
 
   String regionPath = "/region";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
index b36d8cc..e634945 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.assertEquals;
@@ -36,6 +35,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class TopEntriesCollectorJUnitTest {
+
   EntryScore r1_1 = new EntryScore("1-1", .9f);
   EntryScore r1_2 = new EntryScore("1-2", .7f);
   EntryScore r1_3 = new EntryScore("1-3", .5f);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
index 4f93587..bbfb735 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.assertEquals;
@@ -44,6 +43,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class TopEntriesFunctionCollectorJUnitTest {
+
   EntryScore r1_1, r1_2, r2_1, r2_2;
   TopEntriesCollector result1, result2;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
index d849c8e..923553c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.assertEquals;
@@ -40,6 +39,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class TopEntriesJUnitTest {
+
   Mockery mockContext;
 
   EntryScore r1_1 = new EntryScore("3", .9f);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/ChunkKeyJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/ChunkKeyJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/ChunkKeyJUnitTest.java
index f28b84a..b49beb5 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/ChunkKeyJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/ChunkKeyJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.filesystem;
 
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileJUnitTest.java
index e4e8752..b4ba92f 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileJUnitTest.java
@@ -16,13 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.filesystem;
 
 import static org.junit.Assert.*;
 
-import java.util.UUID;
-
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -41,7 +38,7 @@ public class FileJUnitTest {
     file.length = 5;
     file.chunks = 7;
     File copy = CopyHelper.deepCopy(file);
-    
+
     assertEquals(file.chunks, copy.chunks);
     assertEquals(file.created, copy.created);
     assertEquals(file.modified, copy.modified);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
index 83d9e03..31dca00 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.filesystem;
 
 import static org.junit.Assert.assertArrayEquals;
@@ -51,6 +50,7 @@ public class FileSystemJUnitTest {
 
   private static final int SMALL_CHUNK = 523;
   private static final int LARGE_CHUNK = 1024 * 1024 * 5 + 33;
+
   private FileSystem system;
   private Random rand = new Random();
   private ConcurrentHashMap<String, File> fileRegion;
@@ -466,7 +466,7 @@ public class FileSystemJUnitTest {
       //File was deleted, but shouldn't have any dangling chunks at this point
       assertEquals(Collections.EMPTY_SET, fileRegion.keySet());
       //TODO - need to purge chunks of deleted files somehow.
-//      assertEquals(Collections.EMPTY_SET, chunkRegion.keySet());
+//      assertIndexDetailsEquals(Collections.EMPTY_SET, chunkRegion.keySet());
     } else {
       file2 = system.getFile(name2);
       assertContents(expected.toByteArray(), file2);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
index 617879f..53c4161 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
@@ -43,7 +43,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.Type2;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -56,7 +56,7 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 public class IndexRepositoryImplJUnitTest {
 
   private IndexRepositoryImpl repo;
-  private HeterogenousLuceneSerializer mapper;
+  private HeterogeneousLuceneSerializer mapper;
   private StandardAnalyzer analyzer = new StandardAnalyzer();
   private IndexWriter writer;
   private Region region;
@@ -69,7 +69,7 @@ public class IndexRepositoryImplJUnitTest {
     IndexWriterConfig config = new IndexWriterConfig(analyzer);
     writer = new IndexWriter(dir, config);
     String[] indexedFields= new String[] {"s", "i", "l", "d", "f", "s2", "missing"};
-    mapper = new HeterogenousLuceneSerializer(indexedFields);
+    mapper = new HeterogeneousLuceneSerializer(indexedFields);
     region = Mockito.mock(Region.class);
     Mockito.when(region.isDestroyed()).thenReturn(false);
     repo = new IndexRepositoryImpl(region, writer, mapper);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index e996f30..cec76ba 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -65,7 +65,7 @@ import com.gemstone.gemfire.cache.lucene.internal.directory.RegionDirectory;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
+import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
 import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 
@@ -119,7 +119,7 @@ public class IndexRepositoryImplPerformanceTest {
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         writer = new IndexWriter(dir, config);
         String[] indexedFields= new String[] {"text"};
-        HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(indexedFields);
+        HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(indexedFields);
         repo = new IndexRepositoryImpl(fileRegion, writer, mapper);
       }
 
@@ -381,7 +381,7 @@ public class IndexRepositoryImplPerformanceTest {
         int size  = callbacks.query(query);
 //        int size  = callbacks.query(parser.parse(word));
         //All of my tests sometimes seem to be missing a couple of words, including the stock lucene
-//        assertEquals("Error on query " + i + " word=" + word, counts[wordIndex], size);
+//        assertIndexDetailsEquals("Error on query " + i + " word=" + word, counts[wordIndex], size);
       }
       end = System.nanoTime();
       results.queryTime = end - start;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializerJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializerJUnitTest.java
new file mode 100644
index 0000000..0cafc57
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogeneousLuceneSerializerJUnitTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.lucene.internal.repository.serializer;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.lucene.document.Document;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.gemstone.gemfire.pdx.PdxInstance;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit test of the ObjectToDocumentMapper. 
+ */
+@Category(UnitTest.class)
+public class HeterogeneousLuceneSerializerJUnitTest {
+
+  /**
+   * Test that the mapper can handle a mix of different
+   * object types.
+   */
+  @Test
+  public void testHeterogeneousObjects() {
+    String[] fields = new String[] {"s", "i", "l", "d", "f", "s2", "missing"};
+    HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(fields);
+    
+    Type1 t1 = new Type1("a", 1, 2L, 3.0, 4.0f);
+    
+    Document doc1 = new Document();
+    mapper.toDocument(t1, doc1);
+    
+    assertEquals(5, doc1.getFields().size());
+    assertEquals("a", doc1.getField("s").stringValue());
+    assertEquals(1, doc1.getField("i").numericValue());
+    assertEquals(2L, doc1.getField("l").numericValue());
+    assertEquals(3.0, doc1.getField("d").numericValue());
+    assertEquals(4.0f, doc1.getField("f").numericValue());
+    
+    Type2 t2 = new Type2("a", 1, 2L, 3.0, 4.0f, "b");
+    
+    Document doc2 = new Document();
+    mapper.toDocument(t2, doc2);
+    
+    assertEquals(6, doc2.getFields().size());
+    assertEquals("a", doc2.getField("s").stringValue());
+    assertEquals("b", doc2.getField("s2").stringValue());
+    assertEquals(1, doc2.getField("i").numericValue());
+    assertEquals(2L, doc2.getField("l").numericValue());
+    assertEquals(3.0, doc2.getField("d").numericValue());
+    assertEquals(4.0f, doc2.getField("f").numericValue());
+    
+    PdxInstance i = Mockito.mock(PdxInstance.class);
+    
+    Mockito.when(i.hasField("s")).thenReturn(true);
+    Mockito.when(i.hasField("i")).thenReturn(true);
+    Mockito.when(i.getField("s")).thenReturn("a");
+    Mockito.when(i.getField("i")).thenReturn(5);
+    
+    Document doc3 = new Document();
+    mapper.toDocument(i, doc3);
+    
+    assertEquals(2, doc3.getFields().size());
+    assertEquals("a", doc3.getField("s").stringValue());
+    assertEquals(5, doc3.getField("i").numericValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializerJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializerJUnitTest.java
deleted file mode 100644
index e54856a..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/HeterogenousLuceneSerializerJUnitTest.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.lucene.internal.repository.serializer;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.Term;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogenousLuceneSerializer;
-import com.gemstone.gemfire.pdx.PdxInstance;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- * Unit test of the ObjectToDocumentMapper. 
- */
-@Category(UnitTest.class)
-public class HeterogenousLuceneSerializerJUnitTest {
-
-  /**
-   * Test that the mapper can handle a mix of different
-   * object types.
-   */
-  @Test
-  public void testHetrogenousObjects() {
-    String[] fields = new String[] {"s", "i", "l", "d", "f", "s2", "missing"};
-    HeterogenousLuceneSerializer mapper = new HeterogenousLuceneSerializer(fields);
-    
-    Type1 t1 = new Type1("a", 1, 2L, 3.0, 4.0f);
-    
-    Document doc1 = new Document();
-    mapper.toDocument(t1, doc1);
-    
-    assertEquals(5, doc1.getFields().size());
-    assertEquals("a", doc1.getField("s").stringValue());
-    assertEquals(1, doc1.getField("i").numericValue());
-    assertEquals(2L, doc1.getField("l").numericValue());
-    assertEquals(3.0, doc1.getField("d").numericValue());
-    assertEquals(4.0f, doc1.getField("f").numericValue());
-    
-    Type2 t2 = new Type2("a", 1, 2L, 3.0, 4.0f, "b");
-    
-    Document doc2 = new Document();
-    mapper.toDocument(t2, doc2);
-    
-    assertEquals(6, doc2.getFields().size());
-    assertEquals("a", doc2.getField("s").stringValue());
-    assertEquals("b", doc2.getField("s2").stringValue());
-    assertEquals(1, doc2.getField("i").numericValue());
-    assertEquals(2L, doc2.getField("l").numericValue());
-    assertEquals(3.0, doc2.getField("d").numericValue());
-    assertEquals(4.0f, doc2.getField("f").numericValue());
-    
-    PdxInstance i = Mockito.mock(PdxInstance.class);
-    
-    Mockito.when(i.hasField("s")).thenReturn(true);
-    Mockito.when(i.hasField("i")).thenReturn(true);
-    Mockito.when(i.getField("s")).thenReturn("a");
-    Mockito.when(i.getField("i")).thenReturn(5);
-    
-    Document doc3 = new Document();
-    mapper.toDocument(i, doc3);
-    
-    assertEquals(2, doc3.getFields().size());
-    assertEquals("a", doc3.getField("s").stringValue());
-    assertEquals(5, doc3.getField("i").numericValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxFieldMapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxFieldMapperJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxFieldMapperJUnitTest.java
index ec31da9..fab1bc3 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxFieldMapperJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/PdxFieldMapperJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.repository.serializer;
 
 import static org.junit.Assert.assertEquals;
@@ -26,7 +25,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.PdxLuceneSerializer;
 import com.gemstone.gemfire.pdx.PdxInstance;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionFieldMapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionFieldMapperJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionFieldMapperJUnitTest.java
index 3ca8fbf..273b57f 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionFieldMapperJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/ReflectionFieldMapperJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.repository.serializer;
 
 import static org.junit.Assert.assertEquals;
@@ -25,7 +24,6 @@ import org.apache.lucene.document.Document;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.ReflectionLuceneSerializer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorJUnitTest.java
index c1e93ad..021dc3d 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlGeneratorJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.xml;
 
 import static org.junit.Assert.*;
@@ -43,7 +42,6 @@ public class LuceneIndexXmlGeneratorJUnitTest {
   
   /**
    * Test of generating and reading cache configuration back in.
-   * @throws SAXException 
    */
   @Test
   public void generateWithFields() throws SAXException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserJUnitTest.java
index 298c92f..64fe22e 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserJUnitTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package com.gemstone.gemfire.cache.lucene.internal.xml;
 
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-pulse/build.gradle
----------------------------------------------------------------------
diff --git a/geode-pulse/build.gradle b/geode-pulse/build.gradle
index 862bacb..fe5a132 100755
--- a/geode-pulse/build.gradle
+++ b/geode-pulse/build.gradle
@@ -26,10 +26,6 @@ sourceSets {
 }
 
 dependencies {
-
-
-
-
   compile 'org.springframework:spring-beans:' + project.'springframework.version'
   
   compile 'commons-beanutils:commons-beanutils:' + project.'commons-beanutils.version'
@@ -71,9 +67,13 @@ dependencies {
   testCompile project(':geode-junit')
   testCompile project(':geode-core')
 
-  testCompile 'org.seleniumhq.selenium:selenium-firefox-driver:' + project.'selenium.version'
+  testCompile ('org.seleniumhq.selenium:selenium-firefox-driver:' + project.'selenium.version') {
+    exclude module: 'cglib-nodep'
+  }
   testCompile 'org.seleniumhq.selenium:selenium-api:' + project.'selenium.version'
-  testCompile 'org.seleniumhq.selenium:selenium-remote-driver:' + project.'selenium.version'
+  testCompile ('org.seleniumhq.selenium:selenium-remote-driver:' + project.'selenium.version') {
+    exclude module: 'cglib-nodep'
+  }
   testCompile 'org.seleniumhq.selenium:selenium-support:' + project.'selenium.version'
   testCompile 'org.springframework:spring-test:' + project.'springframework.version'
   testCompile 'org.powermock:powermock-core:' + project.'powermock.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/controllers/PulseControllerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/controllers/PulseControllerJUnitTest.java b/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/controllers/PulseControllerJUnitTest.java
index 38bf9c4..983ab47 100644
--- a/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/controllers/PulseControllerJUnitTest.java
+++ b/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/controllers/PulseControllerJUnitTest.java
@@ -16,8 +16,25 @@
  */
 package com.vmware.gemfire.tools.pulse.controllers;
 
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.*;
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.powermock.api.mockito.PowerMockito.spy;
+import static org.powermock.api.mockito.PowerMockito.when;
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.*;
+import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.*;
+
+import java.io.File;
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.UUID;
+import javax.servlet.ServletContextListener;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import com.vmware.gemfire.tools.pulse.internal.PulseAppListener;
 import com.vmware.gemfire.tools.pulse.internal.controllers.PulseController;
 import com.vmware.gemfire.tools.pulse.internal.data.Cluster;
@@ -45,29 +62,9 @@ import org.springframework.test.web.servlet.MockMvc;
 import org.springframework.test.web.servlet.setup.MockMvcBuilders;
 import org.springframework.web.context.WebApplicationContext;
 
-import javax.servlet.ServletContextListener;
-import java.io.File;
-import java.security.Principal;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.UUID;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.powermock.api.mockito.PowerMockito.spy;
-import static org.powermock.api.mockito.PowerMockito.when;
-import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
-import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post;
-import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.*;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-/**
- */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 @PrepareForTest(Repository.class)
 @RunWith(PowerMockRunner.class)
 @PowerMockRunnerDelegate(SpringJUnit4ClassRunner.class)
@@ -76,16 +73,6 @@ import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.
 @PowerMockIgnore("*.UnitTest")
 public class PulseControllerJUnitTest {
 
-  @Autowired
-  private WebApplicationContext wac;
-
-  private MockMvc mockMvc;
-
-  private Cluster cluster;
-
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
-
   private static final String PRINCIPAL_USER = "test-user";
 
   private static final String MEMBER_ID = "member1";
@@ -107,6 +94,16 @@ public class PulseControllerJUnitTest {
 
   private final ObjectMapper mapper = new ObjectMapper();
 
+  @Autowired
+  private WebApplicationContext wac;
+
+  private MockMvc mockMvc;
+
+  private Cluster cluster;
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
   @Before
   public void setup() throws Exception {
     this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).build();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/testbed/driver/PulseUITest.java
----------------------------------------------------------------------
diff --git a/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/testbed/driver/PulseUITest.java b/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/testbed/driver/PulseUITest.java
index 221be7f..4f33716 100644
--- a/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/testbed/driver/PulseUITest.java
+++ b/geode-pulse/src/test/java/com/vmware/gemfire/tools/pulse/testbed/driver/PulseUITest.java
@@ -261,7 +261,7 @@ public class PulseUITest {
       List<Region> locatorRegionscount = testBed.getRootDs().getRegions(locator);
       int locatorRegions = Integer.parseInt(driver.findElement(
           By.id("memberRegionsCount")).getText());
-      Assert.assertEquals(locatorRegionscount.size(), locatorRegions);
+      Assert.assertIndexDetailsEquals(locatorRegionscount.size(), locatorRegions);
       prevSelectedMember = peername;
     }*/
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueueDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueueDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueueDUnitTest.java
index fbd5700..190b381 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueueDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueueDUnitTest.java
@@ -116,7 +116,7 @@ public class SerialGatewaySenderQueueDUnitTest extends WANTestBase{
     secondarySenderUpdates = (HashMap)vm5.invoke(() -> WANTestBase.checkQueue());
     
     checkPrimarySenderUpdatesOnVM5(primarySenderUpdates);
-//    assertEquals(primarySenderUpdates, secondarySenderUpdates);
+//    assertIndexDetailsEquals(primarySenderUpdates, secondarySenderUpdates);
     
     vm4.invoke(() -> WANTestBase.resumeSender( "ln"));
     Wait.pause(5000);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
index 7293963..a020bca 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
@@ -197,7 +197,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
 //      TabularResultData tableResultData =
 //        (TabularResultData) cmdResult.getResultData();
 //      List<String> result_Status = tableResultData.retrieveAllValues(CliStrings.RESULT_STATUS);
-//      assertEquals(1, result_Status.size());
+//      assertIndexDetailsEquals(1, result_Status.size());
 //      assertFalse(result_Status.contains(CliStrings.GATEWAY_NOT_RUNNING));
       String strCmdResult = commandResultToString(cmdResult);
       getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
@@ -434,7 +434,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       //TabularResultData tableResultData = (TabularResultData) cmdResult.getResultData();
       //List<String> result_Status = tableResultData.retrieveAllValues(CliStrings.RESULT_STATUS);
-      //assertEquals(1, result_Status.size());
+      //assertIndexDetailsEquals(1, result_Status.size());
       //assertFalse(strCmdResult.contains(CliStrings.GATEWAY_NOT_RUNNING));
       TabularResultData tableResultData =
           ((CompositeResultData)cmdResult.getResultData()).retrieveSection(CliStrings.SECTION_GATEWAY_RECEIVER_AVAILABLE).retrieveTable(CliStrings.TABLE_GATEWAY_RECEIVER);
@@ -463,7 +463,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
 //      TabularResultData tableResultData =
 //        (TabularResultData) cmdResult.getResultData();
 //      List<String> result_Status = tableResultData.retrieveAllValues(CliStrings.RESULT_STATUS);
-//      assertEquals(1, result_Status.size());
+//      assertIndexDetailsEquals(1, result_Status.size());
 //      assertFalse(result_Status.contains(CliStrings.GATEWAY_RUNNING));
       
       TabularResultData tableResultData =

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java
index 196be09..26b4185 100644
--- a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java
@@ -25,13 +25,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
-import com.gemstone.gemfire.management.cli.CliMetaData;
-import com.gemstone.gemfire.management.internal.cli.util.ClasspathScanLoadHelper;
-import com.gemstone.gemfire.management.internal.web.domain.Link;
-import com.gemstone.gemfire.management.internal.web.domain.LinkIndex;
-import com.gemstone.gemfire.management.internal.web.util.UriUtils;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import javax.servlet.http.HttpServletRequest;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -45,7 +39,12 @@ import org.springframework.web.context.request.RequestAttributes;
 import org.springframework.web.context.request.RequestContextHolder;
 import org.springframework.web.context.request.ServletRequestAttributes;
 
-import javax.servlet.http.HttpServletRequest;
+import com.gemstone.gemfire.management.cli.CliMetaData;
+import com.gemstone.gemfire.management.internal.cli.util.ClasspathScanLoadHelper;
+import com.gemstone.gemfire.management.internal.web.domain.Link;
+import com.gemstone.gemfire.management.internal.web.domain.LinkIndex;
+import com.gemstone.gemfire.management.internal.web.util.UriUtils;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * The ShellCommandsControllerJUnitTest class is a test suite of test cases testing the contract and functionality of the

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/ClientHttpRequestJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/ClientHttpRequestJUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/ClientHttpRequestJUnitTest.java
index 60165d7..346879e 100644
--- a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/ClientHttpRequestJUnitTest.java
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/ClientHttpRequestJUnitTest.java
@@ -25,10 +25,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import com.gemstone.gemfire.management.internal.web.AbstractWebTestCase;
-import com.gemstone.gemfire.management.internal.web.domain.Link;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
 import org.jmock.Mockery;
 import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.After;
@@ -40,6 +36,10 @@ import org.springframework.http.MediaType;
 import org.springframework.util.LinkedMultiValueMap;
 import org.springframework.util.MultiValueMap;
 
+import com.gemstone.gemfire.management.internal.web.AbstractWebTestCase;
+import com.gemstone.gemfire.management.internal.web.domain.Link;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * The ClientHttpRequestJUnitTest class is a test suite of test cases testing the contract and functionality of the
  * ClientHttpRequest class.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/converter/SerializableObjectHttpMessageConverterJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/converter/SerializableObjectHttpMessageConverterJUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/converter/SerializableObjectHttpMessageConverterJUnitTest.java
index 81f3e31..1b317c2 100644
--- a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/converter/SerializableObjectHttpMessageConverterJUnitTest.java
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/http/converter/SerializableObjectHttpMessageConverterJUnitTest.java
@@ -24,9 +24,6 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.util.Calendar;
 
-import com.gemstone.gemfire.internal.util.IOUtils;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
 import org.jmock.Expectations;
 import org.jmock.Mockery;
 import org.jmock.lib.legacy.ClassImposteriser;
@@ -39,6 +36,9 @@ import org.springframework.http.HttpInputMessage;
 import org.springframework.http.HttpOutputMessage;
 import org.springframework.http.MediaType;
 
+import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * The SerializableObjectHttpMessageConverterJUnitTest class is a test suite of test cases testing the contract
  * and functionality of the SerializableObjectHttpMessageConverter class.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java
index 15f9a29..34e9f17 100644
--- a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java
@@ -24,15 +24,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import com.gemstone.gemfire.internal.lang.StringUtils;
-import com.gemstone.gemfire.management.internal.cli.CommandRequest;
-import com.gemstone.gemfire.management.internal.web.AbstractWebTestCase;
-import com.gemstone.gemfire.management.internal.web.domain.Link;
-import com.gemstone.gemfire.management.internal.web.domain.LinkIndex;
-import com.gemstone.gemfire.management.internal.web.http.ClientHttpRequest;
-import com.gemstone.gemfire.management.internal.web.http.HttpMethod;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,6 +33,15 @@ import org.springframework.http.HttpStatus;
 import org.springframework.http.ResponseEntity;
 import org.springframework.web.client.ResourceAccessException;
 
+import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.management.internal.cli.CommandRequest;
+import com.gemstone.gemfire.management.internal.web.AbstractWebTestCase;
+import com.gemstone.gemfire.management.internal.web.domain.Link;
+import com.gemstone.gemfire.management.internal.web.domain.LinkIndex;
+import com.gemstone.gemfire.management.internal.web.http.ClientHttpRequest;
+import com.gemstone.gemfire.management.internal.web.http.HttpMethod;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * The RestHttpOperationInvokerJUnitTest class is a test suite of test cases testing the contract and functionality of the
  * RestHttpOperationInvoker class.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index f89f281..fe6fd91 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -17,14 +17,14 @@
 activation.version = 1.1.1
 annotations.version = 3.0.0
 antlr.version = 2.7.7
-asm.version = 5.0.3
-assertj-core.version = 3.2.0
+asm.version = 5.1
+assertj-core.version = 3.4.1
 awaitility.version = 1.7.0
 bcel.version = 5.2
 catch-exception.version = 1.4.4
 catch-throwable.version = 1.4.4
 cdi-api.version = 1.2
-cglib.version = 3.1
+cglib.version = 3.2.2
 classmate.version = 0.9.0
 commons-collections.version = 3.2.2
 commons-configuration.version = 1.6
@@ -63,13 +63,13 @@ javax.transaction-api.version = 1.2
 jedis.version = 2.7.2
 jetty.version = 9.3.6.v20151106
 jline.version = 2.12
-jmock.version = 2.8.1
+jmock.version = 2.8.2
 jna.version = 4.0.0
 json-path.version = 1.2.0
 json4s.version = 3.2.4
 jsr305.version = 3.0.1
 junit.version = 4.12
-JUnitParams.version = 1.0.4
+JUnitParams.version = 1.0.5
 log4j.version = 2.5
 lucene.version = 6.0.0
 mockito-core.version = 1.10.19
@@ -95,10 +95,10 @@ stephenc-findbugs.version = 1.3.9-1
 spymemcached.version = 2.9.0
 swagger.version = 1.3.2
 swagger-springmvc.version = 0.8.2
-system-rules.version = 1.15.0
+system-rules.version = 1.16.0
 tempus-fugit.version = 1.1
 tomcat6.version = 6.0.37
 tomcat7.version = 7.0.30
 mortbay-jetty-servlet-api.version=2.5-20081211
-selenium.version=2.52.0
+selenium.version=2.53.0
 google-gson.version=2.3.1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/gradle/test.gradle
----------------------------------------------------------------------
diff --git a/gradle/test.gradle b/gradle/test.gradle
index 2dd7521..8dd4074 100644
--- a/gradle/test.gradle
+++ b/gradle/test.gradle
@@ -50,7 +50,9 @@ subprojects {
       exclude module: 'junit-dep'
     }
     testCompile 'com.google.code.tempus-fugit:tempus-fugit:' + project.'tempus-fugit.version'
-    testCompile 'com.jayway.awaitility:awaitility:' + project.'awaitility.version'
+    testCompile ('com.jayway.awaitility:awaitility:' + project.'awaitility.version') {
+      exclude module: 'cglib-nodep'
+    }
     testCompile 'edu.umd.cs.mtc:multithreadedtc:' + project.'multithreadedtc.version'
     testCompile 'eu.codearte.catch-exception:catch-exception:' + project.'catch-exception.version'
     testCompile 'eu.codearte.catch-exception:catch-throwable:' + project.'catch-throwable.version'
@@ -58,7 +60,10 @@ subprojects {
     testCompile 'org.assertj:assertj-core:' + project.'assertj-core.version'
     testCompile 'org.mockito:mockito-core:' + project.'mockito-core.version'
     testCompile 'org.hamcrest:hamcrest-all:' + project.'hamcrest-all.version'
-    testCompile 'org.jmock:jmock-junit4:' + project.'jmock.version'
+    testCompile ('org.jmock:jmock-junit4:' + project.'jmock.version') {
+      exclude module: 'asm'
+      exclude module: 'cglib'
+    }
     testCompile 'org.jmock:jmock-legacy:' + project.'jmock.version'
     testCompile 'pl.pragmatists:JUnitParams:' + project.'JUnitParams.version'
     


[16/16] incubator-geode git commit: Extracting IntegrationTests from UnitTests

Posted by kl...@apache.org.
Extracting IntegrationTests from UnitTests


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

Branch: refs/heads/feature/GEODE-1276
Commit: 74c625edbda0a52e73e0accf7240b62e612688cd
Parents: ef0a624
Author: Kirk Lund <kl...@apache.org>
Authored: Tue Apr 26 16:16:33 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Tue Apr 26 16:16:33 2016 -0700

----------------------------------------------------------------------
 ...auncherLifecycleCommandsIntegrationTest.java | 155 ++++++
 .../LocatorLoadSnapshotIntegrationTest.java     | 122 +++++
 .../DeadlockDetectorIntegrationTest.java        | 299 +++++++++++
 .../ClassPathLoaderIntegrationTest.java         | 503 +++++++++++++++++++
 4 files changed, 1079 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/74c625ed/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsIntegrationTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsIntegrationTest.java
new file mode 100644
index 0000000..c7479bb
--- /dev/null
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsIntegrationTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.management.internal.cli.commands;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.jar.Attributes;
+import java.util.jar.Attributes.Name;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.internal.util.IOUtils;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * The LauncherLifecycleCommandsJUnitTest class is a test suite of test cases testing the contract and functionality of
+ * the lifecycle launcher GemFire shell (Gfsh) commands.
+ *
+ * @see com.gemstone.gemfire.management.internal.cli.commands.LauncherLifecycleCommands
+ * @see org.junit.Assert
+ * @see org.junit.Test
+ * @since 7.0
+ */
+@Category(IntegrationTest.class)
+public class LauncherLifecycleCommandsIntegrationTest {
+
+  private LauncherLifecycleCommands launcherCommands;
+
+  @Before
+  public void setup() {
+    launcherCommands = new LauncherLifecycleCommands();
+  }
+
+  @After
+  public void tearDown() {
+    launcherCommands = null;
+  }
+
+  @Test
+  public void testGemFireCoreClasspath() throws IOException {
+    File coreDependenciesJar = new File(LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME);
+
+    assertNotNull(coreDependenciesJar);
+    assertTrue(coreDependenciesJar + " is not a file", coreDependenciesJar.isFile());
+
+    Collection<String> expectedJarDependencies = Arrays.asList("antlr", "commons-io", "commons-lang", "commons-logging",
+            "geode", "jackson-annotations", "jackson-core", "jackson-databind", "jansi", "jline", "snappy-java",
+            "spring-core", "spring-shell", "jetty-server", "jetty-servlet", "jetty-webapp", "jetty-util", "jetty-http",
+            "servlet-api", "jetty-io", "jetty-security", "jetty-xml"
+
+    );
+
+    assertJarFileManifestClassPath(coreDependenciesJar, expectedJarDependencies);
+  }
+
+  @Test
+  public void testReadPid() throws IOException {
+    final int expectedPid = 12345;
+
+    File pidFile = new File(getClass().getSimpleName().concat("_testReadPid.pid"));
+
+    assertTrue(pidFile.createNewFile());
+
+    pidFile.deleteOnExit();
+    writePid(pidFile, expectedPid);
+
+    final int actualPid = getLauncherLifecycleCommands().readPid(pidFile);
+
+    assertEquals(expectedPid, actualPid);
+  }
+
+  private LauncherLifecycleCommands getLauncherLifecycleCommands() {
+    return launcherCommands;
+  }
+
+  private void writePid(final File pidFile, final int pid) throws IOException {
+    final FileWriter fileWriter = new FileWriter(pidFile, false);
+    fileWriter.write(String.valueOf(pid));
+    fileWriter.write("\n");
+    fileWriter.flush();
+    IOUtils.close(fileWriter);
+  }
+
+  private void assertJarFileManifestClassPath(final File dependenciesJar,
+                                              final Collection<String> expectedJarDependencies) throws IOException {
+    JarFile dependenciesJarFile = new JarFile(dependenciesJar);
+    Manifest manifest = dependenciesJarFile.getManifest();
+
+    assertNotNull(manifest);
+
+    Attributes attributes = manifest.getMainAttributes();
+
+    assertNotNull(attributes);
+    assertTrue(attributes.containsKey(Name.CLASS_PATH));
+
+    String[] actualJarDependencies = attributes.getValue(Name.CLASS_PATH).split(" ");
+
+    assertNotNull(actualJarDependencies);
+    assertTrue(String.format("Expected the actual number of JAR dependencies to be (%1$d); but was (%2$d)!",
+            expectedJarDependencies.size(), actualJarDependencies.length),
+            actualJarDependencies.length >= expectedJarDependencies.size());
+    //assertTrue(Arrays.asList(actualJarDependencies).containsAll(expectedJarDependencies));
+
+    List<String> actualJarDependenciesList = new ArrayList<>(Arrays.asList(actualJarDependencies));
+    List<String> missingExpectedJarDependenciesList = new ArrayList<>(expectedJarDependencies.size());
+
+    for (String expectedJarDependency : expectedJarDependencies) {
+      boolean containsExpectedJar = false;
+
+      for (int index = 0, size = actualJarDependenciesList.size(); index < size; index++) {
+        if (actualJarDependenciesList.get(index).toLowerCase().contains(expectedJarDependency.toLowerCase())) {
+          actualJarDependenciesList.remove(index);
+          containsExpectedJar = true;
+          break;
+        }
+      }
+
+      if (!containsExpectedJar) {
+        missingExpectedJarDependenciesList.add(expectedJarDependency);
+      }
+    }
+
+    assertTrue(String.format(
+            "GemFire dependencies JAR file (%1$s) does not contain the expected dependencies (%2$s) in the Manifest Class-Path attribute (%3$s)!",
+            dependenciesJar, missingExpectedJarDependenciesList, attributes.getValue(Name.CLASS_PATH)),
+            missingExpectedJarDependenciesList.isEmpty());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/74c625ed/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotIntegrationTest.java
new file mode 100644
index 0000000..08648d9
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotIntegrationTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.distributed.internal;
+
+import static org.junit.Assert.*;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.server.ServerLoad;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests extracted from LocatorLoadSnapshotJUnitTest
+ */
+@Category(IntegrationTest.class)
+public class LocatorLoadSnapshotIntegrationTest {
+
+  /**
+   * A basic test of concurrent functionality. Starts a number of
+   * threads making requests and expects the load to be balanced between
+   * three servers.
+   * @throws InterruptedException
+   */
+  @Category(FlakyTest.class) // GEODE-613: lots of threads, async action, IntegrationTest-not-UnitTest, thread joins, time sensitive
+  @Test
+  public void testConcurrentBalancing() throws InterruptedException {
+    int NUM_THREADS = 50;
+    final int NUM_REQUESTS = 10000;
+    int ALLOWED_THRESHOLD = 50; //We should never be off by more than
+    //the number of concurrent threads.
+
+    final LocatorLoadSnapshot sn = new LocatorLoadSnapshot();
+    final ServerLocation l1 = new ServerLocation("localhost", 1);
+    final ServerLocation l2 = new ServerLocation("localhost", 2);
+    final ServerLocation l3 = new ServerLocation("localhost", 3);
+
+    int initialLoad1 = (int) (Math.random() * (NUM_REQUESTS / 2));
+    int initialLoad2 = (int) (Math.random() * (NUM_REQUESTS / 2));
+    int initialLoad3 = (int) (Math.random() * (NUM_REQUESTS / 2));
+
+    sn.addServer(l1, new String[0], new ServerLoad(initialLoad1, 1, 0, 1));
+    sn.addServer(l2, new String[0], new ServerLoad(initialLoad2, 1, 0, 1));
+    sn.addServer(l3, new String[0], new ServerLoad(initialLoad3, 1, 0, 1));
+
+    final Map loadCounts = new HashMap();
+    loadCounts.put(l1, new AtomicInteger(initialLoad1));
+    loadCounts.put(l2, new AtomicInteger(initialLoad2));
+    loadCounts.put(l3, new AtomicInteger(initialLoad3));
+
+    Thread[] threads = new Thread[NUM_THREADS];
+//    final Object lock = new Object();
+    for(int i =0; i < NUM_THREADS; i++) {
+      threads[i] = new Thread("Thread-" + i) {
+        public void run() {
+          for(int ii = 0; ii < NUM_REQUESTS; ii++) {
+            ServerLocation location;
+//            synchronized(lock) {
+            location = sn.getServerForConnection(null, Collections.EMPTY_SET);
+//            }
+            AtomicInteger count = (AtomicInteger) loadCounts.get(location);
+            count.incrementAndGet();
+          }
+        }
+      };
+    }
+
+    for(int i =0; i < NUM_THREADS; i++) {
+      threads[i].start();
+    }
+
+    for(int i =0; i < NUM_THREADS; i++) {
+      Thread t = threads[i];
+      long ms = 30 * 1000;
+      t.join(30 * 1000);
+      if (t.isAlive()) {
+        for(int j =0; j < NUM_THREADS; j++) {
+          threads[j].interrupt();
+        }
+        fail("Thread did not terminate after " + ms + " ms: " + t);
+      }
+    }
+
+    double expectedPerServer = ( initialLoad1 + initialLoad2 + initialLoad3 +
+            NUM_REQUESTS * NUM_THREADS) / (double) loadCounts.size();
+//    for(Iterator itr = loadCounts.entrySet().iterator(); itr.hasNext(); ) {
+//      Map.Entry entry = (Entry) itr.next();
+//      ServerLocation location = (ServerLocation) entry.getKey();
+//      AI count= (AI) entry.getValue();
+//    }
+
+    for(Iterator itr = loadCounts.entrySet().iterator(); itr.hasNext(); ) {
+      Map.Entry entry = (Map.Entry) itr.next();
+      ServerLocation location = (ServerLocation) entry.getKey();
+      AtomicInteger count= (AtomicInteger) entry.getValue();
+      int difference = (int) Math.abs(count.get() - expectedPerServer);
+      assertTrue("Count " + count + " for server " + location + " is not within " + ALLOWED_THRESHOLD + " of " + expectedPerServer, difference < ALLOWED_THRESHOLD);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/74c625ed/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorIntegrationTest.java
new file mode 100644
index 0000000..4eaf9cb
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorIntegrationTest.java
@@ -0,0 +1,299 @@
+/*
+ * 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.distributed.internal.deadlock;
+
+import static org.junit.Assert.*;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Set;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * TODO: can we get rid of the Thread.sleep calls?
+ */
+@Category(IntegrationTest.class)
+public class DeadlockDetectorIntegrationTest {
+
+  private volatile Set<Thread> stuckThreads;
+
+  @Before
+  public void setUp() throws Exception {
+    stuckThreads = Collections.synchronizedSet(new HashSet<Thread>());
+  }
+
+  /**
+   * IntegrationTests are forkEvery 1 so cleanup is not necessary
+   */
+  @After
+  public void tearDown() throws Exception {
+    for (Thread thread: stuckThreads) {
+      thread.interrupt();
+    }
+
+    stuckThreads.clear();
+  }
+
+  /**
+   * must be IntegrationTest because: "we can't clean up the threads deadlocked on monitors"
+   */
+  @Test
+  public void testMonitorDeadlock() throws Exception {
+    final Object lock1 = new Object();
+    final Object lock2 = new Object();
+
+    Thread thread1 =  new Thread() {
+      public void run() {
+        stuckThreads.add(Thread.currentThread());
+
+        synchronized(lock1) {
+          Thread thread2 = new Thread() {
+            public void run() {
+              stuckThreads.add(Thread.currentThread());
+              synchronized(lock2) {
+                synchronized(lock1) {
+                  System.out.println("we won't get here");
+                }
+              }
+            }
+          };
+
+          thread2.start();
+
+          try {
+            Thread.sleep(1000);
+            synchronized(lock2) {
+              System.out.println("We won't get here");
+            }
+          } catch (InterruptedException ignore) {
+          }
+        }
+      }
+    };
+
+    thread1.start();
+
+    Thread.sleep(2000);
+
+    DeadlockDetector detector = new DeadlockDetector();
+    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
+    LinkedList<Dependency> deadlocks = detector.findDeadlock();
+
+    System.out.println("deadlocks=" +  DeadlockDetector.prettyFormat(deadlocks));
+
+    assertEquals(4, detector.findDeadlock().size());
+  }
+
+  /**
+   * Make sure that we can detect a deadlock between two threads
+   * that are trying to acquire a two different syncs in the different orders.
+   */
+  @Test
+  public void testSyncDeadlock() throws Exception {
+    final Lock lock1 = new ReentrantLock();
+    final Lock lock2 = new ReentrantLock();
+
+    Thread thread1 =  new Thread() {
+      public void run() {
+        stuckThreads.add(Thread.currentThread());
+
+        lock1.lock();
+
+        Thread thread2 = new Thread() {
+          public void run() {
+            stuckThreads.add(Thread.currentThread());
+            lock2.lock();
+            try {
+              lock1.tryLock(10, TimeUnit.SECONDS);
+            } catch (InterruptedException e) {
+              //ignore
+            }
+            lock2.unlock();
+          }
+        };
+
+        thread2.start();
+
+        try {
+          Thread.sleep(1000);
+          lock2.tryLock(10, TimeUnit.SECONDS);
+        } catch (InterruptedException ignore) {
+        }
+
+        lock1.unlock();
+      }
+    };
+
+    thread1.start();
+
+    Thread.sleep(2000);
+
+    DeadlockDetector detector = new DeadlockDetector();
+    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
+    LinkedList<Dependency> deadlocks = detector.findDeadlock();
+
+    System.out.println("deadlocks=" +  DeadlockDetector.prettyFormat(deadlocks));
+
+    assertEquals(4, detector.findDeadlock().size());
+  }
+
+  @Ignore("Semaphore deadlock detection is not supported by the JDK")
+  @Test
+  public void testSemaphoreDeadlock() throws Exception {
+    final Semaphore lock1 = new Semaphore(1);
+    final Semaphore lock2 = new Semaphore(1);
+
+    Thread thread1 =  new Thread() {
+      public void run() {
+        stuckThreads.add(Thread.currentThread());
+
+        try {
+          lock1.acquire();
+        } catch (InterruptedException e1) {
+          e1.printStackTrace();
+        }
+
+        Thread thread2 = new Thread() {
+          public void run() {
+            stuckThreads.add(Thread.currentThread());
+            try {
+              lock2.acquire();
+              lock1.tryAcquire(10, TimeUnit.SECONDS);
+            } catch (InterruptedException ignore) {
+            }
+            lock2.release();
+          }
+        };
+
+        thread2.start();
+
+        try {
+          Thread.sleep(1000);
+          lock2.tryAcquire(10, TimeUnit.SECONDS);
+        } catch (InterruptedException ignore) {
+        }
+
+        lock1.release();
+      }
+    };
+
+    thread1.start();
+
+    Thread.sleep(2000);
+
+    DeadlockDetector detector = new DeadlockDetector();
+    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
+    LinkedList<Dependency> deadlocks = detector.findDeadlock();
+
+    System.out.println("deadlocks=" +  DeadlockDetector.prettyFormat(deadlocks));
+
+    assertEquals(4, detector.findDeadlock().size());
+  }
+
+  @Ignore("ReadWriteLock deadlock detection is not currently supported by DeadlockDetector")
+  @Test
+  public void testReadLockDeadlock() throws Exception {
+    final ReadWriteLock lock1 = new ReentrantReadWriteLock();
+    final ReadWriteLock lock2 = new ReentrantReadWriteLock();
+
+    Thread thread1 =  new Thread() {
+      @Override
+      public void run() {
+        stuckThreads.add(Thread.currentThread());
+
+        lock1.readLock().lock();
+
+        Thread thread2 = new Thread() {
+          @Override
+          public void run() {
+            stuckThreads.add(Thread.currentThread());
+            lock2.readLock().lock();
+            try {
+              lock1.writeLock().tryLock(10, TimeUnit.SECONDS);
+            } catch (InterruptedException ignore) {
+            }
+            lock2.readLock().unlock();
+          }
+        };
+
+        thread2.start();
+
+        try {
+          Thread.sleep(1000);
+          lock2.writeLock().tryLock(10, TimeUnit.SECONDS);
+        } catch (InterruptedException ignore) {
+        }
+
+        lock1.readLock().unlock();
+      }
+    };
+
+    thread1.start();
+
+    Thread.sleep(2000);
+
+    DeadlockDetector detector = new DeadlockDetector();
+    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
+    LinkedList<Dependency> deadlocks = detector.findDeadlock();
+
+    System.out.println("deadlocks=" +  deadlocks);
+
+    assertEquals(4, detector.findDeadlock().size());
+  }
+
+  /**
+   * A fake dependency monitor.
+   */
+  private static class MockDependencyMonitor implements DependencyMonitor {
+
+    Set<Dependency<Thread, Serializable>> blockedThreads = new HashSet<>();
+    Set<Dependency<Serializable, Thread>> held = new HashSet<>();
+
+    @Override
+    public Set<Dependency<Thread, Serializable>> getBlockedThreads(Thread[] allThreads) {
+      return blockedThreads;
+    }
+
+    public void addDependency(String resource, Thread thread) {
+      held.add(new Dependency<>(resource, thread));
+    }
+
+    public void addDependency(Thread thread, String resource) {
+      blockedThreads.add(new Dependency<>(thread, resource));
+    }
+
+    @Override
+    public Set<Dependency<Serializable, Thread>> getHeldResources(Thread[] allThreads) {
+      return held;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/74c625ed/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderIntegrationTest.java
new file mode 100644
index 0000000..4c836bd
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderIntegrationTest.java
@@ -0,0 +1,503 @@
+/*
+ * 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.internal;
+
+import static org.junit.Assert.*;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+import java.util.Enumeration;
+import java.util.Vector;
+
+import org.apache.bcel.Constants;
+import org.apache.bcel.classfile.JavaClass;
+import org.apache.bcel.generic.ClassGen;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import com.gemstone.gemfire.internal.ClassPathLoaderTest.BrokenClassLoader;
+import com.gemstone.gemfire.internal.ClassPathLoaderTest.NullClassLoader;
+import com.gemstone.gemfire.internal.ClassPathLoaderTest.SimpleClassLoader;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for {@link ClassPathLoader}.
+ *
+ * Extracted from ClassPathLoaderTest.
+ */
+@Category(IntegrationTest.class)
+public class ClassPathLoaderIntegrationTest {
+
+  private static final int TEMP_FILE_BYTES_COUNT = 256;
+
+  private volatile File tempFile;
+  private volatile File tempFile2;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(ClassPathLoader.EXCLUDE_TCCL_PROPERTY, "false");
+    System.setProperty(ClassPathLoader.EXT_LIB_DIR_PARENT_PROPERTY, this.temporaryFolder.getRoot().getAbsolutePath());
+
+    this.tempFile = this.temporaryFolder.newFile("tempFile1.tmp");
+    FileOutputStream fos = new FileOutputStream(this.tempFile);
+    fos.write(new byte[TEMP_FILE_BYTES_COUNT]);
+    fos.close();
+
+    this.tempFile2 = this.temporaryFolder.newFile("tempFile2.tmp");
+    fos = new FileOutputStream(this.tempFile2);
+    fos.write(new byte[TEMP_FILE_BYTES_COUNT]);
+    fos.close();
+  }
+
+  /**
+   * Verifies that <tt>getResource</tt> works with custom loader from {@link ClassPathLoader}.
+   */
+  @Test
+  public void testGetResourceWithCustomLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceWithCustomLoader");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+
+    String resourceToGet = "com/nowhere/testGetResourceWithCustomLoader.rsc";
+    URL url = dcl.getResource(resourceToGet);
+    assertNotNull(url);
+
+    InputStream is = url != null ? url.openStream() : null;
+    assertNotNull(is);
+
+    int totalBytesRead = 0;
+    byte[] input = new byte[128];
+
+    BufferedInputStream bis = new BufferedInputStream(is);
+    for (int bytesRead = bis.read(input); bytesRead > -1;) {
+      totalBytesRead += bytesRead;
+      bytesRead = bis.read(input);
+    }
+    bis.close();
+
+    assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
+  }
+
+  /**
+   * Verifies that <tt>getResources</tt> works with custom loader from {@link ClassPathLoader}.
+   */
+  @Test
+  public void testGetResourcesWithCustomLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceWithCustomLoader");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+
+    String resourceToGet = "com/nowhere/testGetResourceWithCustomLoader.rsc";
+    Enumeration<URL> urls = dcl.getResources(resourceToGet);
+    assertNotNull(urls);
+    assertTrue(urls.hasMoreElements());
+
+    URL url = urls.nextElement();
+    InputStream is = url != null ? url.openStream() : null;
+    assertNotNull(is);
+
+    int totalBytesRead = 0;
+    byte[] input = new byte[128];
+
+    BufferedInputStream bis = new BufferedInputStream(is);
+    for (int bytesRead = bis.read(input); bytesRead > -1;) {
+      totalBytesRead += bytesRead;
+      bytesRead = bis.read(input);
+    }
+    bis.close();
+
+    assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
+  }
+
+  /**
+   * Verifies that <tt>getResourceAsStream</tt> works with custom loader from {@link ClassPathLoader}.
+   */
+  @Test
+  public void testGetResourceAsStreamWithCustomLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceAsStreamWithCustomLoader");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+
+    String resourceToGet = "com/nowhere/testGetResourceAsStreamWithCustomLoader.rsc";
+    InputStream is = dcl.getResourceAsStream(resourceToGet);
+    assertNotNull(is);
+
+    int totalBytesRead = 0;
+    byte[] input = new byte[128];
+
+    BufferedInputStream bis = new BufferedInputStream(is);
+    for (int bytesRead = bis.read(input); bytesRead > -1;) {
+      totalBytesRead += bytesRead;
+      bytesRead = bis.read(input);
+    }
+    bis.close();
+
+    assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
+  }
+
+  /**
+   * Verifies that <tt>getResource</tt> works with TCCL from {@link ClassPathLoader}.
+   */
+  @Test
+  public void testGetResourceWithTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceWithTCCL");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+
+    String resourceToGet = "com/nowhere/testGetResourceWithTCCL.rsc";
+    assertNull(dcl.getResource(resourceToGet));
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      URL url = dcl.getResource(resourceToGet);
+      assertNotNull(url);
+
+      InputStream is = url.openStream();
+      assertNotNull(is);
+
+      int totalBytesRead = 0;
+      byte[] input = new byte[128];
+
+      BufferedInputStream bis = new BufferedInputStream(is);
+      for (int bytesRead = bis.read(input); bytesRead > -1;) {
+        totalBytesRead += bytesRead;
+        bytesRead = bis.read(input);
+      }
+      bis.close();
+
+      assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that <tt>getResources</tt> works with TCCL from {@link ClassPathLoader}.
+   */
+  @Test
+  public void testGetResourcesWithTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceWithTCCL");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+
+    String resourceToGet = "com/nowhere/testGetResourceWithTCCL.rsc";
+    Enumeration<URL> urls = dcl.getResources(resourceToGet);
+    assertNotNull(urls);
+    assertFalse(urls.hasMoreElements());
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      urls = dcl.getResources(resourceToGet);
+      assertNotNull(urls);
+
+      URL url = urls.nextElement();
+      InputStream is = url.openStream();
+      assertNotNull(is);
+
+      int totalBytesRead = 0;
+      byte[] input = new byte[128];
+
+      BufferedInputStream bis = new BufferedInputStream(is);
+      for (int bytesRead = bis.read(input); bytesRead > -1;) {
+        totalBytesRead += bytesRead;
+        bytesRead = bis.read(input);
+      }
+      bis.close();
+
+      assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that <tt>getResourceAsStream</tt> works with TCCL from {@link ClassPathLoader}.
+   */
+  @Test
+  public void testGetResourceAsStreamWithTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceAsStreamWithTCCL");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+
+    String resourceToGet = "com/nowhere/testGetResourceAsStreamWithTCCL.rsc";
+    assertNull(dcl.getResourceAsStream(resourceToGet));
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // ensure that TCCL is only CL that can find this resource
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      InputStream is = dcl.getResourceAsStream(resourceToGet);
+      assertNotNull(is);
+
+      int totalBytesRead = 0;
+      byte[] input = new byte[128];
+
+      BufferedInputStream bis = new BufferedInputStream(is);
+      for (int bytesRead = bis.read(input); bytesRead > -1;) {
+        totalBytesRead += bytesRead;
+        bytesRead = bis.read(input);
+      }
+      bis.close();
+
+      assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that JAR files found in the extlib directory will be correctly
+   * added to the {@link ClassPathLoader}.
+   */
+  @Test
+  public void testJarsInExtLib() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testJarsInExtLib");
+
+    File EXT_LIB_DIR = ClassPathLoader.defineEXT_LIB_DIR();
+    EXT_LIB_DIR.mkdir();
+
+    File subdir = new File(EXT_LIB_DIR, "cplju");
+    subdir.mkdir();
+
+    final ClassBuilder classBuilder = new ClassBuilder();
+
+    writeJarBytesToFile(new File(EXT_LIB_DIR, "ClassPathLoaderJUnit1.jar"),
+            classBuilder.createJarFromClassContent("com/cpljunit1/ClassPathLoaderJUnit1", "package com.cpljunit1; public class ClassPathLoaderJUnit1 {}"));
+    writeJarBytesToFile(new File(subdir, "ClassPathLoaderJUnit2.jar"),
+            classBuilder.createJarFromClassContent("com/cpljunit2/ClassPathLoaderJUnit2", "package com.cpljunit2; public class ClassPathLoaderJUnit2 {}"));
+
+    ClassPathLoader classPathLoader = ClassPathLoader.createWithDefaults(false);
+    try {
+      classPathLoader.forName("com.cpljunit1.ClassPathLoaderJUnit1");
+    } catch (ClassNotFoundException cnfex) {
+      fail("JAR file not correctly added to Classpath");
+    }
+
+    try {
+      classPathLoader.forName("com.cpljunit2.ClassPathLoaderJUnit2");
+    } catch (ClassNotFoundException cnfex) {
+      fail("JAR file not correctly added to Classpath");
+    }
+
+    assertNotNull(classPathLoader.getResource("com/cpljunit2/ClassPathLoaderJUnit2.class"));
+
+    Enumeration<URL> urls = classPathLoader.getResources("com/cpljunit1");
+    if  (!urls.hasMoreElements()) {
+      fail("Resources should return one element");
+    }
+  }
+
+  /**
+   * Verifies that the 3rd custom loader will get the resource. Parent cannot find it and TCCL is broken. This verifies
+   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
+   */
+  @Test
+  public void testGetResourceAsStreamWithMultipleCustomLoaders() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceAsStreamWithMultipleCustomLoaders");
+
+    // create DCL such that the 3rd loader should find the resource
+    // first custom loader becomes parent which won't find anything
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
+    dcl = dcl.addOrReplace(new NullClassLoader());
+
+    String resourceToGet = "com/nowhere/testGetResourceAsStreamWithMultipleCustomLoaders.rsc";
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set TCCL to throw errors which makes sure we find before checking TCCL
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      InputStream is = dcl.getResourceAsStream(resourceToGet);
+      assertNotNull(is);
+      is.close();
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that the 3rd custom loader will get the resource. Parent cannot find it and TCCL is broken. This verifies
+   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
+   */
+  @Test
+  public void testGetResourceWithMultipleCustomLoaders() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceWithMultipleCustomLoaders");
+
+    // create DCL such that the 3rd loader should find the resource
+    // first custom loader becomes parent which won't find anything
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
+    dcl = dcl.addOrReplace(new NullClassLoader());
+
+    String resourceToGet = "com/nowhere/testGetResourceWithMultipleCustomLoaders.rsc";
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set TCCL to throw errors which makes sure we find before checking TCCL
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      URL url = dcl.getResource(resourceToGet);
+      assertNotNull(url);
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that the 3rd custom loader will get the resources. Parent cannot find it and TCCL is broken. This verifies
+   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
+   */
+  @Test
+  public void testGetResourcesWithMultipleCustomLoaders() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceWithMultipleCustomLoaders");
+
+    // create DCL such that the 3rd loader should find the resource
+    // first custom loader becomes parent which won't find anything
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+    dcl = dcl.addOrReplace(new GeneratingClassLoader2());
+    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
+    dcl = dcl.addOrReplace(new NullClassLoader());
+
+    String resourceToGet = "com/nowhere/testGetResourceWithMultipleCustomLoaders.rsc";
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set TCCL to throw errors which makes sure we find before checking TCCL
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      Enumeration<URL> urls = dcl.getResources(resourceToGet);
+      assertNotNull(urls);
+      assertTrue(urls.hasMoreElements());
+
+      URL url = urls.nextElement();
+      assertNotNull(url);
+
+      // Should find two with unique URLs
+      assertTrue("Did not find all resources.", urls.hasMoreElements());
+      URL url2 = urls.nextElement();
+      assertNotNull(url2);
+      assertTrue("Resource URLs should be unique.", !url.equals(url2));
+
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  private void writeJarBytesToFile(File jarFile, byte[] jarBytes) throws IOException {
+    final OutputStream outStream = new FileOutputStream(jarFile);
+    outStream.write(jarBytes);
+    outStream.close();
+  }
+
+  /**
+   * Custom class loader which uses BCEL to always dynamically generate a class for any class name it tries to load.
+   */
+  private class GeneratingClassLoader extends ClassLoader {
+
+    /**
+     * Currently unused but potentially useful for some future test. This causes this loader to only generate a class
+     * that the parent could not find.
+     *
+     * @param parent
+     *          the parent class loader to check with first
+     */
+    @SuppressWarnings("unused")
+    public GeneratingClassLoader(ClassLoader parent) {
+      super(parent);
+    }
+
+    /**
+     * Specifies no parent to ensure that this loader generates the named class.
+     */
+    public GeneratingClassLoader() {
+      super(null); // no parent!!
+    }
+
+    @Override
+    protected Class<?> findClass(String name) throws ClassNotFoundException {
+      ClassGen cg = new ClassGen(name, "java.lang.Object", "<generated>", Constants.ACC_PUBLIC | Constants.ACC_SUPER, null);
+      cg.addEmptyConstructor(Constants.ACC_PUBLIC);
+      JavaClass jClazz = cg.getJavaClass();
+      byte[] bytes = jClazz.getBytes();
+      return defineClass(jClazz.getClassName(), bytes, 0, bytes.length);
+    }
+
+    @Override
+    protected URL findResource(String name) {
+      URL url = null;
+      try {
+        url = getTempFile().getAbsoluteFile().toURI().toURL();
+        System.out.println("GeneratingClassLoader#findResource returning " + url);
+      } catch (IOException e) {
+      }
+      return url;
+    }
+
+    @Override
+    protected Enumeration<URL> findResources(String name) throws IOException {
+      URL url = null;
+      try {
+        url = getTempFile().getAbsoluteFile().toURI().toURL();
+        System.out.println("GeneratingClassLoader#findResources returning " + url);
+      } catch (IOException e) {
+      }
+      Vector<URL> urls = new Vector<URL>();
+      urls.add(url);
+      return urls.elements();
+    }
+
+    protected File getTempFile() {
+      return tempFile;
+    }
+  }
+
+  /**
+   * Custom class loader which uses BCEL to always dynamically generate a class for any class name it tries to load.
+   */
+  private class GeneratingClassLoader2 extends GeneratingClassLoader {
+    @Override
+    protected File getTempFile() {
+      return tempFile2;
+    }
+  }
+}



[15/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
Updating and fixing tests


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

Branch: refs/heads/feature/GEODE-1276
Commit: ef0a6243009118aaaf1a9818275db94c61954000
Parents: 70c5467
Author: Kirk Lund <kl...@apache.org>
Authored: Tue Apr 26 16:15:16 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Tue Apr 26 16:15:16 2016 -0700

----------------------------------------------------------------------
 .../SessionReplicationIntegrationJUnitTest.java |    2 +-
 .../session/Tomcat7SessionsJUnitTest.java       |    6 +-
 .../modules/session/TestSessionsBase.java       |    8 +-
 .../session/Tomcat6SessionsJUnitTest.java       |    6 +-
 .../gemstone/gemfire/BundledJarsJUnitTest.java  |    4 +-
 .../LauncherLifecycleCommandsDUnitTest.java     |    4 +-
 .../LauncherLifecycleCommandsJUnitTest.java     |  194 +--
 .../annotations/ExperimentalJUnitTest.java      |    1 -
 .../gemfire/internal/ClassPathLoader.java       |    8 +-
 .../gemfire/SystemFailureJUnitTest.java         |    1 +
 .../java/com/gemstone/gemfire/TXJUnitTest.java  |   38 +-
 .../cache/AttributesFactoryJUnitTest.java       |   25 +-
 .../gemfire/cache/Bug36619JUnitTest.java        |   77 --
 .../gemfire/cache/Bug52289JUnitTest.java        |   88 --
 .../gemfire/cache/ConnectionPoolDUnitTest.java  |   18 +-
 ...alStaticArrayShouldNotCauseSegFaultTest.java |   86 ++
 ...MembershipAttributesAreSerializableTest.java |   74 +
 .../gemfire/cache/OperationJUnitTest.java       |   96 +-
 .../gemfire/cache/RoleExceptionJUnitTest.java   |   26 +-
 .../internal/OpExecutorImplJUnitTest.java       |  175 ++-
 .../internal/ServerBlackListJUnitTest.java      |   25 +-
 .../locator/LocatorStatusResponseJUnitTest.java |    1 -
 .../pooling/ConnectionManagerJUnitTest.java     |    8 +-
 .../management/ResourceManagerDUnitTest.java    |    2 +-
 .../PutOperationContextJUnitTest.java           |   34 +-
 .../GetOperationContextImplJUnitTest.java       |   37 +-
 .../PartitionRegionHelperDUnitTest.java         |    2 +-
 .../gemfire/cache/query/CacheUtils.java         |    2 +-
 .../query/dunit/PdxStringQueryDUnitTest.java    |  148 +-
 .../query/dunit/QueryUsingPoolDUnitTest.java    |    2 +-
 .../cache/query/dunit/RemoteQueryDUnitTest.java |    4 +-
 .../query/functional/CountStarJUnitTest.java    |   18 +-
 .../IUMRMultiIndexesMultiRegionJUnitTest.java   |   12 +-
 .../IndexMaintenanceAsynchJUnitTest.java        |    2 +-
 .../functional/JavaSerializationJUnitTest.java  |   17 +-
 .../query/functional/LimitClauseJUnitTest.java  |    8 +-
 .../cache/query/functional/MiscJUnitTest.java   |   32 +-
 .../MultiRegionIndexUsageJUnitTest.java         |   32 +-
 .../NonDistinctOrderByReplicatedJUnitTest.java  |    4 +-
 .../NonDistinctOrderByTestImplementation.java   |   16 +-
 .../functional/OrderByReplicatedJUnitTest.java  |    4 +-
 .../functional/OrderByTestImplementation.java   |   16 +-
 .../query/functional/PdxOrderByJUnitTest.java   |    2 +-
 .../CompiledAggregateFunctionJUnitTest.java     |   28 +-
 .../query/internal/CompiledInJUnitTest.java     |   72 +-
 .../internal/NWayMergeResultsJUnitTest.java     |   37 +-
 ...ueryFromClauseCanonicalizationJUnitTest.java |    6 +-
 .../QueryObjectSerializationJUnitTest.java      |   46 +-
 .../query/internal/ResultsBagJUnitTest.java     |   52 +-
 .../ResultsBagLimitBehaviourJUnitTest.java      |   38 +-
 .../ResultsCollectionWrapperLimitJUnitTest.java |   29 +-
 .../SelectResultsComparatorJUnitTest.java       |   24 +-
 .../StructBagLimitBehaviourJUnitTest.java       |   23 +-
 .../query/internal/StructSetJUnitTest.java      |   15 +-
 .../internal/aggregate/AggregatorJUnitTest.java |   23 +-
 .../index/AsynchIndexMaintenanceJUnitTest.java  |    8 +-
 .../internal/index/HashIndexSetJUnitTest.java   |  192 +--
 .../internal/index/IndexElemArrayJUnitTest.java |   24 +-
 .../index/IndexMaintenanceJUnitTest.java        |   16 +-
 .../index/IndexStatisticsJUnitTest.java         |   12 +-
 .../index/MemoryIndexStoreJUnitTest.java        |  698 +++++-----
 ...exStoreWithInplaceModificationJUnitTest.java |   45 +-
 .../PRBasicIndexCreationDUnitTest.java          |    2 +-
 .../PRBasicMultiIndexCreationDUnitTest.java     |    2 +-
 .../query/partitioned/PRQueryDUnitHelper.java   |    6 +-
 .../query/partitioned/PRQueryDUnitTest.java     |    2 +-
 .../internal/ConnectionCountProbeJUnitTest.java |   22 +-
 .../cache/util/PasswordUtilJUnitTest.java       |   11 +-
 .../gemfire/cache30/CacheLoaderTestCase.java    |   14 +-
 .../gemfire/cache30/CacheMapTxnDUnitTest.java   |    2 +-
 .../cache30/CacheStatisticsDUnitTest.java       |    4 +-
 .../gemfire/cache30/CacheXml41DUnitTest.java    |   10 +-
 .../gemfire/cache30/CacheXml57DUnitTest.java    |    2 +-
 .../gemfire/cache30/CacheXml58DUnitTest.java    |    2 +-
 .../gemfire/cache30/CacheXml65DUnitTest.java    |    2 +-
 .../cache30/ClientMembershipDUnitTest.java      |    2 +-
 .../gemfire/cache30/DiskRegionDUnitTest.java    |    2 +-
 .../cache30/DistAckMapMethodsDUnitTest.java     |    2 +-
 ...tributedAckPersistentRegionCCEDUnitTest.java |   28 +-
 .../gemfire/cache30/MultiVMRegionTestCase.java  |   52 +-
 .../gemfire/cache30/PutAllMultiVmDUnitTest.java |    4 +-
 .../gemfire/cache30/QueueMsgDUnitTest.java      |    2 +-
 .../RegionMembershipListenerDUnitTest.java      |    6 +-
 .../gemfire/cache30/RegionTestCase.java         |    4 +-
 .../gemfire/cache30/SlowRecDUnitTest.java       |    4 +-
 .../distributed/AbstractLauncherJUnitTest.java  |    4 +-
 .../AbstractLauncherServiceStatusJUnitTest.java |  123 +-
 .../distributed/LocatorLauncherJUnitTest.java   |   13 +-
 .../LocatorLauncherLocalJUnitTest.java          |    6 +-
 .../LocatorLauncherRemoteJUnitTest.java         |    8 +-
 .../distributed/LocatorStateJUnitTest.java      |   46 +-
 .../distributed/ServerLauncherJUnitTest.java    |   33 +-
 .../ServerLauncherLocalJUnitTest.java           |    4 +-
 .../ServerLauncherRemoteJUnitTest.java          |    4 +-
 .../AtomicLongWithTerminalStateJUnitTest.java   |   13 +-
 .../internal/DistributionConfigJUnitTest.java   |   85 +-
 .../InternalDistributedSystemJUnitTest.java     |    6 +-
 .../internal/LocatorLoadSnapshotJUnitTest.java  |  104 +-
 .../internal/ProductUseLogJUnitTest.java        |  105 +-
 .../internal/ServerLocatorJUnitTest.java        |   21 +-
 .../internal/SharedConfigurationJUnitTest.java  |    1 -
 .../internal/StartupMessageDataJUnitTest.java   |   34 +-
 .../deadlock/DeadlockDetectorJUnitTest.java     |  331 -----
 .../internal/deadlock/DeadlockDetectorTest.java |  140 ++
 .../deadlock/DependencyGraphJUnitTest.java      |   24 +-
 .../deadlock/UnsafeThreadLocalJUnitTest.java    |   47 +-
 .../internal/membership/NetViewJUnitTest.java   |   51 +-
 .../membership/gms/GMSMemberJUnitTest.java      |   51 +-
 .../gms/auth/GMSAuthenticatorJUnitTest.java     |  123 +-
 .../locator/GMSLocatorRecoveryJUnitTest.java    |   34 +-
 .../gms/membership/StatRecorderJUnitTest.java   |   57 +-
 .../messenger/GMSQuorumCheckerJUnitTest.java    |  172 ++-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |   71 +-
 .../TcpServerBackwardCompatDUnitTest.java       |    8 +-
 .../internal/tcpserver/TcpServerJUnitTest.java  |    6 +-
 .../internal/AbstractConfigJUnitTest.java       |  125 +-
 .../gemfire/internal/ArrayEqualsJUnitTest.java  |  193 ---
 .../internal/AvailablePortJUnitTest.java        |   12 +-
 .../internal/ClassPathLoaderJUnitTest.java      | 1261 ------------------
 .../gemfire/internal/ClassPathLoaderTest.java   |  890 ++++++++++++
 .../gemfire/internal/ConfigSourceJUnitTest.java |    9 +-
 .../internal/CopyOnWriteHashSetJUnitTest.java   |   17 +-
 .../internal/DataSerializableJUnitTest.java     |  772 +++++------
 .../gemfire/internal/FileUtilJUnitTest.java     |    7 +-
 .../internal/GemFireVersionJUnitTest.java       |   16 +-
 .../internal/HeapDataOutputStreamJUnitTest.java |   17 +-
 .../gemfire/internal/LineWrapUnitJUnitTest.java |   13 +-
 .../gemfire/internal/NanoTimerJUnitTest.java    |   31 +-
 .../gemfire/internal/ObjIdMapJUnitTest.java     |   29 +-
 .../internal/OneTaskOnlyDecoratorJUnitTest.java |   22 +-
 .../PutAllOperationContextJUnitTest.java        |   14 +-
 ...lityShouldUseArrayEqualsIntegrationTest.java |  194 +++
 .../gemfire/internal/SocketCloserJUnitTest.java |    6 +-
 .../internal/SocketCloserWithWaitJUnitTest.java |    1 +
 .../gemfire/internal/StatSamplerJUnitTest.java  |   52 +-
 .../internal/UniqueIdGeneratorJUnitTest.java    |   18 +-
 .../AbstractDistributedRegionJUnitTest.java     |    8 +-
 .../internal/cache/AbstractRegionJUnitTest.java |    2 -
 .../internal/cache/AbstractRegionMapTest.java   |   23 +-
 .../internal/cache/BucketRegionJUnitTest.java   |   32 +-
 .../internal/cache/DeltaFaultInDUnitTest.java   |    2 +-
 .../internal/cache/DeltaSizingDUnitTest.java    |    2 +-
 .../gemfire/internal/cache/DiskIdJUnitTest.java |   51 +-
 .../internal/cache/DiskInitFileJUnitTest.java   |   35 +-
 .../cache/DiskWriteAttributesJUnitTest.java     |   99 +-
 .../cache/DistributedRegionJUnitTest.java       |   29 +-
 .../internal/cache/EntryEventImplTest.java      |   23 +-
 .../cache/EnumListenerEventJUnitTest.java       |   22 +-
 .../cache/OffHeapValueWrapperJUnitTest.java     |    9 +-
 .../gemfire/internal/cache/OldVLJUnitTest.java  |   19 +-
 .../cache/OldValueImporterTestBase.java         |   17 +-
 .../cache/OplogEntryIdMapJUnitTest.java         |   12 +-
 .../cache/OplogEntryIdSetJUnitTest.java         |   12 +-
 .../gemfire/internal/cache/OplogJUnitTest.java  |   40 +-
 .../internal/cache/OplogRVVJUnitTest.java       |   32 +-
 .../cache/OrderedTombstoneMapJUnitTest.java     |   10 +-
 .../cache/PartitionAttributesImplJUnitTest.java |   11 +-
 .../PartitionedRegionCacheCloseDUnitTest.java   |    2 +-
 .../cache/PartitionedRegionDUnitTestCase.java   |    2 +-
 .../PartitionedRegionDestroyDUnitTest.java      |    2 +-
 .../PartitionedRegionEvictionDUnitTest.java     |    4 +-
 .../cache/PartitionedRegionHelperJUnitTest.java |   15 +-
 .../PartitionedRegionQueryEvaluatorTest.java    |   69 +-
 ...artitionedRegionRedundancyZoneDUnitTest.java |    6 +-
 .../PartitionedRegionSingleHopDUnitTest.java    |    2 +-
 ...onedRegionSingleNodeOperationsJUnitTest.java |   12 +-
 .../cache/PartitionedRegionStatsJUnitTest.java  |    8 +-
 .../PartitionedRegionTestUtilsDUnitTest.java    |   12 +-
 .../cache/RemoteTransactionDUnitTest.java       |   22 +-
 .../cache/SearchLoadAndWriteProcessorTest.java  |    1 -
 .../cache/SimpleDiskRegionJUnitTest.java        |    6 +-
 .../gemfire/internal/cache/VLJUnitTest.java     |   35 +-
 .../cache/control/FilterByPathJUnitTest.java    |   24 +-
 .../control/MemoryThresholdsJUnitTest.java      |    7 +-
 .../control/RebalanceOperationDUnitTest.java    |   28 +-
 ...onFunctionExecutionNoSingleHopDUnitTest.java |    2 +-
 ...onExecutionSelectorNoSingleHopDUnitTest.java |    2 +-
 ...gionFunctionExecutionSingleHopDUnitTest.java |    2 +-
 .../execute/PRCustomPartitioningDUnitTest.java  |    2 +-
 .../execute/PRFunctionExecutionDUnitTest.java   |   12 +-
 .../execute/PRPerformanceTestDUnitTest.java     |    2 +-
 .../SimpleExtensionPointJUnitTest.java          |    7 +-
 .../cache/ha/EventIdOptimizationJUnitTest.java  |  129 +-
 .../internal/cache/ha/HAGIIDUnitTest.java       |   12 +-
 .../cache/ha/HARQAddOperationJUnitTest.java     |    2 +-
 .../cache/ha/HARegionQueueDUnitTest.java        |    2 +-
 .../cache/ha/ThreadIdentifierJUnitTest.java     |   12 +-
 .../cache/locks/TXLockServiceDUnitTest.java     |    2 +-
 .../FetchEntriesMessageJUnitTest.java           |   13 +-
 .../OfflineMembersDetailsJUnitTest.java         |   11 +-
 .../PartitionedRegionLoadModelJUnitTest.java    |  121 +-
 .../persistence/BackupInspectorJUnitTest.java   |   52 +-
 .../TemporaryResultSetFactoryJUnitTest.java     |   44 +-
 .../RegionEntryFactoryBuilderJUnitTest.java     |   11 +-
 .../cache/tier/sockets/Bug37805DUnitTest.java   |    2 +-
 .../CacheServerTransactionsDUnitTest.java       |    8 +-
 .../sockets/ClientHealthMonitorJUnitTest.java   |    8 +-
 .../tier/sockets/ClientServerMiscDUnitTest.java |   22 +-
 .../cache/tier/sockets/ConflationDUnitTest.java |    4 +-
 .../DurableClientReconnectDUnitTest.java        |    2 +-
 .../tier/sockets/FilterProfileJUnitTest.java    |   20 +-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |    4 +-
 .../sockets/InterestListFailoverDUnitTest.java  |    2 +-
 .../cache/tier/sockets/MessageJUnitTest.java    |   20 +-
 .../tier/sockets/ObjectPartListJUnitTest.java   |   27 +-
 .../sockets/RedundancyLevelPart1DUnitTest.java  |   88 +-
 .../sockets/RedundancyLevelPart2DUnitTest.java  |   96 +-
 .../tier/sockets/RedundancyLevelTestBase.java   |    4 +-
 .../tier/sockets/command/CommitCommandTest.java |   12 +-
 .../cache/versions/RVVExceptionJUnitTest.java   |   14 +-
 .../versions/RegionVersionHolder2JUnitTest.java |    5 +-
 .../versions/RegionVersionVectorJUnitTest.java  |   39 +-
 .../ParallelGatewaySenderQueueJUnitTest.java    |    5 +-
 .../xmlcache/AbstractEntityResolverTest.java    |   17 +-
 .../xmlcache/AbstractXmlParserJUnitTest.java    |    1 -
 .../cache/xmlcache/CacheXmlParserJUnitTest.java |    1 -
 .../xmlcache/CacheXmlVersionJUnitTest.java      |    1 -
 .../xmlcache/GeodeEntityResolverJUnitTest.java  |    6 +-
 .../PivotalEntityResolverJUnitTest.java         |    7 +-
 .../gemfire/internal/jndi/ContextJUnitTest.java |  153 +--
 .../internal/logging/LogServiceJUnitTest.java   |    1 -
 .../internal/logging/SortLogFileJUnitTest.java  |    7 +-
 .../logging/log4j/AlertAppenderJUnitTest.java   |    1 -
 .../log4j/LocalizedMessageJUnitTest.java        |    1 -
 .../log4j/LogWriterAppenderJUnitTest.java       |    1 -
 .../internal/offheap/DataTypeJUnitTest.java     |  142 +-
 ...tingOutOfOffHeapMemoryListenerJUnitTest.java |    2 +
 .../internal/offheap/FreeListManagerTest.java   |    1 +
 .../offheap/LifecycleListenerJUnitTest.java     |    2 +-
 .../offheap/MemoryAllocatorJUnitTest.java       |   11 +
 .../offheap/MemoryBlockNodeJUnitTest.java       |    3 +-
 .../offheap/MemoryInspectorImplJUnitTest.java   |    7 +-
 .../offheap/OffHeapHelperJUnitTest.java         |    3 -
 .../offheap/OffHeapStorageJUnitTest.java        |   65 +-
 ...ffHeapStoredObjectAddressStackJUnitTest.java |    1 +
 .../offheap/OffHeapStoredObjectJUnitTest.java   |    1 -
 .../OffHeapStoredObjectSliceJUnitTest.java      |    1 -
 ...ffHeapStoredObjectWithHeapFormJUnitTest.java |    1 -
 .../offheap/TinyMemoryBlockJUnitTest.java       |    1 -
 .../offheap/TinyStoredObjectJUnitTest.java      |  502 ++++---
 .../process/LocalProcessLauncherJUnitTest.java  |   36 +-
 .../internal/process/PidFileJUnitTest.java      |    4 +-
 .../internal/size/ObjectSizerJUnitTest.java     |   21 +-
 .../internal/size/ObjectTraverserJUnitTest.java |   53 +-
 .../size/ReflectionObjectSizerJUnitTest.java    |   15 +-
 .../size/SizeClassOnceObjectSizerJUnitTest.java |   15 +-
 .../size/WellKnownClassSizerJUnitTest.java      |   18 +-
 .../statistics/SampleCollectorJUnitTest.java    |   54 +-
 .../statistics/StatMonitorHandlerJUnitTest.java |   12 +-
 .../statistics/StatisticsMonitorJUnitTest.java  |   51 +-
 .../internal/util/ArrayUtilsJUnitTest.java      |    1 -
 .../gemfire/internal/util/BytesJUnitTest.java   |   27 +-
 .../internal/util/CollectionUtilsJUnitTest.java |    1 -
 .../internal/util/DelayedActionJUnitTest.java   |    9 +-
 .../CompactConcurrentHashSetJUnitTest.java      |   16 +-
 .../concurrent/ReentrantSemaphoreJUnitTest.java |   13 +-
 .../SemaphoreReadWriteLockJUnitTest.java        |   28 +-
 ...ersalMembershipListenerAdapterDUnitTest.java |    2 +-
 .../stats/GatewayReceiverStatsJUnitTest.java    |    8 +-
 .../bean/stats/RegionStatsJUnitTest.java        |    2 +-
 .../bean/stats/StatsRateJUnitTest.java          |   50 +-
 .../internal/JettyHelperJUnitTest.java          |   25 +-
 .../beans/DistributedSystemBridgeJUnitTest.java |    3 +-
 .../internal/cli/CliUtilDUnitTest.java          |    2 +-
 .../internal/cli/CommandManagerJUnitTest.java   |   11 +-
 .../cli/CommandSeparatorEscapeJUnitTest.java    |   47 +-
 .../internal/cli/DataCommandJsonJUnitTest.java  |   14 +-
 .../internal/cli/GfshParserJUnitTest.java       |   18 +-
 .../cli/annotations/CliArgumentJUnitTest.java   |  179 +--
 .../commands/GemfireDataCommandsDUnitTest.java  |    2 +-
 .../HTTPServiceSSLSupportJUnitTest.java         |    2 +-
 .../cli/commands/MemberCommandsDUnitTest.java   |    2 +-
 .../RegionPathConverterJUnitTest.java           |   19 +-
 .../DescribeDiskStoreFunctionJUnitTest.java     |   20 +-
 .../functions/ListIndexFunctionJUnitTest.java   |  108 +-
 .../cli/parser/ParserUtilsJUnitTest.java        |   14 +-
 .../preprocessor/PreprocessorJUnitTest.java     |    7 +-
 .../PreprocessorUtilsJUnitTest.java             |   17 +-
 .../cli/shell/GfshConfigInitFileJUnitTest.java  |   20 +-
 .../shell/GfshExecutionStrategyJUnitTest.java   |   15 +-
 .../cli/shell/GfshHistoryJUnitTest.java         |   22 +-
 .../domain/CacheElementJUnitTest.java           |    3 +-
 .../configuration/utils/XmlUtilsJUnitTest.java  |    1 -
 .../WanCommandsControllerJUnitTest.java         |    9 +-
 .../pdx/JSONPdxClientServerDUnitTest.java       |    2 +-
 .../pdx/PdxDeserializationDUnitTest.java        |   12 +-
 .../security/ClientAuthorizationTestCase.java   |    2 +-
 .../security/NotAuthorizedExceptionTest.java    |    4 +-
 .../security/templates/PKCSPrincipalTest.java   |    3 +-
 .../templates/UsernamePrincipalTest.java        |    3 +-
 .../com/gemstone/gemfire/test/dunit/Assert.java |    2 +-
 .../gemfire/test/dunit/RMIException.java        |    2 +-
 .../test/dunit/SerializableCallable.java        |    2 +-
 .../test/dunit/SerializableRunnable.java        |    2 +-
 .../test/process/MainLauncherJUnitTest.java     |    1 -
 .../cache/query/cq/dunit/CqQueryDUnitTest.java  |    2 +-
 .../cq/dunit/CqQueryUsingPoolDUnitTest.java     |    4 +-
 .../cache/query/cq/dunit/CqStatsDUnitTest.java  |    2 +-
 .../cq/dunit/CqStatsUsingPoolDUnitTest.java     |    2 +-
 .../PartitionedRegionCqQueryDUnitTest.java      |    2 +-
 .../internal/cache/PutAllCSDUnitTest.java       |    2 +-
 .../cache/RemoteCQTransactionDUnitTest.java     |   20 +-
 .../internal/cache/ha/CQListGIIDUnitTest.java   |    2 +-
 .../tier/sockets/DurableClientTestCase.java     |    8 +-
 .../CacheServerManagementDUnitTest.java         |    2 +-
 .../LuceneIndexForPartitionedRegion.java        |    5 +-
 .../HeterogeneousLuceneSerializer.java          |   82 ++
 .../HeterogenousLuceneSerializer.java           |   83 --
 .../internal/LuceneEventListenerJUnitTest.java  |    1 -
 .../LuceneIndexRecoveryHAJUnitTest.java         |    7 +-
 .../LuceneQueryResultsImplJUnitTest.java        |    2 -
 .../LuceneResultStructImpJUnitTest.java         |    2 -
 .../internal/LuceneServiceImplJUnitTest.java    |   14 +-
 .../PartitionedRepositoryManagerJUnitTest.java  |    8 +-
 .../internal/StringQueryProviderJUnitTest.java  |    2 -
 .../directory/RegionDirectoryJUnitTest.java     |    2 +-
 .../DistributedScoringJUnitTest.java            |   21 +-
 .../distributed/EntryScoreJUnitTest.java        |    2 +-
 .../LuceneFunctionContextJUnitTest.java         |    2 +-
 .../distributed/LuceneFunctionJUnitTest.java    |    2 +-
 .../TopEntriesCollectorJUnitTest.java           |    2 +-
 .../TopEntriesFunctionCollectorJUnitTest.java   |    2 +-
 .../distributed/TopEntriesJUnitTest.java        |    2 +-
 .../internal/filesystem/ChunkKeyJUnitTest.java  |    1 -
 .../internal/filesystem/FileJUnitTest.java      |    5 +-
 .../filesystem/FileSystemJUnitTest.java         |    4 +-
 .../IndexRepositoryImplJUnitTest.java           |    6 +-
 .../IndexRepositoryImplPerformanceTest.java     |    6 +-
 .../HeterogeneousLuceneSerializerJUnitTest.java |   86 ++
 .../HeterogenousLuceneSerializerJUnitTest.java  |   90 --
 .../serializer/PdxFieldMapperJUnitTest.java     |    2 -
 .../ReflectionFieldMapperJUnitTest.java         |    2 -
 .../xml/LuceneIndexXmlGeneratorJUnitTest.java   |    2 -
 .../xml/LuceneIndexXmlParserJUnitTest.java      |    1 -
 geode-pulse/build.gradle                        |   12 +-
 .../controllers/PulseControllerJUnitTest.java   |   63 +-
 .../tools/pulse/testbed/driver/PulseUITest.java |    2 +-
 .../SerialGatewaySenderQueueDUnitTest.java      |    2 +-
 .../wancommand/WanCommandStatusDUnitTest.java   |    6 +-
 .../ShellCommandsControllerJUnitTest.java       |   15 +-
 .../web/http/ClientHttpRequestJUnitTest.java    |    8 +-
 ...ableObjectHttpMessageConverterJUnitTest.java |    6 +-
 .../RestHttpOperationInvokerJUnitTest.java      |   18 +-
 gradle/dependency-versions.properties           |   14 +-
 gradle/test.gradle                              |    9 +-
 345 files changed, 5735 insertions(+), 6816 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java b/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
index 004f9fe..d3b3751 100644
--- a/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
+++ b/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
@@ -476,7 +476,7 @@ public class SessionReplicationIntegrationJUnitTest {
 //        request.setURI("/test/request2");
 //        response.parse(tester.getResponses(request.generate()));
 //
-//        assertEquals("null", response.getContent());
+//        assertIndexDetailsEquals("null", response.getContent());
 //
 //        Region r = getRegion();
 //        assertNull("Region should not contain session", r.get(id));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/extensions/geode-modules-tomcat7/src/test/java/com/gemstone/gemfire/modules/session/Tomcat7SessionsJUnitTest.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-tomcat7/src/test/java/com/gemstone/gemfire/modules/session/Tomcat7SessionsJUnitTest.java b/extensions/geode-modules-tomcat7/src/test/java/com/gemstone/gemfire/modules/session/Tomcat7SessionsJUnitTest.java
index 5ac87ab..99cf9c2 100644
--- a/extensions/geode-modules-tomcat7/src/test/java/com/gemstone/gemfire/modules/session/Tomcat7SessionsJUnitTest.java
+++ b/extensions/geode-modules-tomcat7/src/test/java/com/gemstone/gemfire/modules/session/Tomcat7SessionsJUnitTest.java
@@ -17,13 +17,11 @@
 package com.gemstone.gemfire.modules.session;
 
 import com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
-/**
- */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class Tomcat7SessionsJUnitTest extends TestSessionsBase {
 
   // Set up the session manager we need

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/TestSessionsBase.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/TestSessionsBase.java b/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/TestSessionsBase.java
index 544658e..194a7c7 100644
--- a/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/TestSessionsBase.java
+++ b/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/TestSessionsBase.java
@@ -38,10 +38,8 @@ import java.io.PrintWriter;
 
 import static junit.framework.Assert.*;
 
-/**
- *
- */
 public abstract class TestSessionsBase {
+
   private static EmbeddedTomcat server;
 
   private static Region<String, HttpSession> region;
@@ -68,7 +66,7 @@ public abstract class TestSessionsBase {
     servlet = server.addServlet("/test/*", "default", CommandServlet.class.getName());
     server.startContainer();
 
-    /**
+    /*
      * Can only retrieve the region once the container has started up
      * (and the cache has started too).
      */
@@ -212,7 +210,7 @@ public abstract class TestSessionsBase {
 //        req.removeParameter("value");
 //        response = wc.getResponse(req);
 //
-//        assertEquals(value, response.getText());
+//        assertIndexDetailsEquals(value, response.getText());
 //    }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/Tomcat6SessionsJUnitTest.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/Tomcat6SessionsJUnitTest.java b/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/Tomcat6SessionsJUnitTest.java
index 7500a3c..26ed697 100644
--- a/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/Tomcat6SessionsJUnitTest.java
+++ b/extensions/geode-modules/src/test/java/com/gemstone/gemfire/modules/session/Tomcat6SessionsJUnitTest.java
@@ -17,13 +17,11 @@
 package com.gemstone.gemfire.modules.session;
 
 import com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
-/**
- */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class Tomcat6SessionsJUnitTest extends TestSessionsBase {
 
   // Set up the session manager we need

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-assembly/src/test/java/com/gemstone/gemfire/BundledJarsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/BundledJarsJUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/BundledJarsJUnitTest.java
index 8ae3b22..9b59f80 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/BundledJarsJUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/BundledJarsJUnitTest.java
@@ -36,10 +36,10 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.FileUtil;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.gemstone.gemfire.util.test.TestUtil;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class BundledJarsJUnitTest {
 
   private static final String VERSION_PATTERN = "[0-9-_.v]{3,}.*\\.jar$";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
index b394063..e6fa87b 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@ -711,7 +711,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
 
     // TODO figure out what output to assert and validate on now that 'stop locator' uses Gfsh's logger
     // and standard err/out...
-    //assertEquals(CliStrings.format(CliStrings.STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE, pathname),
+    //assertIndexDetailsEquals(CliStrings.format(CliStrings.STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE, pathname),
     //  StringUtils.trim(toString(result)));
 
     WaitCriterion waitCriteria = new WaitCriterion() {
@@ -789,7 +789,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
 
     // TODO figure out what output to assert and validate on now that 'stop locator' uses Gfsh's logger
     // and standard err/out...
-    //assertEquals(CliStrings.format(CliStrings.STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE, memberId),
+    //assertIndexDetailsEquals(CliStrings.format(CliStrings.STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE, memberId),
     //  StringUtils.trim(toString(result)));
 
     WaitCriterion waitCriteria = new WaitCriterion() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
index dc89f03..0b5cdfb 100755
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsJUnitTest.java
@@ -16,6 +16,21 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.Stack;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.GemFireException;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.ServerLauncher;
@@ -26,28 +41,6 @@ import com.gemstone.gemfire.internal.lang.SystemUtils;
 import com.gemstone.gemfire.internal.util.IOUtils;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.Stack;
-import java.util.jar.Attributes;
-import java.util.jar.Attributes.Name;
-import java.util.jar.JarFile;
-import java.util.jar.Manifest;
-
-import static org.junit.Assert.*;
 
 /**
  * The LauncherLifecycleCommandsJUnitTest class is a test suite of test cases testing the contract and functionality of
@@ -62,9 +55,6 @@ import static org.junit.Assert.*;
 @Category(UnitTest.class)
 public class LauncherLifecycleCommandsJUnitTest {
 
-  private static final String GFSH_DEPENDENCIES_JAR_PATHNAME = IOUtils.appendToPath(System.getenv("GEMFIRE"), "lib",
-      "gfsh-dependencies.jar");
-
   private LauncherLifecycleCommands launcherCommands;
 
   @Before
@@ -77,18 +67,6 @@ public class LauncherLifecycleCommandsJUnitTest {
     launcherCommands = null;
   }
 
-  protected LauncherLifecycleCommands getLauncherLifecycleCommands() {
-    return launcherCommands;
-  }
-
-  protected void writePid(final File pidFile, final int pid) throws IOException {
-    final FileWriter fileWriter = new FileWriter(pidFile, false);
-    fileWriter.write(String.valueOf(pid));
-    fileWriter.write("\n");
-    fileWriter.flush();
-    IOUtils.close(fileWriter);
-  }
-
   @Test
   public void testAddGemFirePropertyFileToCommandLine() {
     final List<String> commandLine = new ArrayList<>();
@@ -260,28 +238,6 @@ public class LauncherLifecycleCommandsJUnitTest {
   }
 
   @Test
-  public void testReadPidWithNonExistingFile() {
-    assertEquals(LauncherLifecycleCommands.INVALID_PID,
-        getLauncherLifecycleCommands().readPid(new File("/path/to/non_existing/pid.file")));
-  }
-
-  @Test
-  public void testReadPid() throws IOException {
-    final int expectedPid = 12345;
-
-    File pidFile = new File(getClass().getSimpleName().concat("_testReadPid.pid"));
-
-    assertTrue(pidFile.createNewFile());
-
-    pidFile.deleteOnExit();
-    writePid(pidFile, expectedPid);
-
-    final int actualPid = getLauncherLifecycleCommands().readPid(pidFile);
-
-    assertEquals(expectedPid, actualPid);
-  }
-
-  @Test
   @SuppressWarnings("deprecation")
   public void testGetClasspath() {
     assertEquals(System.getProperty("java.class.path"), getLauncherLifecycleCommands().getClasspath(null));
@@ -295,23 +251,6 @@ public class LauncherLifecycleCommandsJUnitTest {
   }
 
   @Test
-  public void testGemFireCoreClasspath() throws IOException {
-    File coreDependenciesJar = new File(LauncherLifecycleCommands.CORE_DEPENDENCIES_JAR_PATHNAME);
-
-    assertNotNull(coreDependenciesJar);
-    assertTrue(coreDependenciesJar + " is not a file", coreDependenciesJar.isFile());
-
-    Collection<String> expectedJarDependencies = Arrays.asList("antlr", "commons-io", "commons-lang", "commons-logging",
-        "geode", "jackson-annotations", "jackson-core", "jackson-databind", "jansi", "jline", "snappy-java",
-        "spring-core", "spring-shell", "jetty-server", "jetty-servlet", "jetty-webapp", "jetty-util", "jetty-http",
-        "servlet-api", "jetty-io", "jetty-security", "jetty-xml"
-
-    );
-
-    assertJarFileManifestClassPath(coreDependenciesJar, expectedJarDependencies);
-  }
-
-  @Test
   public void testGetSystemClasspath() {
     assertEquals(System.getProperty("java.class.path"), getLauncherLifecycleCommands().getSystemClasspath());
   }
@@ -342,17 +281,6 @@ public class LauncherLifecycleCommandsJUnitTest {
     assertEquals(expectedClasspath, actualClasspath);
   }
 
-  private String toPath(Object... pathElements) {
-    String path = "";
-
-    for (Object pathElement : pathElements) {
-      path += (path.isEmpty() ? StringUtils.EMPTY_STRING : File.pathSeparator);
-      path += pathElement;
-    }
-
-    return path;
-  }
-
   @Test
   public void testToClasspath() {
     final boolean EXCLUDE_SYSTEM_CLASSPATH = false;
@@ -406,64 +334,6 @@ public class LauncherLifecycleCommandsJUnitTest {
     assertEquals(expectedClasspath, actualClasspath);
   }
 
-  private void assertJarFileManifestClassPath(final File dependenciesJar,
-      final Collection<String> expectedJarDependencies) throws IOException {
-    JarFile dependenciesJarFile = new JarFile(dependenciesJar);
-    Manifest manifest = dependenciesJarFile.getManifest();
-
-    assertNotNull(manifest);
-
-    Attributes attributes = manifest.getMainAttributes();
-
-    assertNotNull(attributes);
-    assertTrue(attributes.containsKey(Name.CLASS_PATH));
-
-    String[] actualJarDependencies = attributes.getValue(Name.CLASS_PATH).split(" ");
-
-    assertNotNull(actualJarDependencies);
-    assertTrue(String.format("Expected the actual number of JAR dependencies to be (%1$d); but was (%2$d)!",
-        expectedJarDependencies.size(), actualJarDependencies.length),
-        actualJarDependencies.length >= expectedJarDependencies.size());
-    //assertTrue(Arrays.asList(actualJarDependencies).containsAll(expectedJarDependencies));
-
-    List<String> actualJarDependenciesList = new ArrayList<>(Arrays.asList(actualJarDependencies));
-    List<String> missingExpectedJarDependenciesList = new ArrayList<>(expectedJarDependencies.size());
-
-    for (String expectedJarDependency : expectedJarDependencies) {
-      boolean containsExpectedJar = false;
-
-      for (int index = 0, size = actualJarDependenciesList.size(); index < size; index++) {
-        if (actualJarDependenciesList.get(index).toLowerCase().contains(expectedJarDependency.toLowerCase())) {
-          actualJarDependenciesList.remove(index);
-          containsExpectedJar = true;
-          break;
-        }
-      }
-
-      if (!containsExpectedJar) {
-        missingExpectedJarDependenciesList.add(expectedJarDependency);
-      }
-    }
-
-    assertTrue(String.format(
-        "GemFire dependencies JAR file (%1$s) does not contain the expected dependencies (%2$s) in the Manifest Class-Path attribute (%3$s)!",
-        dependenciesJar, missingExpectedJarDependenciesList, attributes.getValue(Name.CLASS_PATH)),
-        missingExpectedJarDependenciesList.isEmpty());
-  }
-
-  private String toClasspath(final String... jarFilePathnames) {
-    String classpath = StringUtils.EMPTY_STRING;
-
-    if (jarFilePathnames != null) {
-      for (final String jarFilePathname : jarFilePathnames) {
-        classpath += (classpath.isEmpty() ? StringUtils.EMPTY_STRING : File.pathSeparator);
-        classpath += jarFilePathname;
-      }
-    }
-
-    return classpath;
-  }
-
   @Test
   public void testGetJavaPathname() {
     assertEquals(IOUtils.appendToPath(System.getProperty("java.home"), "bin",
@@ -579,4 +449,38 @@ public class LauncherLifecycleCommandsJUnitTest {
         expectedCommandLineElements.isEmpty());
   }
 
+  @Test
+  public void testReadPidWithNonExistingFile() {
+    assertEquals(LauncherLifecycleCommands.INVALID_PID,
+            getLauncherLifecycleCommands().readPid(new File("/path/to/non_existing/pid.file")));
+  }
+
+  private LauncherLifecycleCommands getLauncherLifecycleCommands() {
+    return launcherCommands;
+  }
+
+  private String toClasspath(final String... jarFilePathnames) {
+    String classpath = StringUtils.EMPTY_STRING;
+
+    if (jarFilePathnames != null) {
+      for (final String jarFilePathname : jarFilePathnames) {
+        classpath += (classpath.isEmpty() ? StringUtils.EMPTY_STRING : File.pathSeparator);
+        classpath += jarFilePathname;
+      }
+    }
+
+    return classpath;
+  }
+
+  private String toPath(Object... pathElements) {
+    String path = "";
+
+    for (Object pathElement : pathElements) {
+      path += (path.isEmpty() ? StringUtils.EMPTY_STRING : File.pathSeparator);
+      path += pathElement;
+    }
+
+    return path;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-common/src/test/java/com/gemstone/gemfire/annotations/ExperimentalJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-common/src/test/java/com/gemstone/gemfire/annotations/ExperimentalJUnitTest.java b/geode-common/src/test/java/com/gemstone/gemfire/annotations/ExperimentalJUnitTest.java
index 8067a4a..2df1cd6 100755
--- a/geode-common/src/test/java/com/gemstone/gemfire/annotations/ExperimentalJUnitTest.java
+++ b/geode-common/src/test/java/com/gemstone/gemfire/annotations/ExperimentalJUnitTest.java
@@ -32,7 +32,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * annotation can be applied to Interfaces, Classes, Public and Protected
  * Fields, Enums, Enum Constants, Public and Protected Methods, Packages,
  * and Constructors.
- * 
  */
 @Category(UnitTest.class)
 public class ExperimentalJUnitTest {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/main/java/com/gemstone/gemfire/internal/ClassPathLoader.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/ClassPathLoader.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/ClassPathLoader.java
index 1cb2332..028a57e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/ClassPathLoader.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/ClassPathLoader.java
@@ -87,8 +87,11 @@ public final class ClassPathLoader {
   // class is found in a directory instead of a JAR file (as when testing),
   // then it will be relative to the location of the root of the package and
   // class.
-  public static final File EXT_LIB_DIR = new File((new File(ClassPathLoader.class.getProtectionDomain().getCodeSource()
-      .getLocation().getPath())).getParent(), "ext");
+  public static final String EXT_LIB_DIR_PARENT_PROPERTY = "gemfire.ClassPathLoader.EXT_LIB_DIR";
+  public static final String EXT_LIB_DIR_PARENT_DEFAULT = ClassPathLoader.class.getProtectionDomain().getCodeSource().getLocation().getPath();
+  static final File defineEXT_LIB_DIR() {
+    return new File((new File(System.getProperty(EXT_LIB_DIR_PARENT_PROPERTY, EXT_LIB_DIR_PARENT_DEFAULT))).getParent(), "ext");
+  }
 
   // This token is placed into the list of class loaders to determine where
   // to insert the TCCL when in forName(...), getResource(...), etc.
@@ -205,6 +208,7 @@ public final class ClassPathLoader {
     
     // Add user JAR files from the EXT_LIB_DIR directory using a single ClassLoader
     try {
+      File EXT_LIB_DIR = defineEXT_LIB_DIR();
       if (EXT_LIB_DIR.exists()) {
         if (!EXT_LIB_DIR.isDirectory() || !EXT_LIB_DIR.canRead()) {
           logger.warn("Cannot read from directory when attempting to load JAR files: {}", EXT_LIB_DIR.getAbsolutePath());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/SystemFailureJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/SystemFailureJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/SystemFailureJUnitTest.java
index b138e7f..6d9dcbc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/SystemFailureJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/SystemFailureJUnitTest.java
@@ -33,6 +33,7 @@ public class SystemFailureJUnitTest {
 
   private static final int LONG_WAIT = 30000;
   private int oldWaitTime;
+
   @Before
   public void setWaitTime() {
     oldWaitTime = SystemFailure.SHUTDOWN_WAIT;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/TXJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/TXJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/TXJUnitTest.java
index ff24d24..acdba69 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/TXJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/TXJUnitTest.java
@@ -3069,7 +3069,7 @@ public class TXJUnitTest {
       assertNotNull(event.getRegion().getCache());
       assertNotNull(event.getRegion().getCache().getCacheTransactionManager());
       assertEquals(this.getTXId(), event.getTransactionId());
-      // assertEquals(event.getTransactionId(), event.getRegion().getCache().getCacheTransactionManager().getTransactionId(), );
+      // assertIndexDetailsEquals(event.getTransactionId(), event.getRegion().getCache().getCacheTransactionManager().getTransactionId(), );
       if (!isPR()) assertEquals("IsDistributed Assertion!",  this.isDistributed(), event.isDistributed());
       assertEquals(this.getKey(), event.getKey());
       assertSame(this.getCallBackArg(), event.getCallbackArgument());
@@ -3753,28 +3753,28 @@ public class TXJUnitTest {
         assertTrue(!txIt.hasNext());
       }
       reg1.invalidate("key1");
-//      assertEquals(0, nonTxKeys.size());
+//      assertIndexDetailsEquals(0, nonTxKeys.size());
       assertEquals(1, txKeys.size());
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(0, txValues.size());
       assertTrue(txKeys.contains("key1"));
       assertTrue(!txValues.contains("value1"));
       reg1.create("key2", "value2");
       reg1.create("key3", "value3");
-//      assertEquals(0, nonTxKeys.size());
+//      assertIndexDetailsEquals(0, nonTxKeys.size());
       assertEquals(3, txKeys.size());
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(2, txValues.size());
       reg1.put("key1", "value1");
-//      assertEquals(0, nonTxKeys.size());
+//      assertIndexDetailsEquals(0, nonTxKeys.size());
       assertEquals(3, txKeys.size());
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(3, txValues.size());
       reg1.localInvalidate("key2");
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(2, txValues.size());
       reg1.invalidate("key1");
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(1, txValues.size());
       reg1.destroy("key2");
       reg1.destroy("key3");
@@ -3813,7 +3813,7 @@ public class TXJUnitTest {
       this.txMgr.begin();
       reg1.create("key1", "value1");
       Collection txValues = reg1.values();
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(1, txValues.size());
       assertTrue(txValues.contains("value1"));
       {
@@ -3880,7 +3880,7 @@ public class TXJUnitTest {
       assertEquals(1, txValues.size());
       assertTrue(txValues.iterator().hasNext());
       assertEquals("txValue1", txValues.iterator().next());
-//      assertEquals(0, nonTxValues.size());
+//      assertIndexDetailsEquals(0, nonTxValues.size());
       // non-TX collections can now be used in a transactional context
       try {
         nonTxValues.iterator().hasNext();
@@ -4132,7 +4132,7 @@ public class TXJUnitTest {
     assertEquals("LV 2", reg1.getEntry("key1").getValue());
     this.txMgr.rollback();
     assertTrue(!reg1.containsKey("key1"));
-    // assertEquals("LV 2", reg1.getEntry("key1").getValue());
+    // assertIndexDetailsEquals("LV 2", reg1.getEntry("key1").getValue());
     // reg1.localDestroy("key1");
     // TX load: commit check
     this.txMgr.begin();
@@ -5623,7 +5623,7 @@ public class TXJUnitTest {
         lruRegion.create("key" + i, null);
       }
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_NULL);
-      // assertEquals(lruSize, lruRegion.entrySet(false).size());
+      // assertIndexDetailsEquals(lruSize, lruRegion.entrySet(false).size());
       this.txMgr.begin();
       for(int i=0; i<numToPut; ++i) {
         lruRegion.get("key"+i, new Integer(i));
@@ -5688,8 +5688,8 @@ public class TXJUnitTest {
       // LocalRegion lrReg = (LocalRegion) lruRegion;
       // LRUClockNode lruE = null;
       // assertNotNull(lruE = (LRUClockNode) lrReg.basicGetEntry("key"+(numToPut-1)));
-      // assertEquals(2, lruE.getRefCount());
-      // assertEquals(lruSize, lruRegion.entrySet(false).size());
+      // assertIndexDetailsEquals(2, lruE.getRefCount());
+      // assertIndexDetailsEquals(lruSize, lruRegion.entrySet(false).size());
 
       // Force the Non-Tx "put" to remove each attempt since region is full
       // and all the committed entries are currently part of a TX
@@ -6384,7 +6384,7 @@ public class TXJUnitTest {
     }
     waitForUpdates(aIindex0, 4);
 //     waitForKeys(aIindex0, 3);
-//     assertEquals(3, aIindex0.getStatistics().getNumberOfKeys()); // Shouldn't this be 1, again?
+//     assertIndexDetailsEquals(3, aIindex0.getStatistics().getNumberOfKeys()); // Shouldn't this be 1, again?
     assertEquals(2, aIindex0.getStatistics().getNumberOfValues());
     assertEquals(4, aIindex0.getStatistics().getNumUpdates());
 
@@ -6413,7 +6413,7 @@ public class TXJUnitTest {
     assertEquals(2, aIindex1.getStatistics().getNumberOfKeys()); 
     assertEquals(2, aIindex1.getStatistics().getNumberOfValues());
     assertEquals(2, aIindex1.getStatistics().getNumUpdates());
-//     assertEquals(3, aIindex0.getStatistics().getNumberOfKeys()); 
+//     assertIndexDetailsEquals(3, aIindex0.getStatistics().getNumberOfKeys());
     assertEquals(2, aIindex0.getStatistics().getNumberOfValues());
     assertEquals(4, aIindex0.getStatistics().getNumUpdates());
 
@@ -6456,7 +6456,7 @@ public class TXJUnitTest {
     waitForUpdates(aIindex1, 2);
      waitForKeys(aIindex0, 2);
 //     waitForKeys(aIindex1, 2);
-//     assertEquals(2, aIindex1.getStatistics().getNumberOfKeys()); 
+//     assertIndexDetailsEquals(2, aIindex1.getStatistics().getNumberOfKeys());
     assertEquals(2, aIindex1.getStatistics().getNumberOfValues());
     assertEquals(2, aIindex1.getStatistics().getNumUpdates());
     assertEquals(2, aIindex0.getStatistics().getNumberOfKeys()); 
@@ -6501,7 +6501,7 @@ public class TXJUnitTest {
     assertEquals(3, aIindex1.getStatistics().getNumberOfKeys()); 
     assertEquals(3, aIindex1.getStatistics().getNumberOfValues()); // Shouldn't this be 4?
     assertEquals(5, aIindex1.getStatistics().getNumUpdates()); 
-    //assertEquals(4, aIindex0.getStatistics().getNumberOfKeys()); 
+    //assertIndexDetailsEquals(4, aIindex0.getStatistics().getNumberOfKeys());
     assertEquals(3, aIindex0.getStatistics().getNumberOfValues()); // Shouldn't this be 4?
     assertEquals(7, aIindex0.getStatistics().getNumUpdates());
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
index a1d374f..69f0092 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
@@ -16,13 +16,14 @@
  */
 package com.gemstone.gemfire.cache;
 
+import static org.junit.Assert.*;
+
 import java.io.File;
 import java.util.Arrays;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.cache.util.CacheWriterAdapter;
 import com.gemstone.gemfire.compression.SnappyCompressor;
@@ -31,18 +32,12 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 /**
  * Tests the functionality of the {@link AttributesFactory} class.
  *
- *
  * @since 3.0
  */
 @Category(UnitTest.class)
-public class AttributesFactoryJUnitTest extends TestCase {
-
-  public AttributesFactoryJUnitTest(String name) {
-    super(name);
-  }
-
-  ///////////////////////  Test Methods ///////////////////////
+public class AttributesFactoryJUnitTest {
 
+  @Test
   public void testCopyConstructor() {
     AttributesFactory f1 = new AttributesFactory();
     f1.setLockGrantor(true);
@@ -60,6 +55,7 @@ public class AttributesFactoryJUnitTest extends TestCase {
    * Tests the {@link AttributesFactory#create} throws
    * the appropriate exception with poorly-configured factory.
    */
+  @Test
   public void testInvalidConfigurations() {
     AttributesFactory factory;
 
@@ -233,6 +229,7 @@ public class AttributesFactoryJUnitTest extends TestCase {
    * default} attributes factory has the advertised default
    * configuration.
    */
+  @Test
   public void testDefaultConfiguration() {
     AttributesFactory factory = new AttributesFactory();
     RegionAttributes attrs = factory.create();
@@ -275,6 +272,8 @@ public class AttributesFactoryJUnitTest extends TestCase {
     assertEquals(1, diskSizes.length);
     assertEquals(DiskStoreFactory.DEFAULT_DISK_DIR_SIZE, diskSizes[0]);
   }
+
+  @Test
   public void testDiskSynchronous() {
     {
       AttributesFactory factory = new AttributesFactory();
@@ -316,10 +315,12 @@ public class AttributesFactoryJUnitTest extends TestCase {
       assertEquals(false, attrs.isDiskSynchronous());
     }
   }
+
   /**
    * Tests the cacheListener functionality
    * @since 5.0
    */
+  @Test
   public void testCacheListeners() {
     RegionAttributes ra;
     CacheListener cl1 = new MyCacheListener();
@@ -386,9 +387,11 @@ public class AttributesFactoryJUnitTest extends TestCase {
     ra = factory.create();
     assertEquals(Arrays.asList(new CacheListener[]{cl2}), Arrays.asList(ra.getCacheListeners()));
   }
+
   /**
    * @since 5.7
    */
+  @Test
   public void testConnectionPool() {
     CacheLoader cl = new CacheLoader() {
         public Object load(LoaderHelper helper) throws CacheLoaderException {
@@ -413,7 +416,7 @@ public class AttributesFactoryJUnitTest extends TestCase {
    * Trivial cache listener impl
    * @since 5.0
    */
-  public static class MyCacheListener extends CacheListenerAdapter {
+  private static class MyCacheListener extends CacheListenerAdapter {
     // empty impl
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug36619JUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug36619JUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug36619JUnitTest.java
deleted file mode 100644
index 9f0e458..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug36619JUnitTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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;
-
-import java.io.*;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.TestCase;
-
-/** Tests classes of Bug36619 to make sure they are Serializable */
-@Category(UnitTest.class)
-public class Bug36619JUnitTest extends TestCase {
-  
-  public Bug36619JUnitTest(String name) {
-    super(name);
-  }
-
-  protected void setUp() throws Exception {
-    super.setUp();
-  }
-
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  /**
-   * Assert that MembershipAttributes are serializable.
-   */
-  public void testMembershipAttributesAreSerializable() throws Exception {
-    String[] roles = {"a", "b", "c"};
-    MembershipAttributes outMA = new MembershipAttributes(roles);
-    ByteArrayOutputStream baos = new ByteArrayOutputStream(1000);
-    ObjectOutputStream oos = new ObjectOutputStream(baos);
-    oos.writeObject(outMA);
-    
-    byte[] data = baos.toByteArray();
-    
-    ByteArrayInputStream bais = new ByteArrayInputStream(data);
-    ObjectInputStream ois = new ObjectInputStream(bais);
-    MembershipAttributes inMA = (MembershipAttributes) ois.readObject();
-    assertEquals(outMA, inMA);
-  }
-  /**
-   * Assert that SubscriptionAttributes are serializable.
-   */
-  public void testSubscriptionAttributesAreSerializable() throws Exception {
-    SubscriptionAttributes outSA = new SubscriptionAttributes();
-    ByteArrayOutputStream baos = new ByteArrayOutputStream(1000);
-    ObjectOutputStream oos = new ObjectOutputStream(baos);
-    oos.writeObject(outSA);
-    
-    byte[] data = baos.toByteArray();
-    
-    ByteArrayInputStream bais = new ByteArrayInputStream(data);
-    ObjectInputStream ois = new ObjectInputStream(bais);
-    SubscriptionAttributes inSA = (SubscriptionAttributes) ois.readObject();
-    assertEquals(outSA, inSA);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug52289JUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug52289JUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug52289JUnitTest.java
deleted file mode 100644
index 0ba5154..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/Bug52289JUnitTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- * Test case for Trac <a
- * href="https://svn.gemstone.com/trac/gemfire/ticket/52289">#52289</a>.
- * 
- * Asserts fixes for bug JDK-8076152 in JDK 1.8.0u20 to 1.8.0.u45.
- * http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8076152
- * 
- * The JVM crashes when hotspot compiling a method that uses an array consisting
- * of objects of a base class when different child classes is used as actual
- * instance objects AND when the array is constant (declared final). The crash
- * occurs during process of the aaload byte code.
- * 
- * This test and its corrections can be removed after the release of JDK
- * 1.8.0u60 if we choose to not support 1.8.0u20 - 1.8.0u45 inclusive.
- * 
- *
- * @since 8.2
- * 
- */
-@Category(UnitTest.class)
-public class Bug52289JUnitTest {
-
-  @Test
-  public void test() throws IOException, ClassNotFoundException {
-    // Iterate enough to cause JIT to compile
-    // javax.print.attribute.EnumSyntax::readResolve
-    for (int i = 0; i < 100_000; i++) {
-      // Must execute two or more subclasses with final static arrays of
-      // different types.
-      doEvictionAlgorithm();
-      doEvictionAction();
-    }
-  }
-
-  protected void doEvictionAlgorithm() throws IOException, ClassNotFoundException {
-    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    final ObjectOutputStream oos = new ObjectOutputStream(baos);
-    oos.writeObject(EvictionAlgorithm.NONE);
-    oos.close();
-
-    final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    final ObjectInputStream ois = new ObjectInputStream(bais);
-    ois.readObject();
-    ois.close();
-  }
-
-  protected void doEvictionAction() throws IOException, ClassNotFoundException {
-    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    final ObjectOutputStream oos = new ObjectOutputStream(baos);
-    oos.writeObject(EvictionAction.NONE);
-    oos.close();
-
-    final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    final ObjectInputStream ois = new ObjectInputStream(bais);
-    ois.readObject();
-    ois.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
index bbc2839..3761966 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
@@ -4461,7 +4461,7 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
           };
           Wait.waitForCriterion(ev, maxTime, 200, true);
 //            Set prox = ccn.getClientProxies();
-//            assertEquals(1, prox.size());
+//            assertIndexDetailsEquals(1, prox.size());
 //            for (Iterator cpi = prox.iterator(); cpi.hasNext(); ) {
 //              CacheClientProxy ccp = (CacheClientProxy) cpi.next();
 //              start = System.currentTimeMillis();
@@ -5447,7 +5447,7 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
           assertNull(r.getEntry(key1));
           r.registerInterest(key1);
           assertNotNull(r.getEntry(key1));
-          assertEquals(val1, r.getEntry(key1).getValue());
+          assertIndexDetailsEquals(val1, r.getEntry(key1).getValue());
           r.registerInterest(key2);
           assertNull(r.getEntry(key2));
         }
@@ -5464,7 +5464,7 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
           factory.setCacheListener(new CertifiableTestCacheListener(getLogWriter()));
           Region r = createRootRegion(name, factory.create());
           assertNull(r.getEntry(key1));
-          assertEquals(val1, r.get(key1));
+          assertIndexDetailsEquals(val1, r.get(key1));
           assertNull(r.getEntry(key2));
           r.registerInterest(key2);
           assertNull(r.getEntry(key2));
@@ -5490,11 +5490,11 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
 
           ctl.waitForUpdated(key1);
           assertNotNull(r.getEntry(key1));
-          assertEquals(val2, r.getEntry(key1).getValue()); // new value should have been pushed
+          assertIndexDetailsEquals(val2, r.getEntry(key1).getValue()); // new value should have been pushed
 
           ctl.waitForCreated(key2);
           assertNotNull(r.getEntry(key2)); // new entry should have been pushed
-          assertEquals(val2, r.getEntry(key2).getValue());
+          assertIndexDetailsEquals(val2, r.getEntry(key2).getValue());
         }
       });
 
@@ -5507,11 +5507,11 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
           ctl.waitForInvalidated(key1);
           assertNotNull(r.getEntry(key1));
           assertNull(r.getEntry(key1).getValue()); // Invalidate should have been pushed
-          assertEquals(val2, r.get(key1)); // New value should be fetched
+          assertIndexDetailsEquals(val2, r.get(key1)); // New value should be fetched
 
           assertNull(r.getEntry(key2));
           // assertNull(r.getEntry(key2).getValue());
-          assertEquals(val2, r.get(key2)); // New entry should be fetched
+          assertIndexDetailsEquals(val2, r.get(key2)); // New entry should be fetched
         }
       });
       tearDown();
@@ -5740,7 +5740,7 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
             }
           };
           Wait.waitForCriterion(ev, 10 * 1000, 200, true);
-//          assertEquals(3, region.size());
+//          assertIndexDetailsEquals(3, region.size());
           assertTrue(region.containsKey("k1"));
           assertTrue(region.containsKey("k2"));
           assertTrue(region.containsKey("k3"));
@@ -5857,7 +5857,7 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
             }
           };
           Wait.waitForCriterion(ev, 10 * 1000, 200, true);
-//          assertEquals(3, region.size());
+//          assertIndexDetailsEquals(3, region.size());
           assertTrue(region.containsKey("k1"));
           assertTrue(region.containsKey("k2"));
           assertTrue(region.containsKey("k3"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/FinalStaticArrayShouldNotCauseSegFaultTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/FinalStaticArrayShouldNotCauseSegFaultTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/FinalStaticArrayShouldNotCauseSegFaultTest.java
new file mode 100644
index 0000000..4f0335a
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/FinalStaticArrayShouldNotCauseSegFaultTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Test case for Trac <a
+ * href="https://svn.gemstone.com/trac/gemfire/ticket/52289">#52289</a>.
+ * 
+ * Asserts fixes for bug JDK-8076152 in JDK 1.8.0u20 to 1.8.0.u45.
+ * http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8076152
+ * 
+ * The JVM crashes when hotspot compiling a method that uses an array consisting
+ * of objects of a base class when different child classes is used as actual
+ * instance objects AND when the array is constant (declared final). The crash
+ * occurs during process of the aaload byte code.
+ * 
+ * This test and its corrections can be removed after the release of JDK
+ * 1.8.0u60 if we choose to not support 1.8.0u20 - 1.8.0u45 inclusive.
+ * 
+ * @since 8.2
+ */
+@Category(UnitTest.class)
+public class FinalStaticArrayShouldNotCauseSegFaultTest {
+
+  @Test
+  public void test() throws IOException, ClassNotFoundException {
+    // Iterate enough to cause JIT to compile
+    // javax.print.attribute.EnumSyntax::readResolve
+    for (int i = 0; i < 100_000; i++) {
+      // Must execute two or more subclasses with final static arrays of
+      // different types.
+      doEvictionAlgorithm();
+      doEvictionAction();
+    }
+  }
+
+  protected void doEvictionAlgorithm() throws IOException, ClassNotFoundException {
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    final ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(EvictionAlgorithm.NONE);
+    oos.close();
+
+    final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    final ObjectInputStream ois = new ObjectInputStream(bais);
+    ois.readObject();
+    ois.close();
+  }
+
+  protected void doEvictionAction() throws IOException, ClassNotFoundException {
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    final ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(EvictionAction.NONE);
+    oos.close();
+
+    final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    final ObjectInputStream ois = new ObjectInputStream(bais);
+    ois.readObject();
+    ois.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/MembershipAttributesAreSerializableTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/MembershipAttributesAreSerializableTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/MembershipAttributesAreSerializableTest.java
new file mode 100644
index 0000000..f0e6479
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/MembershipAttributesAreSerializableTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Tests MembershipAttributes and SubscriptionAttributes to  make sure they are Serializable
+ */
+@Category(UnitTest.class)
+public class MembershipAttributesAreSerializableTest {
+  
+  /**
+   * Assert that MembershipAttributes are serializable.
+   */
+  @Test
+  public void testMembershipAttributesAreSerializable() throws Exception {
+    String[] roles = {"a", "b", "c"};
+    MembershipAttributes outMA = new MembershipAttributes(roles);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(1000);
+    ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(outMA);
+    
+    byte[] data = baos.toByteArray();
+    
+    ByteArrayInputStream bais = new ByteArrayInputStream(data);
+    ObjectInputStream ois = new ObjectInputStream(bais);
+    MembershipAttributes inMA = (MembershipAttributes) ois.readObject();
+    assertEquals(outMA, inMA);
+  }
+
+  /**
+   * Assert that SubscriptionAttributes are serializable.
+   */
+  @Test
+  public void testSubscriptionAttributesAreSerializable() throws Exception {
+    SubscriptionAttributes outSA = new SubscriptionAttributes();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(1000);
+    ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(outSA);
+    
+    byte[] data = baos.toByteArray();
+    
+    ByteArrayInputStream bais = new ByteArrayInputStream(data);
+    ObjectInputStream ois = new ObjectInputStream(bais);
+    SubscriptionAttributes inSA = (SubscriptionAttributes) ois.readObject();
+    assertEquals(outSA, inSA);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/OperationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/OperationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/OperationJUnitTest.java
index 4f23a89..f4bb936 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/OperationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/OperationJUnitTest.java
@@ -16,33 +16,20 @@
  */
 package com.gemstone.gemfire.cache;
 
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class OperationJUnitTest extends TestCase {
-  public OperationJUnitTest(String name) {
-    super(name);
-  }
-
-  public OperationJUnitTest() {
-    // TODO Auto-generated constructor stub
-  }
-
-  protected void setUp() throws Exception {
-    super.setUp();
-  }
+public class OperationJUnitTest {
 
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
   /**
    * Check CREATE Operation.
    */
+  @Test
   public void testCREATE() {
     Operation op = Operation.CREATE;
     assertTrue(op.isCreate());
@@ -64,9 +51,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check PUTALL_CREATE Operation.
    */
+  @Test
   public void testPUTALL_CREATE() {
     Operation op = Operation.PUTALL_CREATE;
     assertTrue(op.isCreate());
@@ -88,9 +77,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check SEARCH_CREATE Operation.
    */
+  @Test
   public void testSEARCH_CREATE() {
     Operation op = Operation.SEARCH_CREATE;
     assertTrue(op.isCreate());
@@ -112,9 +103,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check LOCAL_LOAD_CREATE Operation.
    */
+  @Test
   public void testLOCAL_LOAD_CREATE() {
     Operation op = Operation.LOCAL_LOAD_CREATE;
     assertTrue(op.isCreate());
@@ -136,9 +129,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check NET_LOAD_CREATE Operation.
    */
+  @Test
   public void testNET_LOAD_CREATE() {
     Operation op = Operation.NET_LOAD_CREATE;
     assertTrue(op.isCreate());
@@ -160,9 +155,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check UPDATE Operation.
    */
+  @Test
   public void testUPDATE() {
     Operation op = Operation.UPDATE;
     assertFalse(op.isCreate());
@@ -184,9 +181,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check PUTALL_UPDATE Operation.
    */
+  @Test
   public void testPUTALL_UPDATE() {
     Operation op = Operation.PUTALL_UPDATE;
     assertFalse(op.isCreate());
@@ -208,9 +207,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check SEARCH_UPDATE Operation.
    */
+  @Test
   public void testSEARCH_UPDATE() {
     Operation op = Operation.SEARCH_UPDATE;
     assertFalse(op.isCreate());
@@ -232,9 +233,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check LOCAL_LOAD_UPDATE Operation.
    */
+  @Test
   public void testLOCAL_LOAD_UPDATE() {
     Operation op = Operation.LOCAL_LOAD_UPDATE;
     assertFalse(op.isCreate());
@@ -256,9 +259,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check NET_LOAD_UPDATE Operation.
    */
+  @Test
   public void testNET_LOAD_UPDATE() {
     Operation op = Operation.NET_LOAD_UPDATE;
     assertFalse(op.isCreate());
@@ -280,9 +285,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check INVALIDATE Operation.
    */
+  @Test
   public void testINVALIDATE() {
     Operation op = Operation.INVALIDATE;
     assertFalse(op.isCreate());
@@ -304,9 +311,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check LOCAL_INVALIDATE Operation.
    */
+  @Test
   public void testLOCAL_INVALIDATE() {
     Operation op = Operation.LOCAL_INVALIDATE;
     assertFalse(op.isCreate());
@@ -328,9 +337,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check DESTROY Operation.
    */
+  @Test
   public void testDESTROY() {
     Operation op = Operation.DESTROY;
     assertFalse(op.isCreate());
@@ -352,9 +363,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REMOVEALL Operation.
    */
+  @Test
   public void testREMOVEALL() {
     Operation op = Operation.REMOVEALL_DESTROY;
     assertFalse(op.isCreate());
@@ -377,9 +390,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check LOCAL_DESTROY Operation.
    */
+  @Test
   public void testLOCAL_DESTROY() {
     Operation op = Operation.LOCAL_DESTROY;
     assertFalse(op.isCreate());
@@ -401,9 +416,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check EVICT_DESTROY Operation.
    */
+  @Test
   public void testEVICT_DESTROY() {
     Operation op = Operation.EVICT_DESTROY;
     assertFalse(op.isCreate());
@@ -425,9 +442,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_LOAD_SNAPSHOT Operation.
    */
+  @Test
   public void testREGION_LOAD_SNAPSHOT() {
     Operation op = Operation.REGION_LOAD_SNAPSHOT;
     assertFalse(op.isCreate());
@@ -449,9 +468,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_LOCAL_DESTROY Operation.
    */
+  @Test
   public void testREGION_LOCAL_DESTROY() {
     Operation op = Operation.REGION_LOCAL_DESTROY;
     assertFalse(op.isCreate());
@@ -473,9 +494,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_CREATE Operation.
    */
+  @Test
   public void testREGION_CREATE() {
     Operation op = Operation.REGION_CREATE;
     assertFalse(op.isCreate());
@@ -497,9 +520,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_CLOSE Operation.
    */
+  @Test
   public void testREGION_CLOSE() {
     Operation op = Operation.REGION_CLOSE;
     assertFalse(op.isCreate());
@@ -521,9 +546,11 @@ public class OperationJUnitTest extends TestCase {
     assertTrue(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_DESTROY Operation.
    */
+  @Test
   public void testREGION_DESTROY() {
     Operation op = Operation.REGION_DESTROY;
     assertFalse(op.isCreate());
@@ -545,9 +572,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check EXPIRE_DESTROY Operation.
    */
+  @Test
   public void testEXPIRE_DESTROY() {
     Operation op = Operation.EXPIRE_DESTROY;
     assertFalse(op.isCreate());
@@ -569,9 +598,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check EXPIRE_LOCAL_DESTROY Operation.
    */
+  @Test
   public void testEXPIRE_LOCAL_DESTROY() {
     Operation op = Operation.EXPIRE_LOCAL_DESTROY;
     assertFalse(op.isCreate());
@@ -593,9 +624,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check EXPIRE_INVALIDATE Operation.
    */
+  @Test
   public void testEXPIRE_INVALIDATE() {
     Operation op = Operation.EXPIRE_INVALIDATE;
     assertFalse(op.isCreate());
@@ -617,9 +650,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check EXPIRE_LOCAL_INVALIDATE Operation.
    */
+  @Test
   public void testEXPIRE_LOCAL_INVALIDATE() {
     Operation op = Operation.EXPIRE_LOCAL_INVALIDATE;
     assertFalse(op.isCreate());
@@ -641,9 +676,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_EXPIRE_DESTROY Operation.
    */
+  @Test
   public void testREGION_EXPIRE_DESTROY() {
     Operation op = Operation.REGION_EXPIRE_DESTROY;
     assertFalse(op.isCreate());
@@ -665,9 +702,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_EXPIRE_LOCAL_DESTROY Operation.
    */
+  @Test
   public void testREGION_EXPIRE_LOCAL_DESTROY() {
     Operation op = Operation.REGION_EXPIRE_LOCAL_DESTROY;
     assertFalse(op.isCreate());
@@ -689,9 +728,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_EXPIRE_INVALIDATE Operation.
    */
+  @Test
   public void testREGION_EXPIRE_INVALIDATE() {
     Operation op = Operation.REGION_EXPIRE_INVALIDATE;
     assertFalse(op.isCreate());
@@ -713,9 +754,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_EXPIRE_LOCAL_INVALIDATE Operation.
    */
+  @Test
   public void testREGION_EXPIRE_LOCAL_INVALIDATE() {
     Operation op = Operation.REGION_EXPIRE_LOCAL_INVALIDATE;
     assertFalse(op.isCreate());
@@ -737,9 +780,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_LOCAL_INVALIDATE Operation.
    */
+  @Test
   public void testREGION_LOCAL_INVALIDATE() {
     Operation op = Operation.REGION_LOCAL_INVALIDATE;
     assertFalse(op.isCreate());
@@ -761,9 +806,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_INVALIDATE Operation.
    */
+  @Test
   public void testREGION_INVALIDATE() {
     Operation op = Operation.REGION_INVALIDATE;
     assertFalse(op.isCreate());
@@ -777,17 +824,19 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isLocal());
     assertTrue(op.isDistributed());
     assertFalse(op.isEntry());
+    assertFalse(op.isClose());
     assertFalse(op.isExpiration());
     assertFalse(op.isLocalLoad());
     assertFalse(op.isNetLoad());
     assertFalse(op.isLoad());
     assertFalse(op.isNetSearch());
-    assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_CLEAR Operation.
    */
+  @Test
   public void testREGION_CLEAR() {
     Operation op = Operation.REGION_CLEAR;
     assertFalse(op.isCreate());
@@ -809,9 +858,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertTrue(op.isClear());
   }
+
   /**
    * Check REGION_LOCAL_CLEAR Operation.
    */
+  @Test
   public void testREGION_LOCAL_CLEAR() {
     Operation op = Operation.REGION_LOCAL_CLEAR;
     assertFalse(op.isCreate());
@@ -833,9 +884,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertTrue(op.isClear());
   }
+
   /**
    * Check CACHE_CREATE Operation
    */
+  @Test
   public void testCACHE_CREATE() {
     Operation op = Operation.CACHE_CREATE;
     assertFalse(op.isCreate());
@@ -857,9 +910,11 @@ public class OperationJUnitTest extends TestCase {
     assertFalse(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check CACHE_CLOSE Operation.
    */
+  @Test
   public void testCACHE_CLOSE() {
     Operation op = Operation.CACHE_CLOSE;
     assertFalse(op.isCreate());
@@ -881,9 +936,11 @@ public class OperationJUnitTest extends TestCase {
     assertTrue(op.isClose());
     assertFalse(op.isClear());
   }
+
   /**
    * Check REGION_REINITIALIZE Operation.
    */
+  @Test
   public void testREGION_REINITIALIZE() {
     Operation op = Operation.REGION_REINITIALIZE;
     assertFalse(op.isCreate());
@@ -909,6 +966,7 @@ public class OperationJUnitTest extends TestCase {
   /**
    * Check UPDATE_VERSION Operation.
    */
+  @Test
   public void testUPDATE_VERSION() {
     Operation op = Operation.UPDATE_VERSION_STAMP;
     assertFalse(op.isCreate());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/RoleExceptionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/RoleExceptionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/RoleExceptionJUnitTest.java
index 42c146f..61d550b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/RoleExceptionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/RoleExceptionJUnitTest.java
@@ -16,35 +16,27 @@
  */
 package com.gemstone.gemfire.cache;
 
+import static org.junit.Assert.*;
+
 import java.io.*;
 import java.util.*;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.distributed.internal.membership.InternalRole;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/** Tests the subclasses of RoleException to make sure they are Serializable */
+/**
+ * Tests the subclasses of RoleException to make sure they are Serializable
+ */
 @Category(UnitTest.class)
-public class RoleExceptionJUnitTest extends TestCase {
+public class RoleExceptionJUnitTest {
   
-  public RoleExceptionJUnitTest(String name) {
-    super(name);
-  }
-
-  protected void setUp() throws Exception {
-    super.setUp();
-  }
-
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
   /**
    * Assert that RegionAccessException is serializable.
    */
+  @Test
   public void testRegionAccessExceptionIsSerializable() throws Exception {
     RegionAccessException out = createRegionAccessException();
     ByteArrayOutputStream baos = new ByteArrayOutputStream(100);
@@ -65,6 +57,7 @@ public class RoleExceptionJUnitTest extends TestCase {
   /**
    * Assert that RegionDistributionException is serializable.
    */
+  @Test
   public void testRegionDistributionExceptionIsSerializable() throws Exception {
     RegionDistributionException out = createRegionDistributionException();
     ByteArrayOutputStream baos = new ByteArrayOutputStream(100);
@@ -85,6 +78,7 @@ public class RoleExceptionJUnitTest extends TestCase {
   /**
    * Assert that CommitDistributionException is serializable.
    */
+  @Test
   public void testCommitDistributionExceptionIsSerializable() throws Exception {
     String s = "MyString";
     Set outExceptions = new HashSet();



[09/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderTest.java
new file mode 100755
index 0000000..d9ad65a
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderTest.java
@@ -0,0 +1,890 @@
+/*
+ * 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.internal;
+
+import static org.junit.Assert.*;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Enumeration;
+import java.util.Vector;
+
+import org.apache.bcel.Constants;
+import org.apache.bcel.classfile.JavaClass;
+import org.apache.bcel.generic.ClassGen;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link ClassPathLoader}.
+ * 
+ * @since 6.5.1.4
+ */
+@Category(UnitTest.class)
+public class ClassPathLoaderTest {
+
+  private static final int GENERATED_CLASS_BYTES_COUNT = 362;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(ClassPathLoader.EXCLUDE_TCCL_PROPERTY, "false");
+  }
+
+  /**
+   * Verifies that {@link ClassPathLoader#getLatest()} is always initialized and returns a <tt>ClassPathLoader</tt>
+   * instance.
+   */
+  @Test
+  public void testLatestExists() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testLatestExists");
+
+    assertNotNull(ClassPathLoader.getLatest());
+  }
+
+  /**
+   * Verifies that {@link ClassPathLoader#getLatest()} throws <tt>ClassNotFoundException</tt> when class does not exist.
+   */
+  @Test
+  public void testForNameThrowsClassNotFoundException() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testForNameThrowsClassNotFoundException");
+
+    try {
+      String classToLoad = "com.nowhere.DoesNotExist";
+      ClassPathLoader.getLatest().forName(classToLoad);
+      fail();
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+  }
+
+  /**
+   * Verifies that {@link ClassPathLoader#getLatest()} finds and loads class via
+   * <tt>Class.forName(String, boolean, ClassLoader)</tt> when class does exist.
+   */
+  @Test
+  public void testForName() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testForName");
+
+    String classToLoad = "com.gemstone.gemfire.internal.classpathloaderjunittest.DoesExist";
+    Class<?> clazz = ClassPathLoader.getLatest().forName(classToLoad);
+    assertNotNull(clazz);
+  }
+
+  /**
+   * Verifies that {@link ClassPathLoader#getLatest()} can actually <tt>getResource</tt> when it exists.
+   */
+  @Test
+  public void testGetResource() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResource");
+
+    String resourceToGet = "com/gemstone/gemfire/internal/classpathloaderjunittest/DoesExist.class";
+    URL url = ClassPathLoader.getLatest().getResource(resourceToGet);
+    assertNotNull(url);
+
+    InputStream is = url != null ? url.openStream() : null;
+    assertNotNull(is);
+
+    int totalBytesRead = 0;
+    byte[] input = new byte[256];
+
+    BufferedInputStream bis = new BufferedInputStream(is);
+    for (int bytesRead = bis.read(input); bytesRead > -1;) {
+      totalBytesRead += bytesRead;
+      bytesRead = bis.read(input);
+    }
+    bis.close();
+
+    // if the following fails then maybe javac changed and DoesExist.class
+    // contains other than 374 bytes of data... consider updating this test
+    assertEquals(GENERATED_CLASS_BYTES_COUNT, totalBytesRead);
+  }
+
+  /**
+   * Verifies that {@link ClassPathLoader#getLatest()} can actually <tt>getResources</tt> when it exists.
+   */
+  @Test
+  public void testGetResources() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResources");
+
+    String resourceToGet = "com/gemstone/gemfire/internal/classpathloaderjunittest/DoesExist.class";
+    Enumeration<URL> urls = ClassPathLoader.getLatest().getResources(resourceToGet);
+    assertNotNull(urls);
+    assertTrue(urls.hasMoreElements());
+
+    URL url = urls.nextElement();
+    InputStream is = url != null ? url.openStream() : null;
+    assertNotNull(is);
+
+    int totalBytesRead = 0;
+    byte[] input = new byte[256];
+
+    BufferedInputStream bis = new BufferedInputStream(is);
+    for (int bytesRead = bis.read(input); bytesRead > -1;) {
+      totalBytesRead += bytesRead;
+      bytesRead = bis.read(input);
+    }
+    bis.close();
+
+    // if the following fails then maybe javac changed and DoesExist.class
+    // contains other than 374 bytes of data... consider updating this test
+    assertEquals(GENERATED_CLASS_BYTES_COUNT, totalBytesRead);
+  }
+
+  /**
+   * Verifies that {@link ClassPathLoader#getLatest()} can actually <tt>getResourceAsStream</tt> when it exists.
+   */
+  @Test
+  public void testGetResourceAsStream() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceAsStream");
+
+    String resourceToGet = "com/gemstone/gemfire/internal/classpathloaderjunittest/DoesExist.class";
+    InputStream is = ClassPathLoader.getLatest().getResourceAsStream(resourceToGet);
+    assertNotNull(is);
+
+    int totalBytesRead = 0;
+    byte[] input = new byte[256];
+
+    BufferedInputStream bis = new BufferedInputStream(is);
+    for (int bytesRead = bis.read(input); bytesRead > -1;) {
+      totalBytesRead += bytesRead;
+      bytesRead = bis.read(input);
+    }
+    bis.close();
+
+    // if the following fails then maybe javac changed and DoesExist.class
+    // contains other than 374 bytes of data... consider updating this test
+    assertEquals(GENERATED_CLASS_BYTES_COUNT, totalBytesRead);
+  }
+
+  /**
+   * Verifies that the {@link GeneratingClassLoader} works and always generates the named class. This is a control which
+   * ensures that tests depending on <tt>GeneratingClassLoader</tt> are valid.
+   */
+  @Test
+  public void testGeneratingClassLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGeneratingClassLoader");
+
+    ClassLoader gcl = new GeneratingClassLoader();
+    String classToLoad = "com.nowhere.TestGeneratingClassLoader";
+
+    Class<?> clazz = gcl.loadClass(classToLoad);
+    assertNotNull(clazz);
+    assertEquals(classToLoad, clazz.getName());
+
+    Object obj = clazz.newInstance();
+    assertEquals(clazz.getName(), obj.getClass().getName());
+
+    try {
+      Class.forName(classToLoad);
+      fail("Should have thrown ClassNotFoundException");
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+
+    Class<?> clazzForName = Class.forName(classToLoad, true, gcl);
+    assertNotNull(clazzForName);
+    assertEquals(clazz, clazzForName);
+
+    Object objForName = clazzForName.newInstance();
+    assertEquals(classToLoad, objForName.getClass().getName());
+  }
+
+  /**
+   * Verifies that custom loader is used to load class.
+   */
+  @Test
+  public void testForNameWithCustomLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testForNameWithCustomLoader");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+
+    String classToLoad = "com.nowhere.TestForNameWithCustomLoader";
+    Class<?> clazz = dcl.forName(classToLoad);
+    assertNotNull(clazz);
+    assertEquals(classToLoad, clazz.getName());
+
+    Object obj = clazz.newInstance();
+    assertEquals(classToLoad, obj.getClass().getName());
+  }
+
+  /**
+   * Verifies that {@link Class#forName(String, boolean, ClassLoader)} used with {@link ClassPathLoader} works as
+   * expected with named object arrays, while {@link ClassLoader#loadClass(String)} throws ClassNotFoundException for
+   * named object arrays.
+   */
+  @Test
+  public void testForNameWithObjectArray() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testForNameWithObjectArray");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+
+    String classToLoad = "[Ljava.lang.String;";
+    Class<?> clazz = null;
+    clazz = dcl.forName(classToLoad);
+    assertEquals(classToLoad, clazz.getName());
+  }
+
+  /**
+   * Verifies that TCCL finds the class when {@link Class#forName(String, boolean, ClassLoader)} uses
+   * {@link ClassPathLoader}.
+   */
+  @Test
+  public void testForNameWithTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testForNameWithTCCL");
+
+    final ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    final String classToLoad = "com.nowhere.TestForNameWithTCCL";
+
+    try {
+      dcl.forName(classToLoad);
+      fail("Should have thrown ClassNotFoundException");
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // ensure that TCCL is only CL that can find this class
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      Class<?> clazz = dcl.forName(classToLoad);
+      assertNotNull(clazz);
+      Object instance = clazz.newInstance();
+      assertNotNull(instance);
+      assertEquals(classToLoad, instance.getClass().getName());
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+
+    try {
+      dcl.forName(classToLoad);
+      fail("Should have thrown ClassNotFoundException");
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+
+  }
+
+  /**
+   * Verifies that the {@link NullClassLoader} works and never finds the named class. This is a control which ensures
+   * that tests depending on <tt>NullClassLoader</tt> are valid.
+   */
+  @Test
+  public void testNullClassLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testNullClassLoader");
+
+    ClassLoader cl = new NullClassLoader();
+    String classToLoad = "java.lang.String";
+
+    try {
+      Class.forName(classToLoad, true, cl);
+      fail();
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+
+    String resourceToGet = "java/lang/String.class";
+
+    URL url = cl.getResource(resourceToGet);
+    assertNull(url);
+
+    InputStream is = cl.getResourceAsStream(resourceToGet);
+    assertNull(is);
+  }
+
+  /**
+   * Verifies that the {@link SimpleClassLoader} works and finds classes that the parent can find. This is a control
+   * which ensures that tests depending on <tt>SimpleClassLoader</tt> are valid.
+   */
+  @Test
+  public void testSimpleClassLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testSimpleClassLoader");
+
+    ClassLoader cl = new SimpleClassLoader(getClass().getClassLoader());
+    String classToLoad = "java.lang.String";
+
+    Class<?> clazz = Class.forName(classToLoad, true, cl);
+    assertNotNull(clazz);
+
+    String resourceToGet = "java/lang/String.class";
+
+    URL url = cl.getResource(resourceToGet);
+    assertNotNull(url);
+
+    InputStream is = cl.getResourceAsStream(resourceToGet);
+    assertNotNull(is);
+  }
+
+  /**
+   * Verifies that the {@link BrokenClassLoader} is broken and always throws errors. This is a control which ensures
+   * that tests depending on <tt>BrokenClassLoader</tt> are valid.
+   */
+  @Test
+  public void testBrokenClassLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testBrokenClassLoader");
+
+    ClassLoader cl = new BrokenClassLoader();
+
+    String classToLoad = "java.lang.String";
+    try {
+      Class.forName(classToLoad, true, cl);
+      fail();
+    } catch (ClassNotFoundException e) {
+      throw e;
+    } catch (BrokenError expected) {
+      // Expected
+    }
+
+    String resourceToGet = "java/lang/String.class";
+    try {
+      cl.getResource(resourceToGet);
+      fail();
+    } catch (BrokenError expected) {
+      // Expected
+    }
+    try {
+      cl.getResourceAsStream(resourceToGet);
+      fail();
+    } catch (BrokenError expected) {
+      // Expected
+    }
+  }
+
+  /**
+   * Verifies that the {@link BrokenClassLoader} is broken and always throws errors even when used as a TCCL from
+   * {@link ClassPathLoader}. This is primarily a control which ensures that tests depending on
+   * <tt>BrokenClassLoader</tt> are valid, but it also verifies that TCCL is included by default by
+   * <tt>ClassPathLoader</tt>.
+   */
+  @Test
+  public void testBrokenTCCLThrowsErrors() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testBrokenTCCLThrowsErrors");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl.addOrReplace(new NullClassLoader());
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set the TCCL to throw errors
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      String classToLoad = "java.lang.String";
+      try {
+        dcl.forName(classToLoad);
+        fail();
+      } catch (ClassNotFoundException e) {
+        throw e;
+      } catch (BrokenError expected) {
+        // Expected
+      }
+
+      String resourceToGet = "java/lang/String.class";
+      try {
+        dcl.getResource(resourceToGet);
+        fail();
+      } catch (BrokenError expected) {
+        // Expected
+      }
+
+      try {
+        dcl.getResourceAsStream(resourceToGet);
+        fail();
+      } catch (BrokenError expected) {
+        // Expected
+      }
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that the class classloader or system classloader will find the class or resource. Parent is a
+   * {@link NullClassLoader} while the TCCL is an excluded {@link BrokenClassLoader}.
+   */
+  @Test
+  public void testEverythingWithDefaultLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testEverythingWithDefaultLoader");
+
+    // create DCL such that parent cannot find anything
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
+    dcl.addOrReplace(new NullClassLoader());
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set the TCCL to never find anything
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      String classToLoad = "java.lang.String";
+      Class<?> clazz = dcl.forName(classToLoad);
+      assertNotNull(clazz);
+
+      String resourceToGet = "java/lang/String.class";
+      URL url = dcl.getResource(resourceToGet);
+      assertNotNull(url);
+      InputStream is = dcl.getResourceAsStream(resourceToGet);
+      assertNotNull(is);
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that the 3rd custom loader will find the class. Parent cannot find it and TCCL is broken. This verifies
+   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
+   */
+  @Test
+  public void testForNameWithMultipleCustomLoaders() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testForNameWithMultipleCustomLoaders");
+
+    // create DCL such that the 3rd loader should find the class
+    // first custom loader becomes parent which won't find anything
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    final GeneratingClassLoader generatingClassLoader = new GeneratingClassLoader();
+    dcl = dcl.addOrReplace(generatingClassLoader);
+    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
+    dcl = dcl.addOrReplace(new NullClassLoader());
+
+    String classToLoad = "com.nowhere.TestForNameWithMultipleCustomLoaders";
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set TCCL to throw errors which makes sure we find before checking TCCL
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      Class<?> clazz = dcl.forName(classToLoad);
+      assertNotNull(clazz);
+      assertEquals(classToLoad, clazz.getName());
+      assertTrue("Class not loaded by a GeneratingClassLoader.", clazz.getClassLoader() instanceof GeneratingClassLoader);
+      assertEquals("Class not loaded by generatingClassLoader.", generatingClassLoader, clazz.getClassLoader());
+
+      Object obj = clazz.newInstance();
+      assertEquals(classToLoad, obj.getClass().getName());
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that setting <tt>excludeThreadContextClassLoader</tt> to true will indeed exclude the TCCL.
+   */
+  @Test
+  public void testExcludeTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testExcludeTCCL");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
+
+    String classToLoad = "com.nowhere.TestExcludeTCCL";
+
+    try {
+      dcl.forName(classToLoad);
+      fail("Should have thrown ClassNotFoundException");
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // ensure that TCCL is only CL that can find this class
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      dcl.forName(classToLoad);
+      fail("Should have thrown ClassNotFoundException");
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+
+  }
+
+
+  /**
+   * Verifies that <tt>getResource</tt> will skip TCCL if <tt>excludeThreadContextClassLoader</tt> has been set to true.
+   */
+  @Test
+  public void testGetResourceExcludeTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceExcludeTCCL");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
+
+    String resourceToGet = "com/nowhere/testGetResourceExcludeTCCL.rsc";
+    assertNull(dcl.getResource(resourceToGet));
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // ensure that TCCL is only CL that can find this resource
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      assertNull(dcl.getResource(resourceToGet));
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  /**
+   * Verifies that <tt>getResourceAsStream</tt> will skip TCCL if <tt>excludeThreadContextClassLoader</tt> has been set
+   * to true.
+   */
+  @Test
+  public void testGetResourceAsStreamExcludeTCCL() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testGetResourceAsStreamExcludeTCCL");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
+
+    String resourceToGet = "com/nowhere/testGetResourceAsStreamExcludeTCCL.rsc";
+    assertNull(dcl.getResourceAsStream(resourceToGet));
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // ensure that TCCL is only CL that can find this resource
+      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
+      assertNull(dcl.getResourceAsStream(resourceToGet));
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  @Test
+  public void testAddFindsLatestClassLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testAddFindsLatestClassLoader");
+
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    dcl = dcl.addOrReplace(new GeneratingClassLoader());
+
+    String classToLoad = "com.nowhere.TestAddFindsLatestClassLoader";
+    Class<?> clazz = dcl.forName(classToLoad);
+    assertNotNull(clazz);
+
+    dcl = dcl.addOrReplace(new BrokenClassLoader());
+
+    try {
+      dcl.forName(classToLoad);
+      fail();
+    } catch (BrokenError expected) {
+      // Expected
+    }
+  }
+
+  /**
+   * Verifies removing a ClassLoader.
+   */
+  @Test
+  public void testRemoveClassLoader() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testRemoveClassLoader");
+
+    GeneratingClassLoader genClassLoader = new GeneratingClassLoader();
+    ClassPathLoader cpl = ClassPathLoader.createWithDefaults(false);
+    cpl = cpl.addOrReplace(genClassLoader);
+
+    String classToLoad = "com.nowhere.TestRemoveClassLoader";
+    Class<?> clazz = cpl.forName(classToLoad);
+    assertNotNull(clazz);
+
+    cpl = cpl.remove(genClassLoader);
+
+    try {
+      clazz = cpl.forName(classToLoad);
+      fail();
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+  }
+
+  /**
+   * Verifies that a ClassLoader will be replaced when added more than once.
+   */
+  @Test
+  public void testClassLoaderReplace() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testClassLoaderReplace");
+
+    String class1ToLoad = "ClassA";
+    String class2ToLoad = "ClassB";
+
+    ClassPathLoader cpl = ClassPathLoader.createWithDefaults(false);
+    cpl = cpl.addOrReplace(new OneClassClassLoader(class1ToLoad));
+
+    try {
+      @SuppressWarnings("unused")
+      Class<?> clazz = cpl.forName(class1ToLoad);
+    } catch (ClassNotFoundException unexpected) {
+      fail();
+    }
+
+    try {
+      @SuppressWarnings("unused")
+      Class<?> clazz = cpl.forName(class2ToLoad);
+      fail();
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+
+    cpl = cpl.addOrReplace(new OneClassClassLoader(class2ToLoad));
+    try {
+      @SuppressWarnings("unused")
+      Class<?> clazz = cpl.forName(class2ToLoad);
+    } catch (ClassNotFoundException unexpected) {
+      fail();
+    }
+
+    try {
+      @SuppressWarnings("unused")
+      Class<?> clazz = cpl.forName(class1ToLoad);
+      fail();
+    } catch (ClassNotFoundException expected) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testAsClassLoaderLoadClassWithMultipleCustomLoaders() throws Exception {
+    System.out.println("\nStarting ClassPathLoaderTest#testAsClassLoaderLoadClassWithMultipleCustomLoaders");
+
+    // create DCL such that the 3rd loader should find the class
+    // first custom loader becomes parent which won't find anything
+    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
+    final GeneratingClassLoader generatingClassLoader = new GeneratingClassLoader();
+    dcl = dcl.addOrReplace(generatingClassLoader);
+    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
+    dcl = dcl.addOrReplace(new NullClassLoader());
+
+    final String classToLoad = "com.nowhere.TestForNameWithMultipleCustomLoaders";
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    try {
+      // set TCCL to throw errors which makes sure we find before checking TCCL
+      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
+
+      final ClassLoader classLoader = dcl.asClassLoader();
+      final Class<?> clazz = classLoader.loadClass(classToLoad);
+      assertNotNull(clazz);
+      assertEquals(classToLoad, clazz.getName());
+      assertTrue(clazz.getClassLoader() instanceof GeneratingClassLoader);
+      assertEquals(generatingClassLoader, clazz.getClassLoader());
+
+      final Object obj = clazz.newInstance();
+      assertEquals(classToLoad, obj.getClass().getName());
+      
+      final Class<?> clazz2 = dcl.forName(classToLoad);
+      assertSame("Should load same class as calling classLoader.", clazz, clazz2);
+
+      final Class<?> clazz3 = Class.forName(classToLoad, true, classLoader);
+      assertSame("Should load same class as calling classLoader.", clazz, clazz3);
+
+    } finally {
+      Thread.currentThread().setContextClassLoader(cl);
+    }
+  }
+
+  private static void exploreClassLoaders() {
+    System.out.println("Thread.currentThread().getContextClassLoader()...");
+    exploreClassLoader(Thread.currentThread().getContextClassLoader(), 1);
+
+    System.out.println("class.getClassLoader()...");
+    exploreClassLoader(ClassPathLoaderTest.class.getClassLoader(), 1);
+
+    System.out.println("ClassLoader.getSystemClassLoader()...");
+    exploreClassLoader(ClassLoader.getSystemClassLoader(), 1);
+  }
+
+  private static void exploreClassLoader(ClassLoader cl, int indent) {
+    String prefix = "";
+    for (int i = 0; i < indent; i++) {
+      prefix += "\t";
+    }
+    System.out.println(prefix + "ClassLoader toString() = " + cl);
+
+    Class<?> clazz = cl.getClass();
+    System.out.println(prefix + "ClassLoader getClass().getName() = " + clazz.getName());
+    exploreClassLoaderSuperClass(prefix, clazz);
+
+    try {
+      URL[] urls = ((URLClassLoader) cl).getURLs();
+      StringBuilder sb = new StringBuilder(prefix).append("ClassLoader getURLs = [");
+      for (int i = 0; i < urls.length; i++) {
+        if (i > 0)
+          sb.append(", ");
+        sb.append(urls[i].toString());
+      }
+      sb.append("]");
+      System.out.println(sb.toString());
+    } catch (Exception e) {
+      System.out.println(prefix + "ClassLoader is not a URLClassLoader");
+    }
+
+    ClassLoader parent = cl.getParent();
+    if (parent != null) {
+      System.out.println(prefix + "ClassLoader has parent...");
+      exploreClassLoader(parent, ++indent);
+    }
+  }
+
+  private static void exploreClassLoaderSuperClass(String prefix, Class<?> clazz) {
+    Class<?> superClazz = clazz.getSuperclass();
+    if (superClazz != null) {
+      System.out.println(prefix + "                       getSuperclass().getName() = " + superClazz.getName());
+      exploreClassLoaderSuperClass(prefix, superClazz);
+    }
+  }
+
+  /**
+   * Custom class loader which will never find any class or resource.
+   */
+  static class NullClassLoader extends ClassLoader {
+    public NullClassLoader() {
+      super(null); // no parent!!
+    }
+
+    @Override
+    public Class<?> loadClass(String name) throws ClassNotFoundException {
+      throw new ClassNotFoundException(name);
+    }
+
+    @Override
+    public URL getResource(String name) {
+      return null;
+    }
+  }
+
+  /**
+   * Custom class loader which will find anything the parent can find.
+   */
+  static class SimpleClassLoader extends ClassLoader {
+    public SimpleClassLoader(ClassLoader parent) {
+      super(parent);
+    }
+  }
+
+  /**
+   * Custom class loader which is broken and always throws errors.
+   */
+  static class BrokenClassLoader extends ClassLoader {
+    public BrokenClassLoader() {
+      super(null); // no parent!!
+    }
+
+    @Override
+    public Class<?> loadClass(String name) throws ClassNotFoundException {
+      throw new BrokenError();
+    }
+
+    @Override
+    public URL getResource(String name) {
+      throw new BrokenError();
+    }
+  }
+
+  /**
+   * Custom class loader which uses BCEL to always dynamically generate a class for any class name it tries to load.
+   */
+  static class GeneratingClassLoader extends ClassLoader {
+
+    /**
+     * Currently unused but potentially useful for some future test. This causes this loader to only generate a class
+     * that the parent could not find.
+     *
+     * @param parent
+     *          the parent class loader to check with first
+     */
+    @SuppressWarnings("unused")
+    public GeneratingClassLoader(ClassLoader parent) {
+      super(parent);
+    }
+
+    /**
+     * Specifies no parent to ensure that this loader generates the named class.
+     */
+    public GeneratingClassLoader() {
+      super(null); // no parent!!
+    }
+
+    @Override
+    protected Class<?> findClass(String name) throws ClassNotFoundException {
+      ClassGen cg = new ClassGen(name, "java.lang.Object", "<generated>", Constants.ACC_PUBLIC | Constants.ACC_SUPER, null);
+      cg.addEmptyConstructor(Constants.ACC_PUBLIC);
+      JavaClass jClazz = cg.getJavaClass();
+      byte[] bytes = jClazz.getBytes();
+      return defineClass(jClazz.getClassName(), bytes, 0, bytes.length);
+    }
+
+    @Override
+    protected URL findResource(String name) {
+      URL url = null;
+      try {
+        url = getTempFile().getAbsoluteFile().toURI().toURL();
+        System.out.println("GeneratingClassLoader#findResource returning " + url);
+      } catch (IOException e) {
+      }
+      return url;
+    }
+
+    @Override
+    protected Enumeration<URL> findResources(String name) throws IOException {
+      URL url = null;
+      try {
+        url = getTempFile().getAbsoluteFile().toURI().toURL();
+        System.out.println("GeneratingClassLoader#findResources returning " + url);
+      } catch (IOException e) {
+      }
+      Vector<URL> urls = new Vector<URL>();
+      urls.add(url);
+      return urls.elements();
+    }
+
+    protected File getTempFile() {
+      return null;
+    }
+  }
+
+  static class OneClassClassLoader extends ClassLoader {
+
+    private final GeneratingClassLoader genClassLoader = new GeneratingClassLoader();
+    private String className;
+
+    public OneClassClassLoader(final String className) {
+      super(null); // no parent!!
+      this.className = className;
+    }
+    
+    @Override
+    public Class<?> findClass(String name) throws ClassNotFoundException {
+      if (!name.equals(className)) {
+        throw new ClassNotFoundException();
+      } else {
+        return this.genClassLoader.findClass(name);
+      }
+    }
+    
+    @Override
+    public boolean equals(final Object other) {
+      return (other instanceof OneClassClassLoader);
+    }
+  }
+  
+  @SuppressWarnings("serial")
+  static class BrokenError extends Error {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/ConfigSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ConfigSourceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ConfigSourceJUnitTest.java
index 73cd9a0..360fc8b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/ConfigSourceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/ConfigSourceJUnitTest.java
@@ -16,17 +16,16 @@
  */
 package com.gemstone.gemfire.internal;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.junit.Assert.*;
+
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * Created by jiliao on 2/2/16.
- */
 @Category(UnitTest.class)
 public class ConfigSourceJUnitTest {
+
   @Test
   public void testDescriptions() {
     ConfigSource cs = ConfigSource.api();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/CopyOnWriteHashSetJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/CopyOnWriteHashSetJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/CopyOnWriteHashSetJUnitTest.java
index 09df931..39e2058 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/CopyOnWriteHashSetJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/CopyOnWriteHashSetJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ObjectInputStream;
@@ -25,18 +27,15 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class CopyOnWriteHashSetJUnitTest extends TestCase {
-  
+public class CopyOnWriteHashSetJUnitTest {
+
+  @Test
   public void testSnapshot() {
     CopyOnWriteHashSet<String> set = new CopyOnWriteHashSet<String>();
     set.add("a");
@@ -46,9 +45,9 @@ public class CopyOnWriteHashSetJUnitTest extends TestCase {
     
     assertEquals(copy, snap);
   }
-  
+
+  @Test
   public void testAllMethods() throws Exception {
-    
     CopyOnWriteHashSet<String> set = new CopyOnWriteHashSet<String>();
     assertTrue(set.add("a"));
     assertFalse(set.add("a"));


[02/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/DataCommandJsonJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/DataCommandJsonJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/DataCommandJsonJUnitTest.java
index c95faee..f11afd0 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/DataCommandJsonJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/DataCommandJsonJUnitTest.java
@@ -16,26 +16,23 @@
  */
 package com.gemstone.gemfire.management.internal.cli;
 
+import static org.junit.Assert.*;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.management.internal.cli.dto.Car;
 import com.gemstone.gemfire.management.internal.cli.util.JsonUtil;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * 
- *
- */
 @Category(UnitTest.class)
-public class DataCommandJsonJUnitTest  extends TestCase{
+public class DataCommandJsonJUnitTest {
 
-  
+  @Test
   public void testCollectionTypesInJson(){    
     String json = "{'attributes':{'power':'90hp'},'make':'502.1825','model':'502.1825','colors':['red','white','blue'],'attributeSet':['red','white','blue'], 'attributeArray':['red','white','blue']}";
     Car car = (Car)JsonUtil.jsonToObject(json, Car.class);    
@@ -55,6 +52,5 @@ public class DataCommandJsonJUnitTest  extends TestCase{
     assertNotNull(car.getAttributeArray());
     assertTrue(car.getAttributeArray() instanceof String[]);
     assertEquals(3, car.getAttributeArray().length);
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
index 91dd853..1ff647e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
@@ -16,9 +16,7 @@
  */
 package com.gemstone.gemfire.management.internal.cli;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.lang.reflect.Method;
 import java.util.ArrayList;
@@ -53,7 +51,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 /**
  * GfshParserJUnitTest - Includes tests to check the parsing and auto-completion
  * capabilities of {@link GfshParser}
- *
  */
 @Category(UnitTest.class)
 public class GfshParserJUnitTest {
@@ -446,7 +443,6 @@ public class GfshParserJUnitTest {
 
   private void assertSimpleCompletionValues(List<String> expected,
       List<String> actual) {
-    // TODO Auto-generated method stub
     assertEquals("Check size", expected.size(), actual.size());
     for (int i = 0; i < expected.size(); i++) {
       assertEquals("Check completion value no." + i +". Expected("+expected.get(i)+") & Actual("+actual.get(i)+").", expected.get(i),
@@ -756,7 +752,6 @@ public class GfshParserJUnitTest {
 
   private void assertAdvancedCompletionValues(List<Completion> expected,
       List<Completion> actual) {
-    // TODO Auto-generated method stub
     assertEquals("Check size", expected.size(), actual.size());
     for (int i = 0; i < expected.size(); i++) {
       assertEquals("Check completion value no." + i+". Expected("+expected.get(i)+") & Actual("+actual.get(i)+").",
@@ -995,10 +990,12 @@ public class GfshParserJUnitTest {
     assertEquals(arguments[1], "this is a second argument");
   }
 
+  @Test
   public void testDefaultAvailabilityMessage() throws Exception {
     checkAvailabilityMessage(new AvailabilityCommands(), AvailabilityCommands.C2_NAME, AvailabilityCommands.C2_MSG_UNAVAILABLE, AvailabilityCommands.C2_PROP);
   }
 
+  @Test
   public void testCustomAvailabilityMessage() throws Exception {
     checkAvailabilityMessage(new AvailabilityCommands(), AvailabilityCommands.C1_NAME, AvailabilityCommands.C1_MSG_UNAVAILABLE, AvailabilityCommands.C1_PROP);
   }
@@ -1024,9 +1021,6 @@ public class GfshParserJUnitTest {
       System.setProperty(availabiltyBooleanProp, "true");
       parseResult = parser.parse(commandString);
       assertNotNull("ParseResult should not be null for available command.", parseResult);
-    } catch (CommandProcessingException e) {
-      fail("Command \""+commandString+"\" is expected to be available");
-      e.printStackTrace();
     } finally {
       System.clearProperty(availabiltyBooleanProp);
     }
@@ -1083,24 +1077,24 @@ public class GfshParserJUnitTest {
 
   static class SimpleConverter implements Converter<String> {
 
+    @Override
     public boolean supports(Class<?> type, String optionContext) {
-      // TODO Auto-generated method stub
       if (type.isAssignableFrom(String.class)) {
         return true;
       }
       return false;
     }
 
+    @Override
     public String convertFromText(String value, Class<?> targetType,
         String optionContext) {
-      // TODO Auto-generated method stub
       return value;
     }
 
+    @Override
     public boolean getAllPossibleValues(List<Completion> completions,
         Class<?> targetType, String existingData, String context,
         MethodTarget target) {
-      // TODO Auto-generated method stub
       if (context.equals(ARGUMENT1_CONTEXT)) {
         for (Completion completion : ARGUMENT1_COMPLETIONS) {
           completions.add(completion);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/annotations/CliArgumentJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/annotations/CliArgumentJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/annotations/CliArgumentJUnitTest.java
index f7b55f3..e38a9ca 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/annotations/CliArgumentJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/annotations/CliArgumentJUnitTest.java
@@ -16,20 +16,19 @@
  */
 package com.gemstone.gemfire.management.internal.cli.annotations;
 
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.management.internal.cli.annotation.CliArgument;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Includes tests for checking assignment of {@link CliArgument}
- * 
- * 
  */
 @Category(UnitTest.class)
-public class CliArgumentJUnitTest extends TestCase {
+public class CliArgumentJUnitTest {
 
   private static final String ARGUMENT_NAME = "ARGUMENT_NAME";
   private static final String ARGUMENT_HELP = "ARGUMENT_HELP";
@@ -43,169 +42,87 @@ public class CliArgumentJUnitTest extends TestCase {
   /**
    * Test for {@link CliArgument#name()}
    */
-  public void testName() {
-    try {
-      String name = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "defaultArgumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).name();
-      assertNotNull(name);
-      assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, name, ARGUMENT_NAME);
-      name = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "argumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-          .name();
-      assertNotNull(name);
-      assertEquals(MESSAGE_FOR_ARGUMENT, name, ARGUMENT_NAME);
-    } catch (SecurityException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (NoSuchMethodException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+  @Test
+  public void testName() throws Exception {
+    String name = ((CliArgument) (ArgumentTestingClass.class.getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0])).name();
+    assertNotNull(name);
+    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, name, ARGUMENT_NAME);
+    name = ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).name();
+    assertNotNull(name);
+    assertEquals(MESSAGE_FOR_ARGUMENT, name, ARGUMENT_NAME);
   }
 
   /**
    * Test for {@link CliArgument#help()}
    */
-  public void testHelp() {
-    try {
-      String help = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "defaultArgumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).help();
-      assertNotNull(help);
-      assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, help, "");
-      help = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "argumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-          .help();
-      assertNotNull(help);
-      assertEquals(MESSAGE_FOR_ARGUMENT, help, ARGUMENT_HELP);
-    } catch (SecurityException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (NoSuchMethodException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+  @Test
+  public void testHelp() throws Exception {
+    String help = ((CliArgument) (ArgumentTestingClass.class.getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0])).help();
+    assertNotNull(help);
+    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, help, "");
+    help = ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).help();
+    assertNotNull(help);
+    assertEquals(MESSAGE_FOR_ARGUMENT, help, ARGUMENT_HELP);
   }
 
   /**
    * Test for {@link CliArgument#mandatory()}
    */
-  public void testMandatory() {
-    try {
-      boolean mandatory = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "defaultArgumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).mandatory();
-      assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, mandatory, false);
-      mandatory = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "argumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-          .mandatory();
-      assertEquals(MESSAGE_FOR_ARGUMENT, mandatory, ARGUMENT_MANDATORY);
-    } catch (SecurityException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (NoSuchMethodException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+  @Test
+  public void testMandatory() throws Exception {
+    boolean mandatory = ((CliArgument) (ArgumentTestingClass.class.getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0])).mandatory();
+    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, mandatory, false);
+    mandatory = ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).mandatory();
+    assertEquals(MESSAGE_FOR_ARGUMENT, mandatory, ARGUMENT_MANDATORY);
   }
 
   /**
    * Test for {@link CliArgument#argumentContext()}
    */
-  public void testArgumentContext() {
-    try {
-      String argumentContext = ((CliArgument) (ArgumentTestingClass.class
-          .getMethod("defaultArgumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).argumentContext();
-      assertNotNull(argumentContext);
-      assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, argumentContext, "");
-      argumentContext = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "argumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-          .argumentContext();
-      assertNotNull(argumentContext);
-      assertEquals(MESSAGE_FOR_ARGUMENT, argumentContext, ARGUMENT_CONTEXT);
-    } catch (SecurityException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (NoSuchMethodException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+  @Test
+  public void testArgumentContext() throws Exception {
+    String argumentContext = ((CliArgument) (ArgumentTestingClass.class.getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0])).argumentContext();
+    assertNotNull(argumentContext);
+    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, argumentContext, "");
+    argumentContext = ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).argumentContext();
+    assertNotNull(argumentContext);
+    assertEquals(MESSAGE_FOR_ARGUMENT, argumentContext, ARGUMENT_CONTEXT);
   }
 
   /**
    * Test for {@link CliArgument#systemProvided()}
    */
-  public void testSystemProvided() {
-    try {
-      boolean systemProvided = ((CliArgument) (ArgumentTestingClass.class
-          .getMethod("defaultArgumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).systemProvided();
-      assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, systemProvided, false);
-      systemProvided = ((CliArgument) (ArgumentTestingClass.class.getMethod(
-          "argumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-          .systemProvided();
-      assertEquals(MESSAGE_FOR_ARGUMENT, systemProvided, SYSTEM_PROVIDED);
-    } catch (SecurityException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (NoSuchMethodException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+  @Test
+  public void testSystemProvided() throws Exception {
+    boolean systemProvided = ((CliArgument) (ArgumentTestingClass.class.getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0])).systemProvided();
+    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, systemProvided, false);
+    systemProvided = ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).systemProvided();
+    assertEquals(MESSAGE_FOR_ARGUMENT, systemProvided, SYSTEM_PROVIDED);
   }
 
   /**
    * Test for {@link CliArgument#unspecifiedDefaultValue()}
    */
-  public void testUnspecifiedDefaultValue() {
-    try {
-      String unspecifiedDefaultValue = ((CliArgument) (ArgumentTestingClass.class
-          .getMethod("defaultArgumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).unspecifiedDefaultValue();
-      assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, unspecifiedDefaultValue,
-          "__NULL__");
-      unspecifiedDefaultValue = ((CliArgument) (ArgumentTestingClass.class
-          .getMethod("argumentTestingMethod", String.class)
-          .getParameterAnnotations()[0][0])).unspecifiedDefaultValue();
-      assertEquals(MESSAGE_FOR_ARGUMENT, unspecifiedDefaultValue,
-          ARGUMENT_UNSPECIFIED_DEFAULT_VALUE);
-    } catch (SecurityException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (NoSuchMethodException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+  @Test
+  public void testUnspecifiedDefaultValue() throws Exception {
+    String unspecifiedDefaultValue = ((CliArgument) (ArgumentTestingClass.class.getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0])).unspecifiedDefaultValue();
+    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, unspecifiedDefaultValue, "__NULL__");
+    unspecifiedDefaultValue = ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).unspecifiedDefaultValue();
+    assertEquals(MESSAGE_FOR_ARGUMENT, unspecifiedDefaultValue, ARGUMENT_UNSPECIFIED_DEFAULT_VALUE);
   }
 
   /**
    * Class used by the tests
-   * 
-   * 
    */
   public static class ArgumentTestingClass {
 
-    /**
-     * @param defaultargument
-     * @return Object
-     */
     @SuppressWarnings("unused")
-    public static Object defaultArgumentTestingMethod(
-        @CliArgument(name = ARGUMENT_NAME)
-        String defaultargument) {
+    public static Object defaultArgumentTestingMethod(@CliArgument(name = ARGUMENT_NAME) String defaultArgument) {
       return null;
     }
 
-    /**
-     * @param argument
-     * @return Object
-     */
     @SuppressWarnings("unused")
-    public static Object argumentTestingMethod(
-        @CliArgument(name = ARGUMENT_NAME, help = ARGUMENT_HELP, mandatory = ARGUMENT_MANDATORY, argumentContext = ARGUMENT_CONTEXT, systemProvided = SYSTEM_PROVIDED, unspecifiedDefaultValue = ARGUMENT_UNSPECIFIED_DEFAULT_VALUE)
-        String argument) {
+    public static Object argumentTestingMethod(@CliArgument(name = ARGUMENT_NAME, help = ARGUMENT_HELP, mandatory = ARGUMENT_MANDATORY, argumentContext = ARGUMENT_CONTEXT, systemProvided = SYSTEM_PROVIDED, unspecifiedDefaultValue = ARGUMENT_UNSPECIFIED_DEFAULT_VALUE) String argument) {
       return null;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index 6858f78..d99bbd4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -306,7 +306,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
               "Members Array for region " + region + " : " + StringUtils.objectToString(membersName, true, 10));
           if (bean.getMemberCount() < 1) fail(
               "Even after waiting mbean reports number of member hosting region " + DATA_REGION_NAME_VM1_PATH + " is less than one");
-          //assertEquals(1, membersName.length); //exists in one members vm1
+          //assertIndexDetailsEquals(1, membersName.length); //exists in one members vm1
         }
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
index 344d116..38849c7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
@@ -118,7 +118,7 @@ public class HTTPServiceSSLSupportJUnitTest {
 
     assertEquals(config.getHttpServiceSSLProperties().get("javax.net.ssl.keyStore"), jks.getCanonicalPath());
     assertEquals(config.getHttpServiceSSLProperties().get("javax.net.ssl.keyStorePassword"), "password");
-    // assertEquals(system.getConfig().getHttpServiceSSLKeyStoreType(),"JKS");
+    // assertIndexDetailsEquals(system.getConfig().getHttpServiceSSLKeyStoreType(),"JKS");
     assertEquals(config.getHttpServiceSSLProperties().get("javax.net.ssl.trustStore"), jks.getCanonicalPath());
     assertEquals(config.getHttpServiceSSLProperties().get("javax.net.ssl.trustStorePassword"), "password");
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
index d2bdcba..c8e0ed9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
@@ -278,7 +278,7 @@ public class MemberCommandsDUnitTest extends JUnit4CacheTestCase {
           EMPTY_ENV).process();
       assertEquals(true, result.getStatus().equals(Status.OK));
       getLogWriter().info("#SB" + getResultAsString(result));
-      //assertEquals(true, result.getStatus().equals(Status.OK));
+      //assertIndexDetailsEquals(true, result.getStatus().equals(Status.OK));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/converters/RegionPathConverterJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/converters/RegionPathConverterJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/converters/RegionPathConverterJUnitTest.java
index 6a2e1c0..44d6d3f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/converters/RegionPathConverterJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/converters/RegionPathConverterJUnitTest.java
@@ -39,11 +39,9 @@ public class RegionPathConverterJUnitTest {
 
   @Before
   public void setup() {
-    mockContext = new Mockery() {
-      {
-        setImposteriser(ClassImposteriser.INSTANCE);
-      }
-    };
+    mockContext = new Mockery() {{
+      setImposteriser(ClassImposteriser.INSTANCE);
+    }};
   }
 
   @After
@@ -55,12 +53,10 @@ public class RegionPathConverterJUnitTest {
   protected RegionPathConverter createMockRegionPathConverter(final String[] allRegionPaths) {
     
     final RegionPathConverter mockRegionPathConverter = mockContext.mock(RegionPathConverter.class, "RPC");
-    mockContext.checking(new Expectations() {
-      {
-        oneOf(mockRegionPathConverter).getAllRegionPaths();
-        will(returnValue(new TreeSet<String>(Arrays.asList(allRegionPaths))));
-      }
-    });
+    mockContext.checking(new Expectations() {{
+      oneOf(mockRegionPathConverter).getAllRegionPaths();
+      will(returnValue(new TreeSet<String>(Arrays.asList(allRegionPaths))));
+    }});
 
     return mockRegionPathConverter;
   }
@@ -76,7 +72,6 @@ public class RegionPathConverterJUnitTest {
     Set<String> mocked = mockRegionPathConverter.getAllRegionPaths();
     
     assertEquals("mocked paths don't match expectedPaths.", mocked, expectedPaths);
-    
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeDiskStoreFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeDiskStoreFunctionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeDiskStoreFunctionJUnitTest.java
index 1f919bc..825053e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeDiskStoreFunctionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeDiskStoreFunctionJUnitTest.java
@@ -26,6 +26,15 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
+import org.apache.logging.log4j.Logger;
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.DiskStore;
@@ -49,16 +58,6 @@ import com.gemstone.gemfire.management.internal.cli.domain.DiskStoreDetails;
 import com.gemstone.gemfire.management.internal.cli.util.DiskStoreNotFoundException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.apache.logging.log4j.Logger;
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
-
 /**
  * The DescribeDiskStoreFunctionJUnitTest test suite class tests the contract and functionality of the
  * DescribeDiskStoreFunction class.
@@ -444,7 +443,6 @@ public class DescribeDiskStoreFunctionJUnitTest {
   }
 
   @Test
-  @Ignore("Removing the old WAN broke this method. Unignore when merged")
   public void testExecute() throws Throwable {
     final UUID diskStoreId = UUID.randomUUID();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/ListIndexFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/ListIndexFunctionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/ListIndexFunctionJUnitTest.java
index 413d0ac..5d052d7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/ListIndexFunctionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/functions/ListIndexFunctionJUnitTest.java
@@ -26,6 +26,14 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.FunctionContext;
@@ -43,15 +51,6 @@ import com.gemstone.gemfire.management.internal.cli.domain.IndexDetails;
 import com.gemstone.gemfire.management.internal.cli.domain.IndexDetails.IndexStatisticsDetails;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * The ListIndexFunctionJUnitTest class is test suite of test cases testing the contract and functionality of the
  * ListIndexFunction GemFire function.
@@ -68,15 +67,15 @@ import org.junit.experimental.categories.Category;
 @Category(UnitTest.class)
 public class ListIndexFunctionJUnitTest {
 
-  private AtomicLong mockCounter = new AtomicLong(0l);
-
   private Mockery mockContext;
+  private AtomicLong mockCounter;
 
   @Before
   public void setup() {
     mockContext = new Mockery() {{
       setImposteriser(ClassImposteriser.INSTANCE);
     }};
+    mockCounter = new AtomicLong(0l);
   }
 
   @After
@@ -85,38 +84,34 @@ public class ListIndexFunctionJUnitTest {
     mockContext = null;
   }
 
-  protected void assertEquals(final IndexDetails expectedIndexDetails, final IndexDetails actualIndexDetails) {
-    Assert.assertEquals(expectedIndexDetails.getFromClause(), actualIndexDetails.getFromClause());
-    Assert.assertEquals(expectedIndexDetails.getIndexedExpression(), actualIndexDetails.getIndexedExpression());
-    Assert.assertEquals(expectedIndexDetails.getIndexName(), actualIndexDetails.getIndexName());
-    assertEquals(expectedIndexDetails.getIndexStatisticsDetails(), actualIndexDetails.getIndexStatisticsDetails());
-    Assert.assertEquals(expectedIndexDetails.getIndexType(), actualIndexDetails.getIndexType());
-    Assert.assertEquals(expectedIndexDetails.getMemberId(), actualIndexDetails.getMemberId());
-    Assert.assertEquals(expectedIndexDetails.getMemberName(), actualIndexDetails.getMemberName());
-    Assert.assertEquals(expectedIndexDetails.getProjectionAttributes(), actualIndexDetails.getProjectionAttributes());
-    Assert.assertEquals(expectedIndexDetails.getRegionName(), actualIndexDetails.getRegionName());
-    Assert.assertEquals(expectedIndexDetails.getRegionPath(), actualIndexDetails.getRegionPath());
+  private void assertIndexDetailsEquals(final IndexDetails expectedIndexDetails, final IndexDetails actualIndexDetails) {
+    assertEquals(expectedIndexDetails.getFromClause(), actualIndexDetails.getFromClause());
+    assertEquals(expectedIndexDetails.getIndexedExpression(), actualIndexDetails.getIndexedExpression());
+    assertEquals(expectedIndexDetails.getIndexName(), actualIndexDetails.getIndexName());
+    assertIndexStatisticsDetailsEquals(expectedIndexDetails.getIndexStatisticsDetails(), actualIndexDetails.getIndexStatisticsDetails());
+    assertEquals(expectedIndexDetails.getIndexType(), actualIndexDetails.getIndexType());
+    assertEquals(expectedIndexDetails.getMemberId(), actualIndexDetails.getMemberId());
+    assertEquals(expectedIndexDetails.getMemberName(), actualIndexDetails.getMemberName());
+    assertEquals(expectedIndexDetails.getProjectionAttributes(), actualIndexDetails.getProjectionAttributes());
+    assertEquals(expectedIndexDetails.getRegionName(), actualIndexDetails.getRegionName());
+    assertEquals(expectedIndexDetails.getRegionPath(), actualIndexDetails.getRegionPath());
   }
 
-  protected void assertEquals(final IndexStatisticsDetails expectedIndexStatisticsDetails, final IndexStatisticsDetails actualIndexStatisticsDetails) {
+  private void assertIndexStatisticsDetailsEquals(final IndexStatisticsDetails expectedIndexStatisticsDetails, final IndexStatisticsDetails actualIndexStatisticsDetails) {
     if (expectedIndexStatisticsDetails != null) {
       assertNotNull(actualIndexStatisticsDetails);
-      Assert.assertEquals(expectedIndexStatisticsDetails.getNumberOfKeys(), actualIndexStatisticsDetails
-        .getNumberOfKeys());
-      Assert.assertEquals(expectedIndexStatisticsDetails.getNumberOfUpdates(), actualIndexStatisticsDetails
-        .getNumberOfUpdates());
-      Assert.assertEquals(expectedIndexStatisticsDetails.getNumberOfValues(), actualIndexStatisticsDetails
-        .getNumberOfValues());
-      Assert.assertEquals(expectedIndexStatisticsDetails.getTotalUpdateTime(), actualIndexStatisticsDetails
-        .getTotalUpdateTime());
-      Assert.assertEquals(expectedIndexStatisticsDetails.getTotalUses(), actualIndexStatisticsDetails.getTotalUses());
-    }
-    else {
+      assertEquals(expectedIndexStatisticsDetails.getNumberOfKeys(), actualIndexStatisticsDetails.getNumberOfKeys());
+      assertEquals(expectedIndexStatisticsDetails.getNumberOfUpdates(), actualIndexStatisticsDetails.getNumberOfUpdates());
+      assertEquals(expectedIndexStatisticsDetails.getNumberOfValues(), actualIndexStatisticsDetails.getNumberOfValues());
+      assertEquals(expectedIndexStatisticsDetails.getTotalUpdateTime(), actualIndexStatisticsDetails.getTotalUpdateTime());
+      assertEquals(expectedIndexStatisticsDetails.getTotalUses(), actualIndexStatisticsDetails.getTotalUses());
+
+    } else {
       assertNull(actualIndexStatisticsDetails);
     }
   }
 
-  protected IndexDetails createIndexDetails(final String memberId,
+  private IndexDetails createIndexDetails(final String memberId,
                                             final String regionPath,
                                             final String indexName,
                                             final IndexType indexType,
@@ -124,8 +119,7 @@ public class ListIndexFunctionJUnitTest {
                                             final String indexedExpression,
                                             final String memberName,
                                             final String projectionAttributes,
-                                            final String regionName)
-  {
+                                            final String regionName) {
     final IndexDetails indexDetails = new IndexDetails(memberId, regionPath, indexName);
 
     indexDetails.setFromClause(fromClause);
@@ -138,12 +132,11 @@ public class ListIndexFunctionJUnitTest {
     return indexDetails;
   }
 
-  protected IndexStatisticsDetails createIndexStatisticsDetails(final Long numberOfKeys,
+  private IndexStatisticsDetails createIndexStatisticsDetails(final Long numberOfKeys,
                                                                 final Long numberOfUpdates,
                                                                 final Long numberOfValues,
                                                                 final Long totalUpdateTime,
-                                                                final Long totalUses)
-  {
+                                                                final Long totalUses) {
     final IndexStatisticsDetails indexStatisticsDetails = new IndexStatisticsDetails();
 
     indexStatisticsDetails.setNumberOfKeys(numberOfKeys);
@@ -155,16 +148,14 @@ public class ListIndexFunctionJUnitTest {
     return indexStatisticsDetails;
   }
 
-  protected ListIndexFunction createListIndexFunction(final Cache cache) {
+  private ListIndexFunction createListIndexFunction(final Cache cache) {
     return new TestListIndexFunction(cache);
   }
 
-  protected Index createMockIndex(final IndexDetails indexDetails) {
-    final Index mockIndex = mockContext.mock(Index.class, "Index " + indexDetails.getIndexName() + " "
-      + mockCounter.getAndIncrement());
+  private Index createMockIndex(final IndexDetails indexDetails) {
+    final Index mockIndex = mockContext.mock(Index.class, "Index " + indexDetails.getIndexName() + " " + mockCounter.getAndIncrement());
 
-    final Region mockRegion = mockContext.mock(Region.class, "Region " + indexDetails.getRegionPath() + " "
-      + mockCounter.getAndIncrement());
+    final Region mockRegion = mockContext.mock(Region.class, "Region " + indexDetails.getRegionPath() + " " + mockCounter.getAndIncrement());
 
     mockContext.checking(new Expectations() {{
       oneOf(mockIndex).getFromClause();
@@ -186,8 +177,7 @@ public class ListIndexFunctionJUnitTest {
     }});
 
     if (indexDetails.getIndexStatisticsDetails() != null) {
-      final IndexStatistics mockIndexStatistics = mockContext.mock(IndexStatistics.class, "IndexStatistics "
-        + indexDetails.getIndexName() + " " + mockCounter.getAndIncrement());
+      final IndexStatistics mockIndexStatistics = mockContext.mock(IndexStatistics.class, "IndexStatistics " + indexDetails.getIndexName() + " " + mockCounter.getAndIncrement());
 
       mockContext.checking(new Expectations() {{
         exactly(2).of(mockIndex).getStatistics();
@@ -203,8 +193,8 @@ public class ListIndexFunctionJUnitTest {
         oneOf(mockIndexStatistics).getTotalUses();
         will(returnValue(indexDetails.getIndexStatisticsDetails().getTotalUses()));
       }});
-    }
-    else {
+
+    } else {
       mockContext.checking(new Expectations() {{
         oneOf(mockIndex).getStatistics();
         will(returnValue(null));
@@ -214,7 +204,7 @@ public class ListIndexFunctionJUnitTest {
     return mockIndex;
   }
 
-  protected IndexType getIndexType(final IndexDetails.IndexType type) {
+  private IndexType getIndexType(final IndexDetails.IndexType type) {
     switch (type) {
       case FUNCTIONAL:
         return IndexType.FUNCTIONAL;
@@ -289,12 +279,12 @@ public class ListIndexFunctionJUnitTest {
     final List<?> results = testResultSender.getResults();
 
     assertNotNull(results);
-    Assert.assertEquals(1, results.size());
+    assertEquals(1, results.size());
 
     final Set<IndexDetails> actualIndexDetailsSet = (Set<IndexDetails>) results.get(0);
 
     assertNotNull(actualIndexDetailsSet);
-    Assert.assertEquals(expectedIndexDetailsSet.size(), actualIndexDetailsSet.size());
+    assertEquals(expectedIndexDetailsSet.size(), actualIndexDetailsSet.size());
 
     for (final IndexDetails expectedIndexDetails : expectedIndexDetailsSet) {
       final IndexDetails actualIndexDetails = CollectionUtils.findBy(actualIndexDetailsSet, new Filter<IndexDetails>() {
@@ -304,7 +294,7 @@ public class ListIndexFunctionJUnitTest {
       });
 
       assertNotNull(actualIndexDetails);
-      assertEquals(expectedIndexDetails, actualIndexDetails);
+      assertIndexDetailsEquals(expectedIndexDetails, actualIndexDetails);
     }
   }
 
@@ -342,7 +332,7 @@ public class ListIndexFunctionJUnitTest {
     final List<?> results = testResultSender.getResults();
 
     assertNotNull(results);
-    Assert.assertEquals(1, results.size());
+    assertEquals(1, results.size());
 
     final Set<IndexDetails> actualIndexDetailsSet = (Set<IndexDetails>) results.get(0);
 
@@ -382,10 +372,9 @@ public class ListIndexFunctionJUnitTest {
 
     try {
       testResultSender.getResults();
-    }
-    catch (Throwable t) {
+    } catch (Throwable t) {
       assertTrue(t instanceof RuntimeException);
-      Assert.assertEquals("expected", t.getMessage());
+      assertEquals("expected", t.getMessage());
       throw t;
     }
   }
@@ -418,14 +407,17 @@ public class ListIndexFunctionJUnitTest {
       return Collections.unmodifiableList(results);
     }
 
+    @Override
     public void lastResult(final Object lastResult) {
       results.add(lastResult);
     }
 
+    @Override
     public void sendResult(final Object oneResult) {
       results.add(oneResult);
     }
 
+    @Override
     public void sendException(final Throwable t) {
       this.t = t;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/ParserUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/ParserUtilsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/ParserUtilsJUnitTest.java
index 03d73d3..b4d688c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/ParserUtilsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/ParserUtilsJUnitTest.java
@@ -16,24 +16,23 @@
  */
 package com.gemstone.gemfire.management.internal.cli.parser;
 
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.management.internal.cli.parser.ParserUtils;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Includes tests for all utility methods in {@link ParserUtils}
- * 
- * 
  */
 @Category(UnitTest.class)
-public class ParserUtilsJUnitTest extends TestCase {
+public class ParserUtilsJUnitTest {
 
   /**
    * Test for {@link ParserUtils#split(String, String)}
    */
+  @Test
   public void testSplit() {
     String input = "something::{::}::nothing";
     String[] split = ParserUtils.split(input, "::");
@@ -46,6 +45,7 @@ public class ParserUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link ParserUtils#splitValues(String, String)}
    */
+  @Test
   public void testSplitValues() {
     String input = "something::{::}::nothing::";
     String[] split = ParserUtils.splitValues(input, "::");
@@ -59,6 +59,7 @@ public class ParserUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link ParserUtils#contains(String, String)}
    */
+  @Test
   public void testContains() {
     String input = "something::{::}::nothing::";
     assertTrue("Check Boolean", ParserUtils.contains(input, "::"));
@@ -69,6 +70,7 @@ public class ParserUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link ParserUtils#lastIndexOf(String, String)}
    */
+  @Test
   public void testLastIndexOf() {
     String input = "something::{::}::nothing::";
     assertEquals("lastIndex", 24, ParserUtils.lastIndexOf(input, "::"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
index 5b928e3..2d217cc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
@@ -16,7 +16,7 @@
  */
 package com.gemstone.gemfire.management.internal.cli.parser.preprocessor;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -24,13 +24,10 @@ import org.junit.experimental.categories.Category;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
- * 
  * Test for Preprocessor
- * 
- *
  */
 @Category(UnitTest.class)
-public class PreprocessorJUnitTest extends TestCase{
+public class PreprocessorJUnitTest {
   
   @Test
   public void test1Arg() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
index 2a56747..05be5bb 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
@@ -16,26 +16,24 @@
  */
 package com.gemstone.gemfire.management.internal.cli.parser.preprocessor;
 
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.lang.SystemUtils;
-import com.gemstone.gemfire.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-import com.gemstone.gemfire.management.internal.cli.parser.preprocessor.TrimmedInput;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Includes tests for all utility methods in {@link PreprocessorUtils}
- * 
- * 
  */
 @Category(UnitTest.class)
-public class PreprocessorUtilsJUnitTest extends TestCase {
+public class PreprocessorUtilsJUnitTest {
 
   /**
    * Test for {@link PreprocessorUtils#simpleTrim(String)}
    */
+  @Test
   public void testSimpleTrim() {
     String input = " 1 2 3 ";
     TrimmedInput simpleTrim = PreprocessorUtils.simpleTrim(input);
@@ -51,6 +49,7 @@ public class PreprocessorUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link PreprocessorUtils#trim(String)}
    */
+  @Test
   public void testTrim() {
     String input = " command argument1 argument2 ";
     TrimmedInput trim = PreprocessorUtils.trim(input);
@@ -86,6 +85,7 @@ public class PreprocessorUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link PreprocessorUtils#removeWhiteSpaces(String)}
    */
+  @Test
   public void testRemoveWhiteSpaces() {
     String input = "1 2 3   ";
     String output = PreprocessorUtils.removeWhiteSpaces(input);
@@ -95,6 +95,7 @@ public class PreprocessorUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link PreprocessorUtils#isSyntaxValid(String)}
    */
+  @Test
   public void testIsSyntaxValid() {
     assertTrue(PreprocessorUtils.isSyntaxValid("{}"));
     assertFalse(PreprocessorUtils.isSyntaxValid("{{]}"));
@@ -106,6 +107,7 @@ public class PreprocessorUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link PreprocessorUtils#containsOnlyWhiteSpaces(String)}
    */
+  @Test
   public void testContainsOnlyWhiteSpaces() {
     assertTrue(PreprocessorUtils
         .containsOnlyWhiteSpaces("                                                  "));
@@ -116,6 +118,7 @@ public class PreprocessorUtilsJUnitTest extends TestCase {
   /**
    * Test for {@link PreprocessorUtils#isWhitespace(char)}
    */
+  @Test
   public void testIsWhitespace() {
     assertTrue(PreprocessorUtils.isWhitespace(' '));
     assertTrue(PreprocessorUtils.isWhitespace('\t'));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshConfigInitFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshConfigInitFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshConfigInitFileJUnitTest.java
index bbd1946..81b99c2 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshConfigInitFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshConfigInitFileJUnitTest.java
@@ -26,10 +26,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class GfshConfigInitFileJUnitTest {
+
   private static final String INIT_FILE_NAME = GfshConfig.DEFAULT_INIT_FILE_NAME;
   private static final String INIT_FILE_PROPERTY = GfshConfig.INIT_FILE_PROPERTY;
 
@@ -65,8 +66,7 @@ public class GfshConfigInitFileJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    String userDir = temporaryFolder_CurrentDirectory.getRoot()
-        .getAbsolutePath();
+    String userDir = temporaryFolder_CurrentDirectory.getRoot().getAbsolutePath();
     String userHome = temporaryFolder_HomeDirectory.getRoot().getAbsolutePath();
 
     System.setProperty("user.dir", userDir);
@@ -82,8 +82,7 @@ public class GfshConfigInitFileJUnitTest {
   public void constructorArgumentUsed() throws Exception {
     temporaryFolder_HomeDirectory.newFile(INIT_FILE_NAME);
     temporaryFolder_CurrentDirectory.newFile(INIT_FILE_NAME);
-    System.setProperty(INIT_FILE_PROPERTY, temporaryFolder_AnotherDirectory
-        .newFile(INIT_FILE_NAME).getAbsolutePath());
+    System.setProperty(INIT_FILE_PROPERTY, temporaryFolder_AnotherDirectory.newFile(INIT_FILE_NAME).getAbsolutePath());
 
     String argument = temporaryFolder_AnotherDirectory.newFile("junit")
         .getAbsolutePath();
@@ -107,8 +106,7 @@ public class GfshConfigInitFileJUnitTest {
   public void systemPropertySelectedFirst() throws Exception {
     temporaryFolder_HomeDirectory.newFile(INIT_FILE_NAME);
     temporaryFolder_CurrentDirectory.newFile(INIT_FILE_NAME);
-    String fileName = temporaryFolder_AnotherDirectory.newFile(INIT_FILE_NAME)
-        .getAbsolutePath();
+    String fileName = temporaryFolder_AnotherDirectory.newFile(INIT_FILE_NAME).getAbsolutePath();
     System.setProperty(INIT_FILE_PROPERTY, fileName);
 
     /*
@@ -129,8 +127,7 @@ public class GfshConfigInitFileJUnitTest {
   @Test
   public void currentDirectorySelectedSecond() throws Exception {
     temporaryFolder_HomeDirectory.newFile(INIT_FILE_NAME);
-    String fileName = temporaryFolder_CurrentDirectory.newFile(INIT_FILE_NAME)
-        .getAbsolutePath();
+    String fileName = temporaryFolder_CurrentDirectory.newFile(INIT_FILE_NAME).getAbsolutePath();
 
     /*
      * String historyFileName, String defaultPrompt, int historySize, String
@@ -149,8 +146,7 @@ public class GfshConfigInitFileJUnitTest {
   // Home directory file selected if only one present
   @Test
   public void homeDirectorySelectedThird() throws Exception {
-    String fileName = temporaryFolder_HomeDirectory.newFile(INIT_FILE_NAME)
-        .getAbsolutePath();
+    String fileName = temporaryFolder_HomeDirectory.newFile(INIT_FILE_NAME).getAbsolutePath();
 
     /*
      * String historyFileName, String defaultPrompt, int historySize, String

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
index d4edfeb..9792171 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
@@ -28,24 +28,21 @@ import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 import org.springframework.shell.event.ParseResult;
 
+import com.gemstone.gemfire.management.cli.CliMetaData;
+import com.gemstone.gemfire.management.cli.ConverterHint;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.CommandManager;
 import com.gemstone.gemfire.management.internal.cli.GfshParser;
 import com.gemstone.gemfire.management.internal.cli.annotation.CliArgument;
-import com.gemstone.gemfire.management.cli.CliMetaData;
-import com.gemstone.gemfire.management.cli.ConverterHint;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
-import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
-import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
-import com.gemstone.gemfire.management.internal.cli.shell.GfshExecutionStrategy;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * GfshExecutionStrategyTest - Includes tests to for GfshExecutionStrategyTest
- * 
  */
 @Category(UnitTest.class)
 public class GfshExecutionStrategyJUnitTest {
+
   private static final String COMMAND1_NAME = "command1";
   private static final String COMMAND1_NAME_ALIAS = "command1_alias";
   private static final String COMMAND2_NAME = "command2";
@@ -65,8 +62,7 @@ public class GfshExecutionStrategyJUnitTest {
     assertNotNull("CommandManager should not be null.", commandManager);      
     commandManager.add(Commands.class.newInstance());
     GfshParser parser = new GfshParser(commandManager);
-    String[] command1Names = ((CliCommand) Commands.class.getMethod(
-        COMMAND1_NAME).getAnnotation(CliCommand.class)).value();
+    String[] command1Names = ((CliCommand) Commands.class.getMethod(COMMAND1_NAME).getAnnotation(CliCommand.class)).value();
     String input =command1Names[0];
     ParseResult parseResult = null;   
     parseResult = parser.parse(input);  
@@ -86,8 +82,7 @@ public class GfshExecutionStrategyJUnitTest {
     CommandManager commandManager = CommandManager.getInstance();
     assertNotNull("CommandManager should not be null.", commandManager);
     commandManager.add(Commands.class.newInstance());      
-    String[] command1Names = ((CliCommand) Commands.class.getMethod(
-        COMMAND1_NAME).getAnnotation(CliCommand.class)).value();       
+    String[] command1Names = ((CliCommand) Commands.class.getMethod(COMMAND1_NAME).getAnnotation(CliCommand.class)).value();
     String[] args = new String[] {command1Names[0]  };
     Gfsh gfsh = Gfsh.getInstance(false, args, new GfshConfig());      
     GfshExecutionStrategy gfshExecutionStrategy = new GfshExecutionStrategy(gfsh);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
index 5a5501d..62f16d7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/shell/GfshHistoryJUnitTest.java
@@ -16,7 +16,13 @@
  */
 package com.gemstone.gemfire.management.internal.cli.shell;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.util.List;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -24,16 +30,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 
-import java.io.File;
-import java.lang.reflect.Field;
-import java.nio.file.Files;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-/**
- */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class GfshHistoryJUnitTest {
 
   private File gfshHistoryFile;
@@ -82,7 +81,6 @@ public class GfshHistoryJUnitTest {
     gfsh.executeScriptLine("connect --password=foo --password = foo --password= goo --password =goo --password-param=blah --other-password-param=    gah");
 
     List<String> lines = Files.readAllLines(gfshHistoryFile.toPath());
-    assertEquals("// [failed] connect --password=***** --password = ***** --password= ***** --password =***** --password-param=***** --other-password-param= *****",
-        lines.get(1));
+    assertEquals("// [failed] connect --password=***** --password = ***** --password= ***** --password =***** --password-param=***** --other-password-param= *****", lines.get(1));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
index dca5d0b..df58d1a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
@@ -44,8 +44,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Test cases for {@link CacheElement}.
- * 
- * 
+ *
  * @see CacheElement
  *
  * @since 8.1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.java
index 24702c3..6c1bcca 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.java
@@ -45,7 +45,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 /**
  * Unit tests for {@link XmlUtils}. See Also {@link XmlUtilsAddNewNodeJUnitTest}
  * for tests related to {@link XmlUtils#addNewNode(Document, XmlEntity)}
- * 
  *
  * @since 8.1
  */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
index d9362cd..4a36384 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
@@ -17,9 +17,11 @@
 package com.gemstone.gemfire.management.internal.web.controllers;
 
 import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
-import static junitparams.JUnitParamsRunner.$;
+import static junitparams.JUnitParamsRunner.*;
 import static org.assertj.core.api.Assertions.*;
 
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -27,14 +29,10 @@ import org.junit.runner.RunWith;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-
 /**
  * Unit tests for WanCommandsController. 
  * 
  * Introduced for GEODE-213 "JMX -http manager treats "start gateway-sender" as "start gateway-receiver"
- *  
  */
 @SuppressWarnings("unused")
 @Category(UnitTest.class)
@@ -128,6 +126,7 @@ public class WanCommandsControllerJUnitTest {
    * class while testing just the command string definition class.
    */
   public static class TestableWanCommandsController extends WanCommandsController {
+
     protected String testableCommand;
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
index 14a514d..55aa5aa 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
@@ -372,7 +372,7 @@ public class JSONPdxClientServerDUnitTest extends CacheTestCase {
     byte[] o1 = jsonParse(jd.getJsonByteArray());
     byte[] o2 = jsonParse(jsonByteArray);
     
-   // junit.framework.Assert.assertEquals("Pdx byte aray are not equal after fetching from cache " + jd.getFileName(), o1, o2); 
+   // junit.framework.Assert.assertIndexDetailsEquals("Pdx byte aray are not equal after fetching from cache " + jd.getFileName(), o1, o2);
    compareByteArray(o1, o2);
     
     PdxInstance pdx2 = JSONFormatter.fromJSON(jsonByteArray);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxDeserializationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxDeserializationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxDeserializationDUnitTest.java
index 70ae048..fde1a27 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxDeserializationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxDeserializationDUnitTest.java
@@ -321,21 +321,21 @@ public class PdxDeserializationDUnitTest extends CacheTestCase {
 //    //Test puts and get in a transaction
 //    txManager.begin();
 //    region.put("C", new TestSerializable());
-//    assertEquals(TestSerializable.class, region.get("C").getClass());
+//    assertIndexDetailsEquals(TestSerializable.class, region.get("C").getClass());
 //    txManager.commit();
 //    
 //    txManager.begin();
-//    assertEquals(TestSerializable.class, region.get("C").getClass());
+//    assertIndexDetailsEquals(TestSerializable.class, region.get("C").getClass());
 //    txManager.commit();
 //    
 //    
 //    //Test cache load in a transaction
 //    txManager.begin();
-//    assertEquals(TestSerializable.class, region.get("D").getClass());
+//    assertIndexDetailsEquals(TestSerializable.class, region.get("D").getClass());
 //    txManager.commit();
 //    
 //    txManager.begin();
-//    assertEquals(TestSerializable.class, region.get("D").getClass());
+//    assertIndexDetailsEquals(TestSerializable.class, region.get("D").getClass());
 //    txManager.commit();
   }
   
@@ -343,8 +343,8 @@ public class PdxDeserializationDUnitTest extends CacheTestCase {
     assertEquals(TestSerializable.class, region.get("A").getClass());
     assertEquals(TestSerializable.class, region.get("B").getClass());
     //TODO Transactions don't work
-//    assertEquals(TestSerializable.class, region.get("C").getClass());
-//    assertEquals(TestSerializable.class, region.get("D").getClass());
+//    assertIndexDetailsEquals(TestSerializable.class, region.get("C").getClass());
+//    assertIndexDetailsEquals(TestSerializable.class, region.get("D").getClass());
   }
 
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
index 7a4308f..a850e12 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
@@ -437,7 +437,7 @@ public abstract class ClientAuthorizationTestCase extends JUnit4DistributedTestC
           // if ((flags & OpFlags.CHECK_NOTAUTHZ) > 0) {
           // Object value = region.get(key);
           // assertNotNull(value);
-          // assertEquals(vals[index], value);
+          // assertIndexDetailsEquals(vals[index], value);
           // }
           // else {
           // region.put(key, vals[index]);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
index ec054d7..67dfadc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
@@ -71,10 +71,10 @@ public class NotAuthorizedExceptionTest {
     this.nonSerializablePrincipal = mock(Principal.class);
     this.serializablePrincipal = new SerializablePrincipal(this.principalName);
 
-    assertPreConditions();
+    assertPreconditions();
   }
 
-  private void assertPreConditions() {
+  private void assertPreconditions() {
     catchException(this).clone(this.nonSerializableNamingException);
     assertThat((Throwable)caughtException()).isNotNull();
     assertThat((Throwable)caughtException().getCause()).isInstanceOf(NotSerializableException.class);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
index 677e2d4..95cd99c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
@@ -24,12 +24,13 @@ import org.apache.commons.lang.SerializationUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link PKCSPrincipal}
  */
-@Category(UnitTest.class)
+@Category({ UnitTest.class, SecurityTest.class })
 public class PKCSPrincipalTest {
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
index bce19b4..7fbb454 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
@@ -24,12 +24,13 @@ import org.apache.commons.lang.SerializationUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link UsernamePrincipal}
  */
-@Category(UnitTest.class)
+@Category({ UnitTest.class, SecurityTest.class })
 public class UsernamePrincipalTest {
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Assert.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Assert.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Assert.java
index 5d927eb..683d01c 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Assert.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Assert.java
@@ -20,7 +20,7 @@ package com.gemstone.gemfire.test.dunit;
  * Extends <code>org.junit.Assert</code> with additional assertion and fail
  * methods. 
  * 
- * These methods can be used directly: <code>Assert.assertEquals(...)</code>, 
+ * These methods can be used directly: <code>Assert.assertIndexDetailsEquals(...)</code>,
  * however, they are intended to be referenced through static import:
  *
  * <pre>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/RMIException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/RMIException.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/RMIException.java
index 82ede05..cc0b69c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/RMIException.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/RMIException.java
@@ -30,7 +30,7 @@ import com.gemstone.gemfire.GemFireException;
  *       vm.invoke(() -> this.getUnknownObject());
  *
  *     } catch (RMIException ex) {
- *       assertEquals(ex.getCause() instanceof ObjectException);
+ *       assertIndexDetailsEquals(ex.getCause() instanceof ObjectException);
  *     }
  * </PRE>
  *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableCallable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableCallable.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableCallable.java
index f9291fd..ec40423 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableCallable.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableCallable.java
@@ -39,7 +39,7 @@ import java.util.concurrent.Callable;
  *       }
  *      });
  *   assertNull(vm0.invoke(putMethod));
- *   assertEquals(value, vm1.invoke(putMethod));
+ *   assertIndexDetailsEquals(value, vm1.invoke(putMethod));
  *  }
  * </PRE>
  * 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableRunnable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableRunnable.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableRunnable.java
index 4776775..28ef471 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableRunnable.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/SerializableRunnable.java
@@ -40,7 +40,7 @@ import java.io.Serializable;
  *   vm1.invoke(new SerializableRunnable("Get value") {
  *       public void run() {
  *         ...// get the region //...
- *         assertEquals(value, region.get(name));
+ *         assertIndexDetailsEquals(value, region.get(name));
  *       }
  *     });
  *  }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/test/process/MainLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/process/MainLauncherJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/process/MainLauncherJUnitTest.java
index 15341c6..5d1c05f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/process/MainLauncherJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/process/MainLauncherJUnitTest.java
@@ -37,7 +37,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Quick sanity tests to make sure MainLauncher is functional.
- * 
  */
 @Category(UnitTest.class)
 public class MainLauncherJUnitTest {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryDUnitTest.java
index 4e0ec90..2b2dbd7 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryDUnitTest.java
@@ -1190,7 +1190,7 @@ public class CqQueryDUnitTest extends CacheTestCase {
           // Since ResultSet is not maintained for this release.
           // Instead of resultSize its been validated with total number of events.
           fail("test for event counts instead of results size");
-//        assertEquals("Result Size mismatch", resultSize, listener.getTotalEventCount());
+//        assertIndexDetailsEquals("Result Size mismatch", resultSize, listener.getTotalEventCount());
         }
         
         // Check for create count.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolDUnitTest.java
index 4ba5e48..548dd2f 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolDUnitTest.java
@@ -1019,7 +1019,7 @@ public class CqQueryUsingPoolDUnitTest extends CacheTestCase {
           // Since ResultSet is not maintained for this release.
           // Instead of resultSize its been validated with total number of events.
           fail("test for event counts instead of results size");
-//        assertEquals("Result Size mismatch", resultSize, listener.getTotalEventCount());
+//        assertIndexDetailsEquals("Result Size mismatch", resultSize, listener.getTotalEventCount());
         }
         
         // Check for create count.
@@ -1387,7 +1387,7 @@ public class CqQueryUsingPoolDUnitTest extends CacheTestCase {
         assertNotNull(region);
         
 //        Set keys = region.entrySet();
-//        assertEquals("Mismatch, number of keys in local region is not equal to the interest list size", 
+//        assertIndexDetailsEquals("Mismatch, number of keys in local region is not equal to the interest list size",
 //            size, keys.size());
         // TODO does this WaitCriterion actually help?
         WaitCriterion wc = new WaitCriterion() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsDUnitTest.java
index 26cda3c..fcd8160 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsDUnitTest.java
@@ -146,7 +146,7 @@ public class CqStatsDUnitTest extends CacheTestCase {
         }
 ////      Check for initial results time.
 //        if (initialResultsTime != CqQueryDUnitTest.noTest && cqVsdStats.getCqInitialResultsTime() <= 0) {
-//          assertEquals("Initial results time mismatch", initialResultsTime, cqVsdStats.getCqInitialResultsTime());
+//          assertIndexDetailsEquals("Initial results time mismatch", initialResultsTime, cqVsdStats.getCqInitialResultsTime());
 //        }
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
index d4ba5ff..71b11d8 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
@@ -146,7 +146,7 @@ public class CqStatsUsingPoolDUnitTest extends CacheTestCase {
         }
 ////      Check for initial results time.
 //        if (initialResultsTime != CqQueryUsingPoolDUnitTest.noTest && cqVsdStats.getCqInitialResultsTime() <= 0) {
-//          assertEquals("Initial results time mismatch", initialResultsTime, cqVsdStats.getCqInitialResultsTime());
+//          assertIndexDetailsEquals("Initial results time mismatch", initialResultsTime, cqVsdStats.getCqInitialResultsTime());
 //        }
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryDUnitTest.java
index 5e4dd3d..29eec8e 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryDUnitTest.java
@@ -169,7 +169,7 @@ public class PartitionedRegionCqQueryDUnitTest extends CacheTestCase {
     //cc1 = server1.invoke(() -> PartitionedRegionCqQueryDUnitTest.getCqCountFromRegionProfile());
     cc2 = server2.invoke(() -> PartitionedRegionCqQueryDUnitTest.getCqCountFromRegionProfile());
     
-    //assertEquals("Should have one", 0, cc1);
+    //assertIndexDetailsEquals("Should have one", 0, cc1);
     assertEquals("Should have one", 0, cc2);
     
     cqHelper.closeServer(server2);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/PutAllCSDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/PutAllCSDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/PutAllCSDUnitTest.java
index fafcd9b..68960bd 100755
--- a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/PutAllCSDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/PutAllCSDUnitTest.java
@@ -2168,7 +2168,7 @@ public void testOneServer() throws CacheException, InterruptedException {
     int client2Size = getRegionSize(client2, regionName);
     int server1Size = getRegionSize(server1, regionName);
     LogWriterUtils.getLogWriter().info("region sizes: "+client1Size+","+client2Size+","+server1Size);
-//    assertEquals(server1Size, client1Size);
+//    assertIndexDetailsEquals(server1Size, client1Size);
 
     // restart server2
     createBridgeServer(server2, regionName, serverPort2, true, 0, "ds1");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
index 020f967..5c3a1b3 100755
--- a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
@@ -85,7 +85,7 @@ public class RemoteCQTransactionDUnitTest extends CacheTestCase {
   private final SerializableCallable verifyNoTxState = new SerializableCallable() {
     public Object call() throws Exception {
       //TXManagerImpl mgr = getGemfireCache().getTxManager();
-      //assertEquals(0, mgr.hostedTransactionsInProgressForTest());
+      //assertIndexDetailsEquals(0, mgr.hostedTransactionsInProgressForTest());
       final TXManagerImpl mgr = getGemfireCache().getTxManager();
       Wait.waitForCriterion(new WaitCriterion() {
         @Override
@@ -244,14 +244,14 @@ public class RemoteCQTransactionDUnitTest extends CacheTestCase {
       Region.Entry eR = refRegion.getEntry(custId);
       assertNotNull(eC);
       assertNotNull(eR);
-//      assertEquals(1,custRegion.size());
-  //    assertEquals(1,orderRegion.size());
-    //  assertEquals(1,refRegion.size());
+//      assertIndexDetailsEquals(1,custRegion.size());
+  //    assertIndexDetailsEquals(1,orderRegion.size());
+    //  assertIndexDetailsEquals(1,refRegion.size());
       
     } else {
-      //assertEquals(0,custRegion.size());
-      //assertEquals(0,orderRegion.size());
-      //assertEquals(0,refRegion.size());
+      //assertIndexDetailsEquals(0,custRegion.size());
+      //assertIndexDetailsEquals(0,orderRegion.size());
+      //assertIndexDetailsEquals(0,refRegion.size());
       try {
         Region.Entry eC =  custRegion.getEntry(custId);
         assertNull("should have had an EntryNotFoundException:"+eC,eC);
@@ -303,13 +303,13 @@ public class RemoteCQTransactionDUnitTest extends CacheTestCase {
       assertEquals(expectedCust, custRegion.getEntry(custId).getValue());
       /*
       assertNotNull(orderRegion.getEntry(orderId));
-      assertEquals(expectedOrder, orderRegion.getEntry(orderId).getValue());
+      assertIndexDetailsEquals(expectedOrder, orderRegion.getEntry(orderId).getValue());
       
       assertNotNull(orderRegion.getEntry(orderId2));
-      assertEquals(expectedOrder2, orderRegion.getEntry(orderId2).getValue());
+      assertIndexDetailsEquals(expectedOrder2, orderRegion.getEntry(orderId2).getValue());
       
       assertNotNull(orderRegion.getEntry(orderId3));
-      assertEquals(expectedOrder3, orderRegion.getEntry(orderId3).getValue());
+      assertIndexDetailsEquals(expectedOrder3, orderRegion.getEntry(orderId3).getValue());
       */
       assertNotNull(refRegion.getEntry(custId));
       assertEquals(expectedRef, refRegion.getEntry(custId).getValue());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
index 5d2290b..6d830b8 100755
--- a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
@@ -454,7 +454,7 @@ public class CQListGIIDUnitTest extends DistributedTestCase {
       assertTrue("executeWithInitialResults() state mismatch", cq1.getState()
           .isRunning());
       // if (expectedResultsSize >= 0) {
-      // assertEquals("unexpected results size", expectedResultsSize, cqResults
+      // assertIndexDetailsEquals("unexpected results size", expectedResultsSize, cqResults
       // .size());
       // }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
index 563c9a7..cfe4d67 100755
--- a/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
@@ -153,7 +153,7 @@ public class DurableClientTestCase extends DistributedTestCase {
         assertTrue(proxy.isDurable());
         assertEquals(durableClientId, proxy.getDurableId());
         assertEquals(DistributionConfig.DEFAULT_DURABLE_CLIENT_TIMEOUT, proxy.getDurableTimeout());
-        //assertEquals(DistributionConfig.DEFAULT_DURABLE_CLIENT_KEEP_ALIVE, proxy.getDurableKeepAlive());
+        //assertIndexDetailsEquals(DistributionConfig.DEFAULT_DURABLE_CLIENT_KEEP_ALIVE, proxy.getDurableKeepAlive());
       }
     });
     
@@ -216,7 +216,7 @@ public class DurableClientTestCase extends DistributedTestCase {
 
           assertEquals(dId, proxy.getDurableId());
           assertEquals(DistributionConfig.DEFAULT_DURABLE_CLIENT_TIMEOUT, proxy.getDurableTimeout());
-          // assertEquals(DistributionConfig.DEFAULT_DURABLE_CLIENT_KEEP_ALIVE, proxy.getDurableKeepAlive());
+          // assertIndexDetailsEquals(DistributionConfig.DEFAULT_DURABLE_CLIENT_KEEP_ALIVE, proxy.getDurableKeepAlive());
         }
       });
 
@@ -339,7 +339,7 @@ public class DurableClientTestCase extends DistributedTestCase {
         assertTrue(proxy.isDurable());
         assertEquals(durableClientId, proxy.getDurableId());
         assertEquals(durableClientTimeout, proxy.getDurableTimeout());
-        //assertEquals(durableClientKeepAlive, proxy.getDurableKeepAlive());
+        //assertIndexDetailsEquals(durableClientKeepAlive, proxy.getDurableKeepAlive());
       }
     });
     
@@ -420,7 +420,7 @@ public class DurableClientTestCase extends DistributedTestCase {
         assertTrue(proxy.isDurable());
         assertEquals(durableClientId, proxy.getDurableId());
         assertEquals(durableClientTimeout, proxy.getDurableTimeout());
-        //assertEquals(durableClientKeepAlive, proxy.getDurableKeepAlive());
+        //assertIndexDetailsEquals(durableClientKeepAlive, proxy.getDurableKeepAlive());
       }
     });
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
index 06a5d1a..d784397 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
@@ -473,7 +473,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
   }
 
   protected void assertCacheServerConfig(CacheServerMXBean bean) {
-    // assertEquals(ServerInfo.getInstance().getServerPort(), bean.getPort());
+    // assertIndexDetailsEquals(ServerInfo.getInstance().getServerPort(), bean.getPort());
     assertEquals(CacheServer.DEFAULT_BIND_ADDRESS, bean.getBindAddress());
     assertEquals(CacheServer.DEFAULT_HOSTNAME_FOR_CLIENTS, bean
         .getHostNameForClients());


[06/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
index bdd8654..399f1ca 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
@@ -771,7 +771,7 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
       }
     };
     Wait.waitForCriterion(wc, 60000, 1000, true);
-//    assertEquals(4/*numBuckets*/, prMetaData.getBucketServerLocationsMap_TEST_ONLY().size());    
+//    assertIndexDetailsEquals(4/*numBuckets*/, prMetaData.getBucketServerLocationsMap_TEST_ONLY().size());
   }
 
   public void testMetadataFetchOnlyThroughputAll() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleNodeOperationsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleNodeOperationsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleNodeOperationsJUnitTest.java
index beec9c6..b8d75a5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleNodeOperationsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleNodeOperationsJUnitTest.java
@@ -1064,8 +1064,8 @@ public class PartitionedRegionSingleNodeOperationsJUnitTest
         .values().iterator();
     while (buckRegionIterator.hasNext()) {
       Region bucket = (Region)buckRegionIterator.next();
-//      assertEquals(Scope.LOCAL, bucket.getAttributes().getScope());
-//      assertEquals(DataPolicy.NORMAL, bucket.getAttributes().getDataPolicy());
+//      assertIndexDetailsEquals(Scope.LOCAL, bucket.getAttributes().getScope());
+//      assertIndexDetailsEquals(DataPolicy.NORMAL, bucket.getAttributes().getDataPolicy());
       assertEquals(BucketRegion.class, bucket.getClass());
     }
 
@@ -1079,8 +1079,8 @@ public class PartitionedRegionSingleNodeOperationsJUnitTest
      * putSomeValues(pr1); java.util.Iterator buckRegionIterator1 =
      * pr1.getDataStore().localBucket2RegionMap .values().iterator(); while
      * (buckRegionIterator1.hasNext()) { Region bucket =
-     * (Region)buckRegionIterator1.next(); assertEquals(Scope.DISTRIBUTED_ACK,
-     * bucket.getAttributes().getScope()); assertEquals(MirrorType.KEYS_VALUES,
+     * (Region)buckRegionIterator1.next(); assertIndexDetailsEquals(Scope.DISTRIBUTED_ACK,
+     * bucket.getAttributes().getScope()); assertIndexDetailsEquals(MirrorType.KEYS_VALUES,
      * bucket.getAttributes() .getMirrorType()); }
      * 
      * pr1.destroyRegion();
@@ -1093,8 +1093,8 @@ public class PartitionedRegionSingleNodeOperationsJUnitTest
      * pr2.getDataStore().localBucket2RegionMap .values().iterator(); while
      * (buckRegionIterator2.hasNext()) { Region bucket =
      * (Region)buckRegionIterator2.next();
-     * assertEquals(Scope.DISTRIBUTED_NO_ACK,
-     * bucket.getAttributes().getScope()); assertEquals(MirrorType.KEYS_VALUES,
+     * assertIndexDetailsEquals(Scope.DISTRIBUTED_NO_ACK,
+     * bucket.getAttributes().getScope()); assertIndexDetailsEquals(MirrorType.KEYS_VALUES,
      * bucket.getAttributes() .getMirrorType()); } pr2.destroyRegion();
      */
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
index 089d723..167b89b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
@@ -265,8 +265,8 @@ public class PartitionedRegionStatsJUnitTest
      * int minRedundantCopies = stats.get("minRedundantCopies").intValue();
      * int avgRedundantCopies = stats.get("avgRedundantCopies").intValue();
      * 
-     * assertEquals(minRedundantCopies, 2); assertEquals(maxRedundantCopies,
-     * 2); assertEquals(avgRedundantCopies, 2);
+     * assertIndexDetailsEquals(minRedundantCopies, 2); assertIndexDetailsEquals(maxRedundantCopies,
+     * 2); assertIndexDetailsEquals(avgRedundantCopies, 2);
      */
   }
   
@@ -486,8 +486,8 @@ public class PartitionedRegionStatsJUnitTest
     assertEquals(numEntries , stats.getInt("dataStoreEntryCount"));
     assertEquals((numEntries - entriesInMem) * entryOverflowSize, diskStats.getNumOverflowBytesOnDisk());
     //Disabled for GEODE-93. numEntriesInVM and numOVerflowOnDisk are incorrect
-//    assertEquals(entriesInMem , diskStats.getNumEntriesInVM());
-//    assertEquals((numEntries - entriesInMem) , diskStats.getNumOverflowOnDisk());
+//    assertIndexDetailsEquals(entriesInMem , diskStats.getNumEntriesInVM());
+//    assertIndexDetailsEquals((numEntries - entriesInMem) , diskStats.getNumOverflowOnDisk());
       assertEquals(stats.getLong("dataStoreBytesInUse"), getMemBytes(pr));
       assertEquals(diskStats.getNumOverflowBytesOnDisk(), getDiskBytes(pr));
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionTestUtilsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionTestUtilsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionTestUtilsDUnitTest.java
index 1daa499..b99024b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionTestUtilsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionTestUtilsDUnitTest.java
@@ -280,7 +280,7 @@ public class PartitionedRegionTestUtilsDUnitTest extends
 //        p.put(key1, val1);
 //        assertFalse(p.localCacheContainsKey(key1));
 //        assertFalse(p.localCacheContainsKey(key2));
-//        assertEquals(val1, p.get(key1));
+//        assertIndexDetailsEquals(val1, p.get(key1));
 //        assertTrue(p.localCacheContainsKey(key1));
 //        assertFalse(p.localCacheContainsKey(key2));
 //
@@ -290,12 +290,12 @@ public class PartitionedRegionTestUtilsDUnitTest extends
 //        assertFalse(lset.contains(key2));
 //        
 //        // test localCacheGet
-//        assertEquals(val1, p.localCacheGet(key1));
+//        assertIndexDetailsEquals(val1, p.localCacheGet(key1));
 //        assertNull(p.localCacheGet(key2));
 //        p.put(key2, val2);
 //        assertNull(p.localCacheGet(key2));
-//        assertEquals(val2, p.get(key2));
-//        assertEquals(val2, p.localCacheGet(key2));
+//        assertIndexDetailsEquals(val2, p.get(key2));
+//        assertIndexDetailsEquals(val2, p.localCacheGet(key2));
 //      }
 //    });
 
@@ -424,7 +424,7 @@ public class PartitionedRegionTestUtilsDUnitTest extends
       val = new Integer(i);
       p.put(key, val);
       // Integer gottenVal = (Integer) p.get(key);
-      // assertEquals("Value for key: " + key + " val " + gottenVal + " wasn't expected " + val, val, gottenVal);
+      // assertIndexDetailsEquals("Value for key: " + key + " val " + gottenVal + " wasn't expected " + val, val, gottenVal);
     }
     
     // Assert that the proper number of keys are placed in each bucket 
@@ -433,7 +433,7 @@ public class PartitionedRegionTestUtilsDUnitTest extends
       assertEquals(s.size(), 1);
       key = (TestPRKey) s.iterator().next();
       assertEquals(i, key.hashCode());
-      // assertEquals(new Integer(i), p.get(key)); 
+      // assertIndexDetailsEquals(new Integer(i), p.get(key));
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
index 7afd5fa..9b1f29e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
@@ -123,7 +123,7 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
   private final SerializableCallable verifyNoTxState = new SerializableCallable() {
     public Object call() throws Exception {
       //TXManagerImpl mgr = getGemfireCache().getTxManager();
-      //assertEquals(0, mgr.hostedTransactionsInProgressForTest());
+      //assertIndexDetailsEquals(0, mgr.hostedTransactionsInProgressForTest());
       final TXManagerImpl mgr = getGemfireCache().getTxManager();
       Wait.waitForCriterion(new WaitCriterion() {
         @Override
@@ -356,14 +356,14 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
       Region.Entry eR = refRegion.getEntry(custId);
       assertNotNull(eC);
       assertNotNull(eR);
-//      assertEquals(1,custRegion.size());
-  //    assertEquals(1,orderRegion.size());
-    //  assertEquals(1,refRegion.size());
+//      assertIndexDetailsEquals(1,custRegion.size());
+  //    assertIndexDetailsEquals(1,orderRegion.size());
+    //  assertIndexDetailsEquals(1,refRegion.size());
       
     } else {
-      //assertEquals(0,custRegion.size());
-      //assertEquals(0,orderRegion.size());
-      //assertEquals(0,refRegion.size());
+      //assertIndexDetailsEquals(0,custRegion.size());
+      //assertIndexDetailsEquals(0,orderRegion.size());
+      //assertIndexDetailsEquals(0,refRegion.size());
       try {
         Region.Entry eC =  custRegion.getEntry(custId);
         assertNull("should have had an EntryNotFoundException:"+eC,eC);
@@ -415,13 +415,13 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
       assertEquals(expectedCust, custRegion.getEntry(custId).getValue());
       /*
       assertNotNull(orderRegion.getEntry(orderId));
-      assertEquals(expectedOrder, orderRegion.getEntry(orderId).getValue());
+      assertIndexDetailsEquals(expectedOrder, orderRegion.getEntry(orderId).getValue());
       
       assertNotNull(orderRegion.getEntry(orderId2));
-      assertEquals(expectedOrder2, orderRegion.getEntry(orderId2).getValue());
+      assertIndexDetailsEquals(expectedOrder2, orderRegion.getEntry(orderId2).getValue());
       
       assertNotNull(orderRegion.getEntry(orderId3));
-      assertEquals(expectedOrder3, orderRegion.getEntry(orderId3).getValue());
+      assertIndexDetailsEquals(expectedOrder3, orderRegion.getEntry(orderId3).getValue());
       */
       assertNotNull(refRegion.getEntry(custId));
       assertEquals(expectedRef, refRegion.getEntry(custId).getValue());
@@ -2236,7 +2236,7 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
         assertEquals(6, rr.entrySet().size());
         assertNotNull(rr.get(custId));
         TXStateProxy tx = mgr.internalSuspend();
-        //assertEquals(getCustIdSet(5), rr.entrySet());
+        //assertIndexDetailsEquals(getCustIdSet(5), rr.entrySet());
         assertEquals(5, rr.entrySet().size());
         assertNull(rr.get(custId));
         mgr.resume(tx);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
index 57aca37..3e51fd9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
@@ -16,7 +16,6 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SimpleDiskRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SimpleDiskRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SimpleDiskRegionJUnitTest.java
index 0300331..6bac28b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SimpleDiskRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SimpleDiskRegionJUnitTest.java
@@ -230,7 +230,7 @@ public class SimpleDiskRegionJUnitTest extends DiskRegionTestingBase
 //     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
 //     DiskRegion dr = ((LocalRegion)region).getDiskRegion();
 //     put100Int();
-//     assertEquals(new Integer(1), region.get(new Integer(1)));
+//     assertIndexDetailsEquals(new Integer(1), region.get(new Integer(1)));
 //     Oplog oplog = dr.getChild();
 //     int id = oplog.getOplogId();
 //     StatisticsFactory factory = dr.getOwner().getCache().getDistributedSystem();
@@ -239,7 +239,7 @@ public class SimpleDiskRegionJUnitTest extends DiskRegionTestingBase
 //     dr.setChild(newOplog);
 //     region.clear();
 //     newOplog = dr.getChild();
-//     assertEquals(null, region.get(new Integer(1)));
+//     assertIndexDetailsEquals(null, region.get(new Integer(1)));
 //     try {
 //       dr.addToOplogSet(id, new File(oplog.getOplogFile()
 //           .getPath()), dr.getNextDir());
@@ -254,7 +254,7 @@ public class SimpleDiskRegionJUnitTest extends DiskRegionTestingBase
 //     oplog.close();
 //     dr.setIsRecovering(true);
 //     dr.basicInitializeOwner();
-//     assertEquals(new Integer(1), region.get(new Integer(1)));
+//     assertIndexDetailsEquals(new Integer(1), region.get(new Integer(1)));
 //     closeDown();
 //   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/VLJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/VLJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/VLJUnitTest.java
index 11aaa77..c2b36f8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/VLJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/VLJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -24,10 +26,9 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
@@ -36,10 +37,10 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * TODO these tests need some work. I don't think they really represent
  * edge cases for this variable length value.
- *
  */
 @Category(UnitTest.class)
-public class VLJUnitTest extends TestCase {
+public class VLJUnitTest {
+
   private ByteArrayOutputStream baos;
   private DataOutputStream dos;
 
@@ -56,71 +57,85 @@ public class VLJUnitTest extends TestCase {
     return new DataInputStream(bais);
   }
 
+  @Test
   public void testZero() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0, createDOS());
     assertEquals(0, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
-  
+
+  @Test
   public void testOne() throws IOException {
     InternalDataSerializer.writeUnsignedVL(1, createDOS());
     assertEquals(1, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
-  
+
+  @Test
   public void testMinusOne() throws IOException {
     InternalDataSerializer.writeUnsignedVL(-1, createDOS());
     assertEquals(-1, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMaxByte() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0x7F, createDOS());
     assertEquals(0x7F, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
-  
+
+  @Test
   public void testMaxNegativeByte() throws IOException {
     InternalDataSerializer.writeUnsignedVL(-0x7F, createDOS());
     assertEquals(-0x7F, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMinShort() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0xFF, createDOS());
     assertEquals(0xFF, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
-  
+
+  @Test
   public void testMinNegativeShort() throws IOException {
     InternalDataSerializer.writeUnsignedVL(-0xFF, createDOS());
     assertEquals(-0xFF, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMaxShort() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0x7fff, createDOS());
     assertEquals(0x7fff, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
-  
+
+  @Test
   public void testMaxNegativeShort() throws IOException {
     InternalDataSerializer.writeUnsignedVL(-0x7fff, createDOS());
     assertEquals(-0x7fff, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMin3Byte() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0xffff, createDOS());
     assertEquals(0xffff, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
-  
+
+  @Test
   public void testMin3Negative() throws IOException {
     InternalDataSerializer.writeUnsignedVL(-0xffff, createDOS());
     assertEquals(-0xffff, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMaxInt() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0x7fffffff, createDOS());
     assertEquals(0x7fffffff, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMinLong() throws IOException {
     InternalDataSerializer.writeUnsignedVL(0x7fffffffL + 1, createDOS());
     assertEquals(0x7fffffffL + 1, InternalDataSerializer.readUnsignedVL(createDIS()));
   }
 
+  @Test
   public void testMaxLong() throws IOException {
     InternalDataSerializer.writeUnsignedVL(Long.MAX_VALUE, createDOS());
     assertEquals(Long.MAX_VALUE, InternalDataSerializer.readUnsignedVL(createDIS()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/FilterByPathJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/FilterByPathJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/FilterByPathJUnitTest.java
index 3fa2024..fade291 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/FilterByPathJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/FilterByPathJUnitTest.java
@@ -16,30 +16,31 @@
  */
 package com.gemstone.gemfire.internal.cache.control;
 
+import static org.junit.Assert.*;
+
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.util.HashSet;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class FilterByPathJUnitTest extends TestCase {
+public class FilterByPathJUnitTest {
+
+  @Test
   public void testDefault() {
     FilterByPath filter = new FilterByPath(null, null);
     assertTrue(filter.include(createRegion("a")));
     assertTrue(filter.include(createRegion("b")));
     assertTrue(filter.include(createRegion("c")));
   }
-  
+
+  @Test
   public void testInclude() {
     HashSet<String> included = new HashSet<String>();
     included.add("a");
@@ -50,6 +51,7 @@ public class FilterByPathJUnitTest extends TestCase {
     assertFalse(filter.include(createRegion("c")));
   }
 
+  @Test
   public void testExclude() {
     HashSet<String> excluded = new HashSet<String>();
     excluded.add("a");
@@ -59,7 +61,8 @@ public class FilterByPathJUnitTest extends TestCase {
     assertFalse(filter.include(createRegion("b")));
     assertTrue(filter.include(createRegion("c")));
   }
-  
+
+  @Test
   public void testBoth() {
     HashSet<String> included = new HashSet<String>();
     included.add("a");
@@ -79,17 +82,18 @@ public class FilterByPathJUnitTest extends TestCase {
     return (Region<?,?>) Proxy.newProxyInstance(contextClassLoader, new Class[] {Region.class}, handler);
   }
 
-  public static class RegionHandler implements InvocationHandler {
+  private static class RegionHandler implements InvocationHandler {
 
     private String name;
     
     public RegionHandler(String name) {
       this.name = "/"+name;
     }
+
+    @Override
     public Object invoke(Object proxy, Method method, Object[] args)
         throws Throwable {
       return name;
     }
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/MemoryThresholdsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/MemoryThresholdsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/MemoryThresholdsJUnitTest.java
index 08b2c80..74dc4fe 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/MemoryThresholdsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/MemoryThresholdsJUnitTest.java
@@ -16,20 +16,17 @@
  */
 package com.gemstone.gemfire.internal.cache.control;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-
 import com.gemstone.gemfire.internal.cache.control.MemoryThresholds.MemoryState;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class MemoryThresholdsJUnitTest {
+
   @Test
   public void testDefaults() {
     MemoryThresholds thresholds = new MemoryThresholds(1000);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
index 26ebc16..6049e14 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
@@ -309,7 +309,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
           Cache cache = getCache();
           ResourceManager manager = cache.getResourceManager();
           RebalanceResults results = doRebalance(false, manager);
-//          assertEquals(113, results.getTotalBucketCreatesCompleted());
+//          assertIndexDetailsEquals(113, results.getTotalBucketCreatesCompleted());
         }
       });
 
@@ -407,14 +407,14 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         //We actually *will* transfer buckets, because that improves
         //the balance
         assertEquals(3, results.getTotalBucketTransfersCompleted());
-//        assertEquals(0, results.getTotalBucketTransferBytes());
+//        assertIndexDetailsEquals(0, results.getTotalBucketTransferBytes());
         Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();
         assertEquals(1, detailSet.size());
         PartitionRebalanceInfo details = detailSet.iterator().next();
         assertEquals(0, details.getBucketCreatesCompleted());
         assertEquals(0, details.getPrimaryTransfersCompleted());
         assertEquals(3, details.getBucketTransfersCompleted());
-//        assertEquals(0, details.getBucketTransferBytes());
+//        assertIndexDetailsEquals(0, details.getBucketTransferBytes());
         if(!simulate) {
           verifyStats(manager, results);
         }
@@ -544,7 +544,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
           }
           assertEquals(2, info.getPrimaryCount());
         }
-//        assertEquals(0, details.getBucketTransferBytes());
+//        assertIndexDetailsEquals(0, details.getBucketTransferBytes());
         if(!simulate) {
           verifyStats(manager, results);
         }
@@ -743,7 +743,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
             assertEquals(2, info.getPrimaryCount());
           }
         }
-        //        assertEquals(0, details.getBucketTransferBytes());
+        //        assertIndexDetailsEquals(0, details.getBucketTransferBytes());
         verifyStats(manager, results);
       }
     });
@@ -1695,7 +1695,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         ResourceManager manager = cache.getResourceManager();
         RebalanceResults results = doRebalance(simulate, manager, INCLUDED, EXCLUDED);
         Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();
-//        assertEquals(3, detailSet.size());
+//        assertIndexDetailsEquals(3, detailSet.size());
         Set<String> names = new HashSet<String>();
         for(PartitionRebalanceInfo details: detailSet) {
           assertEquals(0, details.getBucketCreatesCompleted());
@@ -1825,7 +1825,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         assertEquals(0, results.getTotalBucketCreatesCompleted());
         //We don't know how many primaries will move, it depends on
         //if the move bucket code moves the primary or a redundant bucket
-        //assertEquals(0, results.getTotalPrimaryTransfersCompleted());
+        //assertIndexDetailsEquals(0, results.getTotalPrimaryTransfersCompleted());
         assertEquals(8, results.getTotalBucketTransfersCompleted());
         assertTrue(0 < results.getTotalBucketTransferBytes());
         Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();
@@ -1970,7 +1970,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         assertEquals(0, results.getTotalBucketCreatesCompleted());
         //We don't know how many primaries will move, it depends on
         //if the move bucket code moves the primary or a redundant bucket
-        //assertEquals(0, results.getTotalPrimaryTransfersCompleted());
+        //assertIndexDetailsEquals(0, results.getTotalPrimaryTransfersCompleted());
         assertEquals(8, results.getTotalBucketTransfersCompleted());
         assertTrue(0 < results.getTotalBucketTransferBytes());
         Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();
@@ -2017,7 +2017,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
           assertEquals(4, memberDetails.getPrimaryCount());
           afterSize += memberDetails.getSize();
         }
-        //assertEquals(totalSize.longValue(), afterSize);
+        //assertIndexDetailsEquals(totalSize.longValue(), afterSize);
       }
     };
 
@@ -2038,7 +2038,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         assertEquals(0, results.getTotalBucketCreatesCompleted());
         //We don't know how many primaries will move, it depends on
         //if the move bucket code moves the primary or a redundant bucket
-        //assertEquals(0, results.getTotalPrimaryTransfersCompleted());
+        //assertIndexDetailsEquals(0, results.getTotalPrimaryTransfersCompleted());
         assertEquals(6, results.getTotalBucketTransfersCompleted());
         assertTrue(0 < results.getTotalBucketTransferBytes());
         Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();
@@ -2059,7 +2059,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         assertEquals(4, afterDetails.size());
         for(PartitionMemberInfo memberDetails: afterDetails) {
           assertEquals(6, memberDetails.getBucketCount());
-//          assertEquals(3, memberDetails.getPrimaryCount());
+//          assertIndexDetailsEquals(3, memberDetails.getPrimaryCount());
           afterSize += memberDetails.getSize();
         }
         assertEquals(totalSize, afterSize);
@@ -2083,10 +2083,10 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
           long afterSize = 0;
           for(PartitionMemberInfo memberDetails: details.getPartitionMemberInfo()) {
             assertEquals(6, memberDetails.getBucketCount());
-            //            assertEquals(3, memberDetails.getPrimaryCount());
+            //            assertIndexDetailsEquals(3, memberDetails.getPrimaryCount());
             afterSize += memberDetails.getSize();
           }
-          //assertEquals(totalSize.longValue(), afterSize);
+          //assertIndexDetailsEquals(totalSize.longValue(), afterSize);
         }
       };
 
@@ -2154,7 +2154,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         assertEquals(0, results.getTotalBucketCreatesCompleted());
         //We don't know how many primaries will move, it depends on
         //if the move bucket code moves the primary or a redundant bucket
-        //assertEquals(0, results.getTotalPrimaryTransfersCompleted());
+        //assertIndexDetailsEquals(0, results.getTotalPrimaryTransfersCompleted());
         assertEquals(8, results.getTotalBucketTransfersCompleted());
         assertTrue(0 < results.getTotalBucketTransferBytes());
         Set<PartitionRebalanceInfo> detailSet = results.getPartitionRebalanceDetails();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionNoSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionNoSingleHopDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionNoSingleHopDUnitTest.java
index 9d96d9e..e70d260 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionNoSingleHopDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionNoSingleHopDUnitTest.java
@@ -589,7 +589,7 @@ public class PRClientServerRegionFunctionExecutionNoSingleHopDUnitTest extends P
       // resultListForMember.add(resultIterator.next());
       //
       // for (Object result : resultListForMember) {
-      // assertEquals(Boolean.TRUE, result);
+      // assertIndexDetailsEquals(Boolean.TRUE, result);
       // }
       // }
       for (Object result : resultList) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSelectorNoSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSelectorNoSingleHopDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSelectorNoSingleHopDUnitTest.java
index 482ab9a..1a83608 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSelectorNoSingleHopDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSelectorNoSingleHopDUnitTest.java
@@ -551,7 +551,7 @@ public class PRClientServerRegionFunctionExecutionSelectorNoSingleHopDUnitTest e
       // resultListForMember.add(resultIterator.next());
       //
       // for (Object result : resultListForMember) {
-      // assertEquals(Boolean.TRUE, result);
+      // assertIndexDetailsEquals(Boolean.TRUE, result);
       // }
       // }
       for (Object result : resultList) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSingleHopDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSingleHopDUnitTest.java
index 1f65ad2..bedead7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSingleHopDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRClientServerRegionFunctionExecutionSingleHopDUnitTest.java
@@ -585,7 +585,7 @@ import com.gemstone.gemfire.test.dunit.WaitCriterion;
 //          resultListForMember.add(resultIterator.next());
 //
 //          for (Object result : resultListForMember) {
-//            assertEquals(Boolean.TRUE, result);
+//            assertIndexDetailsEquals(Boolean.TRUE, result);
 //          }
 //        }
         for (Object result : resultList) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
index 55ab928..12a143d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
@@ -399,7 +399,7 @@ public class PRCustomPartitioningDUnitTest extends
               PartitionResolver rr = pr.getPartitionResolver();
               Object o = rr.getRoutingObject(eo);
               Integer i = new Integer(o.hashCode()% totalNumBuckets);
-              //assertEquals(bucketId, bucketId);
+              //assertIndexDetailsEquals(bucketId, bucketId);
               assertEquals(bucketId, i);
             }  //getLogWriter().severe("Key " + key + " found in bucket " + b);
           }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRFunctionExecutionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRFunctionExecutionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRFunctionExecutionDUnitTest.java
index a8df3f6..7d2c0ef 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRFunctionExecutionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRFunctionExecutionDUnitTest.java
@@ -1615,10 +1615,10 @@ public class PRFunctionExecutionDUnitTest extends
         ResultCollector rc2 = dataSet.withFilter(testKeysSet).withArgs(testKeysSet)
             .execute(function);
         List l2 = ((List)rc2.getResult());
-        //assertEquals(pr.getTotalNumberOfBuckets(), l2.size());
+        //assertIndexDetailsEquals(pr.getTotalNumberOfBuckets(), l2.size());
         assertEquals(3, l2.size());
         
-        // assertEquals(pr.getTotalNumberOfBuckets(), l.size());
+        // assertIndexDetailsEquals(pr.getTotalNumberOfBuckets(), l.size());
         HashSet foundVals = new HashSet();
         for (Iterator i = l2.iterator(); i.hasNext();) {
           ArrayList subL = (ArrayList)i.next();
@@ -1908,7 +1908,7 @@ public class PRFunctionExecutionDUnitTest extends
         ResultCollector rc1 = dataSet.withFilter(testKeys).withArgs(Boolean.TRUE)
             .execute(function.getId());
         List l = ((List)rc1.getResult());
-        //assertEquals(pr.getTotalNumberOfBuckets(), l.size());
+        //assertIndexDetailsEquals(pr.getTotalNumberOfBuckets(), l.size());
         assertEquals(1, l.size());
         for (Iterator i = l.iterator(); i.hasNext();) {
           assertEquals(Boolean.TRUE, i.next());
@@ -2256,8 +2256,8 @@ public class PRFunctionExecutionDUnitTest extends
         return Boolean.TRUE;
 //        for (int i=0; i<4; i++) {
 //          List l = ((List)rc1.getResult());
-//          assertEquals(4, l.size());
-//          assertEquals(Boolean.TRUE, l.iterator().next());
+//          assertIndexDetailsEquals(4, l.size());
+//          assertIndexDetailsEquals(Boolean.TRUE, l.iterator().next());
 //        }
 //
 //        return Boolean.TRUE;
@@ -2735,7 +2735,7 @@ public class PRFunctionExecutionDUnitTest extends
             try {
               partitionedregion.put(orderId, order);
               //assertTrue(partitionedregion.containsKey(orderId));
-              //assertEquals(order,partitionedregion.get(orderId));
+              //assertIndexDetailsEquals(order,partitionedregion.get(orderId));
               
             }
             catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRPerformanceTestDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRPerformanceTestDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRPerformanceTestDUnitTest.java
index 4643123..6f34acd 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRPerformanceTestDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRPerformanceTestDUnitTest.java
@@ -199,7 +199,7 @@ public class PRPerformanceTestDUnitTest extends
                 assertNotNull(pr);
                 //pr.put(key, Integer.toString(i));
                 pr.put(key, valueArray);
-                //assertEquals(valueArray, pr.get(key));
+                //assertIndexDetailsEquals(valueArray, pr.get(key));
   
               }
               entries--;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/extension/SimpleExtensionPointJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/extension/SimpleExtensionPointJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/extension/SimpleExtensionPointJUnitTest.java
index 5d0ee56..4fd7837 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/extension/SimpleExtensionPointJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/extension/SimpleExtensionPointJUnitTest.java
@@ -30,7 +30,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link SimpleExtensionPoint}.
- * 
  *
  * @since 8.1
  */
@@ -99,7 +98,6 @@ public class SimpleExtensionPointJUnitTest {
     } catch (NoSuchElementException e) {
       // ignore
     }
-
   }
 
   /**
@@ -183,7 +181,7 @@ public class SimpleExtensionPointJUnitTest {
     public void method1();
   }
 
-  private class MockImpl implements MockInterface, Extensible<MockInterface> {
+  private static class MockImpl implements MockInterface, Extensible<MockInterface> {
 
     private SimpleExtensionPoint<MockInterface> extensionPoint = new SimpleExtensionPoint<SimpleExtensionPointJUnitTest.MockInterface>(this, this);
 
@@ -198,7 +196,7 @@ public class SimpleExtensionPointJUnitTest {
 
   }
 
-  private class MockExtension implements Extension<MockInterface> {
+  private static class MockExtension implements Extension<MockInterface> {
 
     @Override
     public XmlGenerator<MockInterface> getXmlGenerator() {
@@ -209,6 +207,5 @@ public class SimpleExtensionPointJUnitTest {
     public void onCreate(Extensible<MockInterface> source, Extensible<MockInterface> target) {
       throw new UnsupportedOperationException();
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationJUnitTest.java
index 2b81cb4..a9cc187 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationJUnitTest.java
@@ -16,12 +16,13 @@
  */
 package com.gemstone.gemfire.internal.cache.ha;
 
+import static org.junit.Assert.*;
+
 import java.nio.ByteBuffer;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
@@ -31,15 +32,13 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * For client to server messages, the membership id part of event-id is not need
  * to be sent with each event. Also, the threadId and sequenceId need not be
  * sent as long if their value is small. This is a junit test for testing the
- * methods written in <code>EventID</code> class for the above optmization.
+ * methods written in <code>EventID</code> class for the above optimization.
  * For distributed testing for the same , please refer
  * {@link EventIdOptimizationDUnitTest}.
- * 
- * 
  */
 @Category(UnitTest.class)
-public class EventIdOptimizationJUnitTest extends TestCase
-{
+public class EventIdOptimizationJUnitTest {
+
   /** The long id (threadId or sequenceId) having value equivalent to byte */
   private static final long ID_VALUE_BYTE = Byte.MAX_VALUE;
 
@@ -53,164 +52,144 @@ public class EventIdOptimizationJUnitTest extends TestCase
   private static final long ID_VALUE_LONG = Long.MAX_VALUE;
 
   /**
-   * Constructor
-   * 
-   * @param arg0 -
-   *          name
-   */
-  public EventIdOptimizationJUnitTest(String arg0) {
-    super(arg0);
-  }
-
-  /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for byte-byte
    * combination for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForByteByte()
-  {
+  @Test
+  public void testOptimizationForByteByte() {
     int expectedLength = 2 + 1 + 1;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_BYTE, ID_VALUE_BYTE,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_BYTE, ID_VALUE_BYTE,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for
    * short-short combination for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForShortShort()
-  {
+  @Test
+  public void testOptimizationForShortShort() {
     int expectedLength = 2 + 2 + 2;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_SHORT, ID_VALUE_SHORT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_SHORT, ID_VALUE_SHORT,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for int-int
    * combination for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForIntInt()
-  {
+  @Test
+  public void testOptimizationForIntInt() {
     int expectedLength = 2 + 4 + 4;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_INT, ID_VALUE_INT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_INT, ID_VALUE_INT,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for long-long
    * combination for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForLongLong()
-  {
+  @Test
+  public void testOptimizationForLongLong() {
     int expectedLength = 2 + 8 + 8;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_LONG, ID_VALUE_LONG,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_LONG, ID_VALUE_LONG,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for byte-short
    * combinations for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForByteShort()
-  {
+  @Test
+  public void testOptimizationForByteShort() {
     int expectedLength = 2 + 1 + 2;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_BYTE, ID_VALUE_SHORT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_BYTE, ID_VALUE_SHORT,
         expectedLength);
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_SHORT, ID_VALUE_BYTE,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_SHORT, ID_VALUE_BYTE,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for byte-int
    * combinations for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForByteInt()
-  {
+  @Test
+  public void testOptimizationForByteInt() {
     int expectedLength = 2 + 1 + 4;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_BYTE, ID_VALUE_INT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_BYTE, ID_VALUE_INT,
         expectedLength);
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_INT, ID_VALUE_BYTE,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_INT, ID_VALUE_BYTE,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for byte-long
    * combinations for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForByteLong()
-  {
+  @Test
+  public void testOptimizationForByteLong() {
     int expectedLength = 2 + 1 + 8;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_BYTE, ID_VALUE_LONG,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_BYTE, ID_VALUE_LONG,
         expectedLength);
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_LONG, ID_VALUE_BYTE,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_LONG, ID_VALUE_BYTE,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for short-int
    * combinations for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForShortInt()
-  {
+  @Test
+  public void testOptimizationForShortInt() {
     int expectedLength = 2 + 2 + 4;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_SHORT, ID_VALUE_INT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_SHORT, ID_VALUE_INT,
         expectedLength);
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_INT, ID_VALUE_SHORT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_INT, ID_VALUE_SHORT,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for short-long
    * combinations for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForShortLong()
-  {
+  @Test
+  public void testOptimizationForShortLong() {
     int expectedLength = 2 + 2 + 8;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_SHORT, ID_VALUE_LONG,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_SHORT, ID_VALUE_LONG,
         expectedLength);
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_LONG, ID_VALUE_SHORT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_LONG, ID_VALUE_SHORT,
         expectedLength);
   }
 
   /**
-   * Tests the eventId optmization APIs
+   * Tests the eventId optimization APIs
    * <code>EventID#getOptimizedByteArrayForEventID</code> and
    * <code>EventID#readEventIdPartsFromOptmizedByteArray</code> for int-long
    * combinations for threadId and sequenceId values.
-   * 
    */
-  public void testOptmizationForIntLong()
-  {
+  @Test
+  public void testOptimizationForIntLong() {
     int expectedLength = 2 + 4 + 8;
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_INT, ID_VALUE_LONG,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_INT, ID_VALUE_LONG,
         expectedLength);
-    writeReadAndVerifyOpmitizedByteArray(ID_VALUE_LONG, ID_VALUE_INT,
+    writeReadAndVerifyOptimizedByteArray(ID_VALUE_LONG, ID_VALUE_INT,
         expectedLength);
   }
 
@@ -229,9 +208,7 @@ public class EventIdOptimizationJUnitTest extends TestCase
    * @param expectedArrayLength
    *          expected length of the optimized byte-array
    */
-  private void writeReadAndVerifyOpmitizedByteArray(long threadId,
-      long sequenceId, int expectedArrayLength)
-  {
+  private void writeReadAndVerifyOptimizedByteArray(long threadId, long sequenceId, int expectedArrayLength) {
     byte[] array = EventID
         .getOptimizedByteArrayForEventID(threadId, sequenceId);
     assertEquals("optimized byte-array length not as expected",

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
index c7497c3..ba1cd61 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
@@ -314,7 +314,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
           return null;
         }
       };
-      // assertEquals( "key-1",r.getEntry("key-1").getValue());
+      // assertIndexDetailsEquals( "key-1",r.getEntry("key-1").getValue());
       // wait until we
       // have a dead
       // server
@@ -327,7 +327,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
         }
       };
       Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-      // assertEquals( "key-2",r.getEntry("key-2").getValue());
+      // assertIndexDetailsEquals( "key-2",r.getEntry("key-2").getValue());
       
       // wait until we
       // have a dead
@@ -341,7 +341,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
         }
       };
       Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-      // assertEquals( "key-3",r.getEntry("key-3").getValue());
+      // assertIndexDetailsEquals( "key-3",r.getEntry("key-3").getValue());
     }
     catch (Exception ex) {
       Assert.fail("failed while verifyEntries()", ex);
@@ -414,7 +414,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
         }
       };
       Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-      // assertEquals( "key-2",r.getEntry("key-2").getValue());
+      // assertIndexDetailsEquals( "key-2",r.getEntry("key-2").getValue());
 
 
       // wait until
@@ -431,8 +431,8 @@ public class HAGIIDUnitTest extends DistributedTestCase
       Wait.waitForCriterion(ev, 60 * 1000, 200, true);
       
       /*
-       * assertEquals( "value-1",r.getEntry("key-1").getValue()); assertEquals(
-       * "value-2",r.getEntry("key-2").getValue()); assertEquals(
+       * assertIndexDetailsEquals( "value-1",r.getEntry("key-1").getValue()); assertIndexDetailsEquals(
+       * "value-2",r.getEntry("key-2").getValue()); assertIndexDetailsEquals(
        * "value-3",r.getEntry("key-3").getValue());
        */
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQAddOperationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQAddOperationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQAddOperationJUnitTest.java
index 00a906c..1c7b0d0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQAddOperationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQAddOperationJUnitTest.java
@@ -788,7 +788,7 @@ public class HARQAddOperationJUnitTest
     regionqueue.remove();
 
     for (int i = 0; i < numOfThreads; i++) {
-      // assertEquals(numOfPuts,
+      // assertIndexDetailsEquals(numOfPuts,
       // regionqueue.getLastDispatchedSequenceId(new EventID(
       // new byte[] { (byte)i }, i, 1)));
       assertEquals(0, regionqueue.getCurrentCounterSet(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueDUnitTest.java
index 83919f4..ea1ed4a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueueDUnitTest.java
@@ -1095,7 +1095,7 @@ public class HARegionQueueDUnitTest extends DistributedTestCase {
       }
     };
     Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-    // assertEquals(0, hrq.getAvalaibleIds().size());
+    // assertIndexDetailsEquals(0, hrq.getAvalaibleIds().size());
   }
 
   public void testForDuplicateEvents()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/ThreadIdentifierJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/ThreadIdentifierJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/ThreadIdentifierJUnitTest.java
index 9e280fc..66ee622 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/ThreadIdentifierJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/ThreadIdentifierJUnitTest.java
@@ -16,6 +16,9 @@
  */
 package com.gemstone.gemfire.internal.cache.ha;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier.WanType;
@@ -24,8 +27,9 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import junit.framework.TestCase;
 
 @Category(UnitTest.class)
-public class ThreadIdentifierJUnitTest extends TestCase {
+public class ThreadIdentifierJUnitTest {
 
+  @Test
   public void testPutAllId() {
     int id = 42;
     int bucketNumber = 113;
@@ -35,7 +39,8 @@ public class ThreadIdentifierJUnitTest extends TestCase {
     assertTrue(ThreadIdentifier.isPutAllFakeThreadID(putAll));
     assertEquals(42, ThreadIdentifier.getRealThreadID(putAll));
   }
-  
+
+  @Test
   public void testWanId() {
     int id = 42;
     
@@ -69,7 +74,8 @@ public class ThreadIdentifierJUnitTest extends TestCase {
       assertTrue(WanType.PARALLEL.matches(real_tid_with_wan));
     }
   }
-  
+
+  @Test
   public void testWanAndPutAllId() {
     int id = 42;
     int bucketNumber = 113;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
index 867c214..8f556dc 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/locks/TXLockServiceDUnitTest.java
@@ -672,7 +672,7 @@ public class TXLockServiceDUnitTest extends DistributedTestCase {
 //    if (this.lockGrantor == null) {
 //      this.lockGrantor = id;
 //    } else {
-//      assertEquals("assertGrantorIsConsistent failed", lockGrantor, id);
+//      assertIndexDetailsEquals("assertGrantorIsConsistent failed", lockGrantor, id);
 //    }
 //  }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
index 39bb5ea..e08a268 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
@@ -16,11 +16,11 @@
  */
 package com.gemstone.gemfire.internal.cache.partitioned;
 
+import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
 import java.io.IOException;
 
-import org.apache.logging.log4j.Logger;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -34,16 +34,13 @@ import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.partitioned.FetchEntriesMessage.FetchEntriesReplyMessage;
 import com.gemstone.gemfire.internal.cache.partitioned.FetchEntriesMessage.FetchEntriesResponse;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.fake.Fakes;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class FetchEntriesMessageJUnitTest extends TestCase {
-  protected static final Logger logger = LogService.getLogger();
-  GemFireCacheImpl cache;
+public class FetchEntriesMessageJUnitTest {
+
+  private GemFireCacheImpl cache;
   
   private VersionTag createVersionTag(boolean validVersionTag) throws ClassNotFoundException, IOException {
     VersionTag tag = VersionTag.create(cache.getMyId());
@@ -72,7 +69,7 @@ public class FetchEntriesMessageJUnitTest extends TestCase {
   }
   
   @Test
-  public void testProcessChunk() throws IOException, ClassNotFoundException {
+  public void testProcessChunk() throws Exception {
     cache = Fakes.cache();
     PartitionedRegion pr = mock(PartitionedRegion.class);
     InternalDistributedSystem system = cache.getDistributedSystem();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/OfflineMembersDetailsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/OfflineMembersDetailsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/OfflineMembersDetailsJUnitTest.java
index b752d9c..9710c63 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/OfflineMembersDetailsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/OfflineMembersDetailsJUnitTest.java
@@ -25,20 +25,17 @@ import java.net.InetAddress;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.internal.cache.persistence.DiskStoreID;
 import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
-public class OfflineMembersDetailsJUnitTest extends TestCase {
-  
+public class OfflineMembersDetailsJUnitTest {
+
+  @Test
   public void testSerialization() throws Exception {
     Set<PersistentMemberID>[] offlineMembers = new Set[5];
     for(int i =0; i < offlineMembers.length; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
index 35e48bc..97a5fd8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionLoadModelJUnitTest.java
@@ -42,7 +42,6 @@ import org.junit.experimental.categories.Category;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.partition.PartitionMemberInfo;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
 import com.gemstone.gemfire.internal.cache.partitioned.rebalance.BucketOperator.Completion;
 import com.gemstone.gemfire.internal.cache.partitioned.rebalance.CompositeDirector;
 import com.gemstone.gemfire.internal.cache.partitioned.rebalance.PartitionedRegionLoadModel;
@@ -52,12 +51,9 @@ import com.gemstone.gemfire.internal.cache.partitioned.rebalance.SimulatedBucket
 import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
-@SuppressWarnings("synthetic-access")
 @Category(UnitTest.class)
 public class PartitionedRegionLoadModelJUnitTest {
+
   private static final int MAX_MOVES = 5000;
   private static final boolean DEBUG = true;
   
@@ -72,7 +68,6 @@ public class PartitionedRegionLoadModelJUnitTest {
    * This test checks basic redundancy satisfaction. It creates two
    * buckets with low redundancy and 1 bucket with full redundancy and excepts
    * copies of the low redundancy buckets to be made. 
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfaction() throws Exception {
@@ -106,7 +101,6 @@ public class PartitionedRegionLoadModelJUnitTest {
    * This test creates buckets with low redundancy, but only 1 of the buckets
    * is small enough to be copied. The other bucket should be rejected because
    * it is too big..
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfactionWithSizeLimit() throws Exception {
@@ -152,7 +146,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   
   /**
    * This test makes sure we ignore the size limit if requested
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfactionDoNotEnforceLocalMaxMemory() throws Exception {
@@ -188,7 +181,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Tests to make sure that redundancy satisfaction prefers to
    * make redundant copies on members with remote IP addresses.
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfactionPreferRemoteIp() throws Exception {
@@ -256,7 +248,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Tests to make sure that redundancy satisfaction balances
    * between nodes to ensure an even load.
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfactionBalanced() throws Exception {
@@ -286,7 +277,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Tests to make sure that redundancy satisfaction balances
    * between nodes to ensure an even load.
-   * @throws Exception
    */
   @Test
   public void testColocatedRegions() throws Exception {
@@ -367,7 +357,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test that we enforce local max memory on a per region basis
    * IE if one of the regions has a low lmm, it will prevent a bucket move
-   * @throws Exception
    */
   @Test
   public void testColocationEnforceLocalMaxMemory() throws Exception {
@@ -404,7 +393,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test that each region indivually honors it's enforce local max
    * memory flag.
-   * @throws Exception
    */
   @Test
   public void testColocationIgnoreEnforceLocalMaxMemory() throws Exception {
@@ -444,8 +432,7 @@ public class PartitionedRegionLoadModelJUnitTest {
    * Test which illustrates the problem with our greedy algorithm. It doesn't
    * necessarily end up with a balanced result.
    * 
-   *  TODO rebalance - change this test of fix the algorithm?
-   * @throws Exception
+   * TODO rebalance - change this test or fix the algorithm?
    */
   @Ignore
   @Test
@@ -471,13 +458,13 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Tests to make sure that redundancy satisfaction balances
    * between nodes to ensure an even load.
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfactionWithFailures() throws Exception {
     InternalDistributedMember member1 = new InternalDistributedMember(InetAddress.getByName("127.0.0.1"), 1);
     final InternalDistributedMember member2 = new InternalDistributedMember(InetAddress.getByName("127.0.0.1"), 2);
     InternalDistributedMember member3 = new InternalDistributedMember(InetAddress.getByName("127.0.0.1"), 3);
+
     MyBucketOperator op = new MyBucketOperator() {
       @Override
       public void createRedundantBucket(
@@ -489,7 +476,6 @@ public class PartitionedRegionLoadModelJUnitTest {
           super.createRedundantBucket(targetMember, i, colocatedRegionBytes, completion);
         }
       }
-      
     };
     
     PartitionedRegionLoadModel model = new PartitionedRegionLoadModel(op ,1, 4, getAddressComparor(false), Collections.<InternalDistributedMember>emptySet(), null);
@@ -516,7 +502,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test that redundancy satisfation can handle asynchronous failures
    * and complete the job correctly. 
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisfactionWithAsyncFailures() throws Exception {
@@ -568,7 +553,6 @@ public class PartitionedRegionLoadModelJUnitTest {
    * Very basic test of moving primaries. Creates two nodes and four buckets, with a copy
    * of each bucket on both nodes. All of the primaries are on one node. It expects half the
    * primaries to move to the other node.
-   * @throws Exception
    */
   @Test
   public void testMovePrimaries() throws Exception {
@@ -596,7 +580,6 @@ public class PartitionedRegionLoadModelJUnitTest {
    * Test that we can move primaries if failures occur during the move.
    * In this case member2 is bad, so primaries should move
    * to member3 instead.
-   * @throws Exception
    */
   @Test
   public void testMovePrimariesWithFailures() throws Exception {
@@ -636,7 +619,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   
   /**
    * Test of moving primaries when nodes are weighted relative to each other
-   * @throws Exception
    */
   @Test
   public void testMovePrimariesWithWeights() throws Exception {
@@ -676,9 +658,6 @@ public class PartitionedRegionLoadModelJUnitTest {
    * Member1   P  P P  P P  P  X  X  X
    * Member2   R  R R R R  R  P  P  R
    * Member3   X  X  X X X  X  R  R  P
-   * 
-   * 
-   * @throws Exception
    */
   @Test
   public void testPrimaryShuffle() throws Exception {
@@ -710,7 +689,6 @@ public class PartitionedRegionLoadModelJUnitTest {
 
   /**
    * Test a case where we seem to get into an infinite loop while balancing primaries.
-   * @throws Exception
    */
   @Test
   public void testBug39953() throws Exception {
@@ -740,7 +718,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   
   /**
    * Very basic test of moving buckets. Creates two nodes and four buckets, with buckets only on one node. Half of the buckets should move to the other node. 
-   * @throws Exception
    */
   @Test
   public void testMoveBuckets() throws Exception {
@@ -768,15 +745,14 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test that moving buckets will work if there are failures while moving buckets
    * member2 refuses the buckets, so the buckets should move to member3
-   * @throws Exception
    */
   @Test
   public void testMoveBucketsWithFailures() throws Exception {
     InternalDistributedMember member1 = new InternalDistributedMember(InetAddress.getByName("127.0.0.1"), 1);
     final InternalDistributedMember member2 = new InternalDistributedMember(InetAddress.getByName("127.0.0.1"), 2);
     InternalDistributedMember member3 = new InternalDistributedMember(InetAddress.getByName("127.0.0.1"), 3);
+
     MyBucketOperator op = new MyBucketOperator() {
-      
       @Override
       public boolean moveBucket(InternalDistributedMember source,
           InternalDistributedMember target, int id,
@@ -787,6 +763,7 @@ public class PartitionedRegionLoadModelJUnitTest {
         return super.moveBucket(source, target, id, colocatedRegionBytes);
       }
     };
+
     PartitionedRegionLoadModel model = new PartitionedRegionLoadModel(op ,0, 4, getAddressComparor(false), Collections.<InternalDistributedMember>emptySet(), null);
     //Create some imbalanced nodes
     PartitionMemberInfoImpl details1 = buildDetails(member1, 500, 500, new long[] {1,1,1,1}, new long[] {1,1,1,1});
@@ -810,7 +787,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test to make sure that we honor the weight
    * of a node while moving buckets. 
-   * @throws Exception
    */
   @Test
   public void testMoveBucketsWithWeights() throws Exception {
@@ -841,7 +817,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test to make sure we honor the size of buckets when
    * choosing which buckets to move.
-   * @throws Exception
    */
   @Test
   public void testMoveBucketsWithSizes() throws Exception {
@@ -872,7 +847,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test to move buckets with redundancy.
    * Makes sure that buckets and primaries are balanced
-   * @throws Exception
    */
   @Test
   public void testMoveBucketsWithRedundancy() throws Exception {
@@ -914,7 +888,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test to move buckets with some large buckets (to make sure there are no issues with buffer overflow);
    * Makes sure that buckets and primaries are balanced
-   * @throws Exception
    */
   @Test
   public void testMoveLargeBucketsWithRedundancy() throws Exception {
@@ -958,7 +931,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test to make sure that moving buckets
    * honors size restrictions for VMs.
-   * @throws Exception
    */
   @Test
   public void testMoveBucketsWithSizeLimits() throws Exception {
@@ -995,7 +967,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * Test to make sure that moving buckets
    * honors size critical members
-   * @throws Exception
    */
   @Test
   public void testMoveBucketsWithCriticalMember() throws Exception {
@@ -1025,7 +996,8 @@ public class PartitionedRegionLoadModelJUnitTest {
     assertEquals(expectedMoves, new HashSet<Move>(bucketOperator.bucketMoves));
   }
 
-  /** Test to make sure two runs with the same information
+  /**
+   * Test to make sure two runs with the same information
    * perform the same moves. 
    */
   @Test
@@ -1069,9 +1041,9 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * This is more of a simulation than a test
    */
-  @Ignore
+  @Ignore("not a real test")
   @Test
-  public void z_testRandom() throws Exception {
+  public void testRandom() throws Exception {
     long seed = System.nanoTime();
     System.out.println("random seed=" + seed);
     try  {
@@ -1130,7 +1102,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   /**
    * This test makes sure that we rebalance correctly
    * with multiple levels of colocation. See bug #40943
-   * @throws Exception
    */
   @Test
   public void testManyColocatedRegions() throws Exception {
@@ -1163,7 +1134,6 @@ public class PartitionedRegionLoadModelJUnitTest {
   
   /**
    * Test to make sure than redundancy satisfaction ignores offline members
-   * @throws Exception
    */
   @Test
   public void testRedundancySatisficationWithOfflineMembers() throws Exception {
@@ -1173,9 +1143,7 @@ public class PartitionedRegionLoadModelJUnitTest {
     
     PartitionMemberInfoImpl details1 = buildDetails(member1, 200, 200, new long[] {30,0,28,30,23}, new long[] {1,0,1,1,1});
     PartitionMemberInfoImpl details2 = buildDetails(member2, 200, 200, new long[] {0,23,0,0,0}, new long[] {0,1,0,0,0});
-    
-    
-    
+
     //Two buckets have an offline members
     Set<PersistentMemberID> o = Collections.singleton(new PersistentMemberID());
     Set<PersistentMemberID> x = Collections.emptySet();
@@ -1201,8 +1169,6 @@ public class PartitionedRegionLoadModelJUnitTest {
     PartitionMemberInfoImpl details1 = buildDetails(member1, 480, 480, new long[] {1,1,1,1,1,1}, new long[] {1,1,1,1,1,1});
     PartitionMemberInfoImpl details2 = buildDetails(member2, 480, 480, new long[] {0,0,0,0,0,0}, new long[] {0,0,0,0,0,0});
     
-    
-    
     //Each bucket has an offline member
     Set<PersistentMemberID> o = Collections.singleton(new PersistentMemberID());
     Set<PersistentMemberID> x = Collections.emptySet();
@@ -1282,20 +1248,24 @@ public class PartitionedRegionLoadModelJUnitTest {
   
   private static AddressComparor getAddressComparor(final boolean enforceUniqueZones) {
     return new AddressComparor() {
+
+      @Override
       public boolean areSameZone(InternalDistributedMember member1,
           InternalDistributedMember member2) {
         return member1.getInetAddress().equals(member2.getInetAddress());
       }
 
+      @Override
       public boolean enforceUniqueZones() {
         return enforceUniqueZones;
       }
     };
   }
  
-  public static class Create {
-    public final InternalDistributedMember targetMember;
-    public final int bucketId;
+  private static class Create {
+
+    private final InternalDistributedMember targetMember;
+    private final int bucketId;
     
     public Create(InternalDistributedMember targetMember, int bucketId) {
       this.targetMember = targetMember;
@@ -1336,15 +1306,17 @@ public class PartitionedRegionLoadModelJUnitTest {
         return false;
       return true;
     }
-    
+
+    @Override
     public String toString() {
       return "Create[member=" + targetMember+",bucketId=" + bucketId +"]";
     }
   }
   
-  public static class Remove {
-    public final InternalDistributedMember targetMember;
-    public final int bucketId;
+  private static class Remove {
+
+    private final InternalDistributedMember targetMember;
+    private final int bucketId;
     
     public Remove(InternalDistributedMember targetMember, int bucketId) {
       this.targetMember = targetMember;
@@ -1385,21 +1357,24 @@ public class PartitionedRegionLoadModelJUnitTest {
         return false;
       return true;
     }
-    
+
+    @Override
     public String toString() {
       return "Remove[member=" + targetMember+",bucketId=" + bucketId +"]";
     }
   }
   
-  public static class Move {
-    public final InternalDistributedMember sourceMember;
-    public final InternalDistributedMember targetMember;
+  private static class Move {
+
+    private final InternalDistributedMember sourceMember;
+    private final InternalDistributedMember targetMember;
 
     public Move(InternalDistributedMember sourceMember,
         InternalDistributedMember targetMember) {
       this.sourceMember = sourceMember;
       this.targetMember = targetMember;
     }
+
     @Override
     public int hashCode() {
       final int prime = 31;
@@ -1410,6 +1385,7 @@ public class PartitionedRegionLoadModelJUnitTest {
           + ((this.targetMember == null) ? 0 : this.targetMember.hashCode());
       return result;
     }
+
     @Override
     public boolean equals(Object obj) {
       if (this == obj)
@@ -1431,19 +1407,20 @@ public class PartitionedRegionLoadModelJUnitTest {
         return false;
       return true;
     }
-    
+
+    @Override
     public String toString() {
       return "Move[source=" + sourceMember+",target=" + targetMember + "]";
     }
   }
   
-  public static class MyBucketOperator extends SimulatedBucketOperator {
-    public List<Create> creates = new ArrayList<Create>();
-    public List<Remove> removes = new ArrayList<Remove>();
-    public List<Move> primaryMoves = new ArrayList<Move>();
-    public List<Move> bucketMoves = new ArrayList<Move>();
+  private static class MyBucketOperator extends SimulatedBucketOperator {
+
+    private List<Create> creates = new ArrayList<Create>();
+    private List<Remove> removes = new ArrayList<Remove>();
+    private List<Move> primaryMoves = new ArrayList<Move>();
+    private List<Move> bucketMoves = new ArrayList<Move>();
     private MoveType lastMove = null;
-    
 
     @Override
     public void createRedundantBucket(
@@ -1489,18 +1466,18 @@ public class PartitionedRegionLoadModelJUnitTest {
       lastMove = MoveType.REMOVE;
       return true;
     }
-    
-    
   }
   
-  public static class BucketOperatorWithFailures extends MyBucketOperator {
-    List<Completion> pendingSuccesses = new ArrayList<Completion>();
-    List<Completion> pendingFailures = new ArrayList<Completion>();
-    Set<InternalDistributedMember> badMembers = new HashSet<InternalDistributedMember> ();
+  private static class BucketOperatorWithFailures extends MyBucketOperator {
+
+    private List<Completion> pendingSuccesses = new ArrayList<Completion>();
+    private List<Completion> pendingFailures = new ArrayList<Completion>();
+    private Set<InternalDistributedMember> badMembers = new HashSet<InternalDistributedMember> ();
 
     public void addBadMember(InternalDistributedMember member) {
       this.badMembers.add(member);
     }
+
     @Override
     public void createRedundantBucket(InternalDistributedMember targetMember,
         int i, Map<String, Long> colocatedRegionBytes, Completion completion) {
@@ -1508,6 +1485,7 @@ public class PartitionedRegionLoadModelJUnitTest {
         pendingFailures.add(completion);
       } else {
         super.createRedundantBucket(targetMember, i, colocatedRegionBytes, new Completion() {
+
           @Override
           public void onSuccess() {
           }
@@ -1518,11 +1496,11 @@ public class PartitionedRegionLoadModelJUnitTest {
         });
         
         pendingSuccesses.add(completion);
-;      }
+      }
     }
   }
   
-  private static enum MoveType {
+  private enum MoveType {
     CREATE,
     MOVE_PRIMARY,
     MOVE_BUCKET,
@@ -1541,15 +1519,18 @@ public class PartitionedRegionLoadModelJUnitTest {
       this.offlineMembers = offlineMembers;
     }
 
+    @Override
     public Set<PersistentMemberID> getOfflineMembers(int bucketId) {
       return this.offlineMembers;
     }
 
+    @Override
     public void fromData(DataInput in) throws IOException,
         ClassNotFoundException {
       offlineMembers = DataSerializer.readObject(in);
     }
 
+    @Override
     public void toData(DataOutput out) throws IOException {
       DataSerializer.writeObject(offlineMembers, out);
       


[04/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.java
index 5a4f7b3..b02c7b3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.java
@@ -34,7 +34,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Test cases for {@link CacheXmlParser}.
- * 
  *
  * @since 8.1
  */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlVersionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlVersionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlVersionJUnitTest.java
index a51ce2b..15f3161 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlVersionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlVersionJUnitTest.java
@@ -25,7 +25,6 @@ import org.junit.experimental.categories.Category;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
- *
  * @since 8.1
  */
 @Category(UnitTest.class)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolverJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolverJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolverJUnitTest.java
index 2fef8b3..a5431e7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolverJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/GeodeEntityResolverJUnitTest.java
@@ -14,14 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.cache.xmlcache;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import org.junit.Before;
 import org.junit.experimental.categories.Category;
 import org.xml.sax.EntityResolver;
 
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * Unit test for {@link GeodeEntityResolver} and
  * {@link DefaultEntityResolver2}.
@@ -38,10 +38,12 @@ public class GeodeEntityResolverJUnitTest extends AbstractEntityResolverTest {
     entityResolver = new GeodeEntityResolver();
   }
 
+  @Override
   public EntityResolver getEntityResolver() {
     return entityResolver;
   }
 
+  @Override
   public String getSystemId() {
     return systemId;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/PivotalEntityResolverJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/PivotalEntityResolverJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/PivotalEntityResolverJUnitTest.java
index ce4941e..182e81c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/PivotalEntityResolverJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/PivotalEntityResolverJUnitTest.java
@@ -14,18 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.cache.xmlcache;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import org.junit.Before;
 import org.junit.experimental.categories.Category;
 import org.xml.sax.EntityResolver;
 
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * Unit test for {@link PivotalEntityResolver} and
  * {@link DefaultEntityResolver2}.
- * 
  *
  * @since 8.1
  */
@@ -41,10 +40,12 @@ public class PivotalEntityResolverJUnitTest extends AbstractEntityResolverTest {
     entityResolver = new PivotalEntityResolver();
   }
 
+  @Override
   public EntityResolver getEntityResolver() {
     return entityResolver;
   }
 
+  @Override
   public String getSystemId() {
     return systemId;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/jndi/ContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/jndi/ContextJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/jndi/ContextJUnitTest.java
index 2fe0518..6a2545a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/jndi/ContextJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/jndi/ContextJUnitTest.java
@@ -16,8 +16,9 @@
  */
 package com.gemstone.gemfire.internal.jndi;
 
-import java.util.Hashtable;
+import static org.junit.Assert.*;
 
+import java.util.Hashtable;
 import javax.naming.Binding;
 import javax.naming.Context;
 import javax.naming.ContextNotEmptyException;
@@ -28,20 +29,18 @@ import javax.naming.NamingEnumeration;
 import javax.naming.NamingException;
 import javax.naming.NoPermissionException;
 
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-//import com.gemstone.gemfire.internal.jndi.InitialContextFactoryImpl;
-
 /**
  * Tests all basic methods of ContextImpl.
- * 
  */
 @Category(UnitTest.class)
-public class ContextJUnitTest extends TestCase {
+public class ContextJUnitTest {
   
   private Context initialCtx;
   
@@ -50,65 +49,47 @@ public class ContextJUnitTest extends TestCase {
   private Context envCtx;
   
   private Context datasourceCtx;
-  
-  public ContextJUnitTest(String name) {
-    super(name);
-  }
-  
-  protected void setUp() throws Exception {
-    // InitialContextFactoryImpl impl = new InitialContextFactoryImpl();
-    //	impl.setAsInitial();
+
+  @Before
+  public void setUp() throws Exception {
     Hashtable table = new Hashtable();
-    table
-    .put(
-        Context.INITIAL_CONTEXT_FACTORY,
-    "com.gemstone.gemfire.internal.jndi.InitialContextFactoryImpl");
-    //	table.put(Context.URL_PKG_PREFIXES,
-    // "com.gemstone.gemfire.internal.jndi");
+    table.put(Context.INITIAL_CONTEXT_FACTORY, "com.gemstone.gemfire.internal.jndi.InitialContextFactoryImpl");
     initialCtx = new InitialContext(table);
     initialCtx.bind("java:gf/env/datasource/oracle", "a");
     gfCtx = (Context) initialCtx.lookup("java:gf");
     envCtx = (Context) gfCtx.lookup("env");
     datasourceCtx = (Context) envCtx.lookup("datasource");
   }
-  
-  protected void tearDown() throws Exception {
-    
+
+  @After
+  public void tearDown() throws Exception {
     clearContext(initialCtx);
     datasourceCtx = null;
     envCtx = null;
     gfCtx = null;
     initialCtx = null;
-    //InitialContextFactoryImpl.revertSetAsInitial();
   }
   
   /**
    * Removes all entries from the specified context, including subcontexts.
    * 
-   * @param context context ot clear
+   * @param context context to clear
    */
-  private void clearContext(Context context)
-  throws NamingException {
-    
-    for (NamingEnumeration e = context.listBindings(""); e
-    .hasMoreElements();) {
+  private void clearContext(Context context) throws NamingException {
+    for (NamingEnumeration e = context.listBindings(""); e.hasMoreElements();) {
       Binding binding = (Binding) e.nextElement();
       if (binding.getObject() instanceof Context) {
         clearContext((Context) binding.getObject());
       }
       context.unbind(binding.getName());
     }
-    
   }
   
-  /*
+  /**
    * Tests inability to create duplicate subcontexts.
-   * 
-   * @throws NamingException
    */
-  public void testSubcontextCreationOfDuplicates()
-  throws NamingException {
-    
+  @Test
+  public void testSubcontextCreationOfDuplicates() throws NamingException {
     // Try to create duplicate subcontext
     try {
       initialCtx.createSubcontext("java:gf");
@@ -125,14 +106,11 @@ public class ContextJUnitTest extends TestCase {
     }
   }
   
-  /*
+  /**
    * Tests inability to destroy non empty subcontexts.
-   * 
-   * @throws NamingException
    */
-  public void testSubcontextNonEmptyDestruction()
-  throws NamingException {
-    
+  @Test
+  public void testSubcontextNonEmptyDestruction() throws Exception {
     // Bind some object in ejb subcontext
     datasourceCtx.bind("Test", "Object");
     // Attempt to destroy any subcontext
@@ -157,14 +135,11 @@ public class ContextJUnitTest extends TestCase {
     }
   }
   
-  /*
+  /**
    * Tests ability to destroy empty subcontexts.
-   * 
-   * @throws NamingException
    */
-  public void testSubcontextDestruction()
-  throws NamingException {
-    
+  @Test
+  public void testSubcontextDestruction() throws Exception {
     // Create three new subcontexts
     datasourceCtx.createSubcontext("sub1");
     datasourceCtx.createSubcontext("sub2");
@@ -195,13 +170,11 @@ public class ContextJUnitTest extends TestCase {
     }
   }
   
-  /*
-   * Tests inability to invoke methods on destroyed subcontexts. @throws
-   * NamingException
+  /**
+   * Tests inability to invoke methods on destroyed subcontexts.
    */
-  public void testSubcontextInvokingMethodsOnDestroyedContext()
-  throws NamingException {
-    
+  @Test
+  public void testSubcontextInvokingMethodsOnDestroyedContext() throws Exception {
     //Create subcontext and destroy it.
     Context sub = datasourceCtx.createSubcontext("sub4");
     initialCtx
@@ -258,11 +231,11 @@ public class ContextJUnitTest extends TestCase {
     }
   }
   
-  /*
-   * Tests ability to bind name to object. @throws NamingException
+  /**
+   * Tests ability to bind name to object.
    */
-  public void testBindLookup() throws NamingException {
-    
+  @Test
+  public void testBindLookup() throws Exception {
     Object obj1 = new String("Object1");
     Object obj2 = new String("Object2");
     Object obj3 = new String("Object3");
@@ -271,8 +244,7 @@ public class ContextJUnitTest extends TestCase {
     initialCtx.bind("java:gf/env/sub23", null);
     initialCtx.bind("java:gf/env/sub24", obj2);
     // Bind to subcontexts that do not exist
-    initialCtx.bind("java:gf/env/datasource/sub25/sub26",
-        obj3);
+    initialCtx.bind("java:gf/env/datasource/sub25/sub26", obj3);
     
     // Try to lookup
     assertNull(datasourceCtx.lookup("sub21"));
@@ -282,11 +254,11 @@ public class ContextJUnitTest extends TestCase {
     assertSame(datasourceCtx.lookup("sub25/sub26"), obj3);
   }
   
-  /*
-   * Tests ability to unbind names. @throws NamingException
+  /**
+   * Tests ability to unbind names.
    */
-  public void testUnbind() throws NamingException {
-    
+  @Test
+  public void testUnbind() throws Exception {
     envCtx.bind("sub31", null);
     gfCtx.bind("env/ejb/sub32", new String("UnbindObject"));
     // Unbind
@@ -320,45 +292,37 @@ public class ContextJUnitTest extends TestCase {
     }
   }
   
-  /*
+  /**
    * Tests ability to list bindings for a context - specified by name through
    * object reference.
-   * 
-   * @throws NamingException
    */
-  public void testListBindings() throws NamingException {
-    
+  @Test
+  public void testListBindings() throws Exception {
     gfCtx.bind("env/datasource/sub41", "ListBindings1");
     envCtx.bind("sub42", "ListBindings2");
     datasourceCtx.bind("sub43", null);
     
     // Verify bindings for context specified by reference
-    verifyListBindings(envCtx, "", "ListBindings1",
-    "ListBindings2");
+    verifyListBindings(envCtx, "", "ListBindings1", "ListBindings2");
     // Verify bindings for context specified by name
-    verifyListBindings(initialCtx, "java:gf/env",
-        "ListBindings1", "ListBindings2");
+    verifyListBindings(initialCtx, "java:gf/env", "ListBindings1", "ListBindings2");
   }
   
-  private void verifyListBindings(Context c, String name,
-      Object obj1, Object obj2) throws NamingException {
-    
+  private void verifyListBindings(Context c, String name, Object obj1, Object obj2) throws NamingException {
     boolean datasourceFoundFlg = false;
     boolean o2FoundFlg = false;
     boolean datasourceO1FoundFlg = false;
     boolean datasourceNullFoundFlg = false;
     
     // List bindings for the specified context
-    for (NamingEnumeration en = c.listBindings(name); en
-    .hasMore();) {
+    for (NamingEnumeration en = c.listBindings(name); en.hasMore();) {
       Binding b = (Binding) en.next();
       if (b.getName().equals("datasource")) {
         assertEquals(b.getObject(), datasourceCtx);
         datasourceFoundFlg = true;
         
         Context nextCon = (Context) b.getObject();
-        for (NamingEnumeration en1 = nextCon
-            .listBindings(""); en1.hasMore();) {
+        for (NamingEnumeration en1 = nextCon.listBindings(""); en1.hasMore();) {
           Binding b1 = (Binding) en1.next();
           if (b1.getName().equals("sub41")) {
             assertEquals(b1.getObject(), obj1);
@@ -376,12 +340,12 @@ public class ContextJUnitTest extends TestCase {
         o2FoundFlg = true;
       }
     }
-    if (!(datasourceFoundFlg && o2FoundFlg
-        && datasourceO1FoundFlg && datasourceNullFoundFlg)) {
+    if (!(datasourceFoundFlg && o2FoundFlg && datasourceO1FoundFlg && datasourceNullFoundFlg)) {
       fail();
     }
   }
-  
+
+  @Test
   public void testCompositeName() throws Exception {
     ContextImpl c = new ContextImpl();
     Object o = new Object();
@@ -389,9 +353,9 @@ public class ContextJUnitTest extends TestCase {
     c.rebind("/a/b/c/", o);
     assertEquals(c.lookup("a/b/c"), o);
     assertEquals(c.lookup("///a/b/c///"), o);
-    
   }
-  
+
+  @Test
   public void testLookup() throws Exception {
     ContextImpl ctx = new ContextImpl();
     Object obj = new Object();
@@ -400,14 +364,13 @@ public class ContextJUnitTest extends TestCase {
     
     ctx.bind("a", obj);
     assertEquals(obj, ctx.lookup("a"));
-    
   }
   
-  /*
+  /**
    * Tests "getCompositeName" method
    */
+  @Test
   public void testGetCompositeName() throws Exception {
-    
     ContextImpl ctx = new ContextImpl();
     ctx.rebind("a/b/c/d", new Object());
     
@@ -418,15 +381,13 @@ public class ContextJUnitTest extends TestCase {
     
     subCtx = (ContextImpl) ctx.lookup("a/b/c");
     assertEquals("a/b/c", subCtx.getCompoundStringName());
-    
   }
   
-  /*
-   * Tests substitution of '.' with '/' when parsing string names. @throws
-   * NamingException
+  /**
+   * Tests substitution of '.' with '/' when parsing string names.
    */
-  public void testTwoSeparatorNames()
-  throws NamingException {
+  @Test
+  public void testTwoSeparatorNames() throws Exception {
     ContextImpl ctx = new ContextImpl();
     Object obj = new Object();
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
index ea2a1e9..e0c6972 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
@@ -38,7 +38,6 @@ import junitparams.Parameters;
 
 /**
  * Unit tests for LogService
- * 
  */
 @Category(UnitTest.class)
 @RunWith(JUnitParamsRunner.class)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/SortLogFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/SortLogFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/SortLogFileJUnitTest.java
index 24002f9..c8966da 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/SortLogFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/SortLogFileJUnitTest.java
@@ -29,6 +29,7 @@ import java.io.StringWriter;
 import java.util.Date;
 import java.util.Random;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.LogWriter;
@@ -37,7 +38,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 /**
  * Tests the functionality of the {@link SortLogFile} program.
  *
- *
  * @since 3.0
  */
 @Category(UnitTest.class)
@@ -48,8 +48,8 @@ public class SortLogFileJUnitTest {
    * order.  Then it sorts the log file and asserts that the entries
    * are sorted order.
    */
-  @org.junit.Test
-  public void testRandomLog() throws IOException {
+  @Test
+  public void testRandomLog() throws Exception {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintWriter pw = new PrintWriter(new OutputStreamWriter(baos), true);
     LogWriter logger = new RandomLogWriter(pw);
@@ -105,6 +105,7 @@ public class SortLogFileJUnitTest {
      * Ignores <code>date</code> and returns the timestamp for a
      * random date.
      */
+    @Override
     protected String formatDate(Date date) {
       long time = date.getTime() + (random.nextInt(100000) * 1000);
       date = new Date(time);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/AlertAppenderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/AlertAppenderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/AlertAppenderJUnitTest.java
index a1d5e9d..eb0fea3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/AlertAppenderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/AlertAppenderJUnitTest.java
@@ -44,7 +44,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Tests the AlertAppender.
- * 
  */
 @Category(UnitTest.class)
 public class AlertAppenderJUnitTest {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LocalizedMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LocalizedMessageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LocalizedMessageJUnitTest.java
index 5c41ae0..603abf3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LocalizedMessageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LocalizedMessageJUnitTest.java
@@ -30,7 +30,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 /**
  * Tests for LocalizedMessage which bridges our StringId LocalizedStrings for 
  * Log4J2 usage.
- * 
  */
 @Category(UnitTest.class)
 public class LocalizedMessageJUnitTest {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenderJUnitTest.java
index 58abcc8..4a485d8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenderJUnitTest.java
@@ -40,7 +40,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Tests the LogWriterAppender.
- * 
  */
 @Category(UnitTest.class)
 public class LogWriterAppenderJUnitTest {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataTypeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataTypeJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataTypeJUnitTest.java
index fe176e0..00ad891 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataTypeJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataTypeJUnitTest.java
@@ -16,7 +16,9 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import static org.junit.Assert.assertEquals;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -49,12 +51,10 @@ import java.util.concurrent.TimeUnit;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Mockito.*;
-import org.powermock.modules.junit4.PowerMockRunner;
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.Instantiator;
@@ -93,6 +93,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("com.gemstone.gemfire.internal.DataSerializableFixedID:" + ReplyMessage.class.getName(), type);
   }
+
   @Test
   public void testDataSerializableFixedIDShort() throws IOException {
     DataSerializableFixedID value = new ShutdownAllResponse();
@@ -103,14 +104,15 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("com.gemstone.gemfire.internal.DataSerializableFixedID:" + ShutdownAllResponse.class.getName(), type);
   }
+
   @Test
   public void testDataSerializableFixedIDInt() throws IOException, ClassNotFoundException {
     Integer someDSFIDInt = new Integer(1);
-    
+
     PowerMockito.mockStatic(DSFIDFactory.class);
     when(DSFIDFactory.create(eq(someDSFIDInt), any())).thenReturn(someDSFIDInt);
     DSFIDFactory.create(someDSFIDInt, null);
-    
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeByte(DSCODE.DS_FIXED_ID_INT, out);
@@ -119,6 +121,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("com.gemstone.gemfire.internal.DataSerializableFixedID:" + Integer.class.getName(), type);
   }
+
   @Test
   public void testDataSerializableFixedIDClass() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -129,6 +132,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("com.gemstone.gemfire.internal.DataSerializableFixedID:" + Integer.class.getName(), type);
   }
+
   @Test
   public void testNull() throws IOException {
     Object value = null;
@@ -139,6 +143,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("null", type);
   }
+
   @Test
   public void testString() throws IOException {
     String value = "this is a string";
@@ -149,6 +154,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.String", type);
   }
+
   @Test
   public void testNullString() throws IOException {
     String value = null;
@@ -159,6 +165,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.String", type);
   }
+
   @Test
   public void testClass() throws IOException {
     Class<?> value = String.class;
@@ -169,9 +176,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Class", type);
   }
+
   @Test
   public void testDate() throws IOException {
-    Date value = new Date(); 
+    Date value = new Date();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out); // NOT writeDate
@@ -179,6 +187,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.Date", type);
   }
+
   @Test
   public void testFile() throws IOException {
     File value = new File("tmp");
@@ -189,6 +198,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.io.File", type);
   }
+
   @Test
   public void testInetAddress() throws IOException {
     InetAddress value = InetAddress.getLocalHost();
@@ -199,6 +209,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.net.InetAddress", type);
   }
+
   @Test
   public void testBoolean() throws IOException {
     Boolean value = Boolean.TRUE;
@@ -209,6 +220,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Boolean", type);
   }
+
   @Test
   public void testCharacter() throws IOException {
     Character value = Character.valueOf('c');
@@ -219,9 +231,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Character", type);
   }
+
   @Test
   public void testByte() throws IOException {
-    Byte value = Byte.valueOf((byte)0);
+    Byte value = Byte.valueOf((byte) 0);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -229,9 +242,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Byte", type);
   }
+
   @Test
   public void testShort() throws IOException {
-    Short value = Short.valueOf((short)1);
+    Short value = Short.valueOf((short) 1);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -239,6 +253,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Short", type);
   }
+
   @Test
   public void testInteger() throws IOException {
     Integer value = Integer.valueOf(1);
@@ -249,6 +264,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Integer", type);
   }
+
   @Test
   public void testLong() throws IOException {
     Long value = Long.valueOf(1);
@@ -259,9 +275,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Long", type);
   }
+
   @Test
   public void testFloat() throws IOException {
-    Float value = Float.valueOf((float)1.0);
+    Float value = Float.valueOf((float) 1.0);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -269,9 +286,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Float", type);
   }
+
   @Test
   public void testDouble() throws IOException {
-    Double value = Double.valueOf((double)1.0);
+    Double value = Double.valueOf((double) 1.0);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -279,6 +297,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Double", type);
   }
+
   @Test
   public void testByteArray() throws IOException {
     byte[] value = new byte[10];
@@ -289,6 +308,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("byte[]", type);
   }
+
   @Test
   public void testByteArrays() throws IOException {
     byte[][] value = new byte[1][1];
@@ -299,6 +319,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("byte[][]", type);
   }
+
   @Test
   public void testShortArray() throws IOException {
     short[] value = new short[1];
@@ -309,6 +330,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("short[]", type);
   }
+
   @Test
   public void testStringArray() throws IOException {
     String[] value = new String[1];
@@ -319,6 +341,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.String[]", type);
   }
+
   @Test
   public void testIntArray() throws IOException {
     int[] value = new int[1];
@@ -329,6 +352,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("int[]", type);
   }
+
   @Test
   public void testFloatArray() throws IOException {
     float[] value = new float[1];
@@ -339,6 +363,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("float[]", type);
   }
+
   @Test
   public void testLongArray() throws IOException {
     long[] value = new long[1];
@@ -349,6 +374,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("long[]", type);
   }
+
   @Test
   public void testDoubleArray() throws IOException {
     double[] value = new double[1];
@@ -359,6 +385,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("double[]", type);
   }
+
   @Test
   public void testBooleanArray() throws IOException {
     boolean[] value = new boolean[1];
@@ -369,6 +396,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("boolean[]", type);
   }
+
   @Test
   public void testCharArray() throws IOException {
     char[] value = new char[1];
@@ -379,6 +407,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("char[]", type);
   }
+
   @Test
   public void testObjectArray() throws IOException {
     Object[] value = new Object[1];
@@ -389,6 +418,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Object[]", type);
   }
+
   @Test
   public void testArrayList() throws IOException {
     ArrayList<Object> value = new ArrayList<Object>();
@@ -399,6 +429,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.ArrayList", type);
   }
+
   @Test
   public void testLinkedList() throws IOException {
     LinkedList<Object> value = new LinkedList<Object>();
@@ -409,6 +440,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.LinkedList", type);
   }
+
   @Test
   public void testHashSet() throws IOException {
     HashSet<Object> value = new HashSet<Object>();
@@ -419,6 +451,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.HashSet", type);
   }
+
   @Test
   public void testLinkedHashSet() throws IOException {
     LinkedHashSet<Object> value = new LinkedHashSet<Object>();
@@ -429,9 +462,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.LinkedHashSet", type);
   }
+
   @Test
   public void testHashMap() throws IOException {
-    HashMap<Object,Object> value = new HashMap<Object,Object>();
+    HashMap<Object, Object> value = new HashMap<Object, Object>();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -439,9 +473,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.HashMap", type);
   }
+
   @Test
   public void testIdentityHashMap() throws IOException {
-    IdentityHashMap<Object,Object> value = new IdentityHashMap<Object,Object>();
+    IdentityHashMap<Object, Object> value = new IdentityHashMap<Object, Object>();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -449,9 +484,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.IdentityHashMap", type);
   }
+
   @Test
   public void testHashtable() throws IOException {
-    Hashtable<Object,Object> value = new Hashtable<Object,Object>();
+    Hashtable<Object, Object> value = new Hashtable<Object, Object>();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -459,9 +495,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.Hashtable", type);
   }
+
   @Test
   public void testConcurrentHashMap() throws IOException { // java.io.Serializable (broken)
-    ConcurrentHashMap<Object,Object> value = new ConcurrentHashMap<Object,Object>();
+    ConcurrentHashMap<Object, Object> value = new ConcurrentHashMap<Object, Object>();
     value.put("key1", "value1");
     value.put("key2", "value2");
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -472,6 +509,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.io.Serializable:java.util.concurrent.ConcurrentHashMap", type);
   }
+
   @Test
   public void testProperties() throws IOException {
     Properties value = new Properties();
@@ -482,10 +520,11 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.Properties", type);
   }
+
   @Test
   public void testTimeUnit() throws IOException {
     final EnumSet<TimeUnit> optimizedTimeUnits = EnumSet.range(TimeUnit.NANOSECONDS, TimeUnit.SECONDS);
-    for (TimeUnit v: TimeUnit.values()) {
+    for (TimeUnit v : TimeUnit.values()) {
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       DataOutputStream out = new DataOutputStream(baos);
       DataSerializer.writeObject(v, out);
@@ -498,6 +537,7 @@ public class DataTypeJUnitTest {
       }
     }
   }
+
   @Test
   public void testVector() throws IOException {
     Vector<Object> value = new Vector<Object>();
@@ -508,6 +548,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.Vector", type);
   }
+
   @Test
   public void testStack() throws IOException {
     Stack<Object> value = new Stack<Object>();
@@ -518,9 +559,10 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.Stack", type);
   }
+
   @Test
   public void testTreeMap() throws IOException {
-    TreeMap<Object,Object> value = new TreeMap<Object,Object>();
+    TreeMap<Object, Object> value = new TreeMap<Object, Object>();
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     DataSerializer.writeObject(value, out);
@@ -528,6 +570,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.TreeMap", type);
   }
+
   @Test
   public void testTreeSet() throws IOException {
     TreeSet<Object> value = new TreeSet<Object>();
@@ -538,6 +581,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.TreeSet", type);
   }
+
   @Test
   public void testBooleanType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -547,6 +591,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Boolean.class", type);
   }
+
   @Test
   public void testCharacterType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -556,6 +601,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Character.class", type);
   }
+
   @Test
   public void testByteType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -565,6 +611,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Byte.class", type);
   }
+
   @Test
   public void testShortType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -574,6 +621,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Short.class", type);
   }
+
   @Test
   public void testIntegerType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -583,6 +631,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Integer.class", type);
   }
+
   @Test
   public void testLongType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -592,6 +641,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Long.class", type);
   }
+
   @Test
   public void testFloatType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -601,6 +651,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Float.class", type);
   }
+
   @Test
   public void testDoubleType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -610,6 +661,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Double.class", type);
   }
+
   @Test
   public void testVoidType() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -619,6 +671,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.lang.Void.class", type);
   }
+
   //TODO: these tests have to corrected once USER_CLASS, USER_CLASS_2, USER_CLASS_4 are implemented.
   @Test
   public void getDataTypeShouldReturnUserClass() throws IOException {
@@ -631,6 +684,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertThat(type).isEqualTo("DataSerializer: with Id:" + someUserClassId);
   }
+
   @Test
   public void getDataTypeShouldReturnUserClass2() throws IOException {
     short someUserClass2Id = 1;
@@ -642,6 +696,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertThat(type).isEqualTo("DataSerializer: with Id:" + someUserClass2Id);
   }
+
   @Test
   public void getDataTypeShouldReturnUserClass4() throws IOException {
     int someUserClass4Id = 1;
@@ -653,72 +708,76 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertThat(type).isEqualTo("DataSerializer: with Id:" + someUserClass4Id);
   }
+
   @Test
   public void getDataTypeShouldReturnUserDataSeriazliable() throws IOException {
     Instantiator mockInstantiator = PowerMockito.mock(Instantiator.class);
     doReturn(CustId.class).when(mockInstantiator).getInstantiatedClass();
     mockInstantiator.getInstantiatedClass();
-    
+
     int someClassId = 1;
-    
+
     PowerMockito.mockStatic(InternalInstantiator.class);
     when(InternalInstantiator.getClassId(mockInstantiator.getClass())).thenReturn(someClassId);
     when(InternalInstantiator.getInstantiator(someClassId)).thenReturn(mockInstantiator);
-        
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     out.writeByte(DSCODE.USER_DATA_SERIALIZABLE);
     out.writeByte(someClassId);
-    
+
     byte[] bytes = baos.toByteArray();
     String type = DataType.getDataType(bytes);
-    
+
     assertThat(type).isEqualTo("com.gemstone.gemfire.Instantiator:com.gemstone.gemfire.internal.cache.execute.data.CustId");
   }
+
   @Test
   public void getDataTypeShouldReturnUserDataSeriazliable2() throws IOException {
     Instantiator mockInstantiator = PowerMockito.mock(Instantiator.class);
     doReturn(CustId.class).when(mockInstantiator).getInstantiatedClass();
     mockInstantiator.getInstantiatedClass();
-    
+
     int someClassId = 1;
-    
+
     PowerMockito.mockStatic(InternalInstantiator.class);
     when(InternalInstantiator.getClassId(mockInstantiator.getClass())).thenReturn(someClassId);
     when(InternalInstantiator.getInstantiator(someClassId)).thenReturn(mockInstantiator);
-        
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     out.writeByte(DSCODE.USER_DATA_SERIALIZABLE_2);
     out.writeShort(someClassId);
-    
+
     byte[] bytes = baos.toByteArray();
     String type = DataType.getDataType(bytes);
-    
+
     assertThat(type).isEqualTo("com.gemstone.gemfire.Instantiator:com.gemstone.gemfire.internal.cache.execute.data.CustId");
   }
+
   @Test
   public void getDataTypeShouldReturnUserDataSeriazliable4() throws IOException {
     Instantiator mockInstantiator = PowerMockito.mock(Instantiator.class);
     doReturn(CustId.class).when(mockInstantiator).getInstantiatedClass();
     mockInstantiator.getInstantiatedClass();
-    
+
     int someClassId = 1;
-    
+
     PowerMockito.mockStatic(InternalInstantiator.class);
     when(InternalInstantiator.getClassId(mockInstantiator.getClass())).thenReturn(someClassId);
     when(InternalInstantiator.getInstantiator(someClassId)).thenReturn(mockInstantiator);
-        
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     out.writeByte(DSCODE.USER_DATA_SERIALIZABLE_4);
     out.writeInt(someClassId);
-    
+
     byte[] bytes = baos.toByteArray();
     String type = DataType.getDataType(bytes);
-    
+
     assertThat(type).isEqualTo("com.gemstone.gemfire.Instantiator:com.gemstone.gemfire.internal.cache.execute.data.CustId");
   }
+
   @Test
   public void testDataSerializable() throws IOException {
     DataSerializableImpl value = new DataSerializableImpl(new Random());
@@ -729,6 +788,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("com.gemstone.gemfire.DataSerializable:" + DataSerializableImpl.class.getName(), type);
   }
+
   @Test
   public void testSerializable() throws IOException {
     SerializableClass value = new SerializableClass();
@@ -739,9 +799,11 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.io.Serializable:" + SerializableClass.class.getName(), type);
   }
+
   @SuppressWarnings("serial")
   public static class SerializableClass implements Serializable {
   }
+
   @Test
   public void getDataTypeShouldReturnPDXType() throws IOException {
     int somePdxTypeInt = 1;
@@ -766,11 +828,12 @@ public class DataTypeJUnitTest {
 
     assertThat(type).isEqualTo("com.gemstone.gemfire.pdx.PdxInstance:PDXType");
   }
+
   @Test
   public void getDataTypeShouldReturnUnknownIfPDXTypeIsNull() throws IOException {
     int somePdxTypeInt = 1;
     PdxType somePdxType = null;
-    
+
     TypeRegistry mockTypeRegistry = mock(TypeRegistry.class);
     when(mockTypeRegistry.getType(somePdxTypeInt)).thenReturn(somePdxType);
 
@@ -787,8 +850,9 @@ public class DataTypeJUnitTest {
     byte[] bytes = baos.toByteArray();
     String type = DataType.getDataType(bytes);
 
-    assertThat(type).isEqualTo("com.gemstone.gemfire.pdx.PdxInstance: unknown id="+somePdxTypeInt);
+    assertThat(type).isEqualTo("com.gemstone.gemfire.pdx.PdxInstance: unknown id=" + somePdxTypeInt);
   }
+
   @Test
   public void getDataTypeShouldReturnPDXRegistryClosedForPDXTypeWhenCacheIsClosed() throws IOException {
     int somePdxTypeInt = 1;
@@ -805,6 +869,7 @@ public class DataTypeJUnitTest {
 
     assertThat(type).isEqualTo("com.gemstone.gemfire.pdx.PdxInstance:PdxRegistryClosed");
   }
+
   @Test
   public void getDataTypeShouldReturnPDXEnumType() throws IOException {
     int somePdxEnumId = 1;
@@ -829,6 +894,7 @@ public class DataTypeJUnitTest {
 
     assertThat(type).isEqualTo("PdxRegistry/java.lang.Enum:PDXENUM");
   }
+
   @Test
   public void getDataTypeShouldReturnPDXRegistryClosedForEnumTypeWhenCacheIsClosed() throws IOException {
     int someArrayLength = 1;
@@ -845,6 +911,7 @@ public class DataTypeJUnitTest {
 
     assertThat(type).isEqualTo("PdxRegistry/java.lang.Enum:PdxRegistryClosed");
   }
+
   @Test
   public void getDataTypeShouldReturnGemfireEnum() throws IOException {
     PowerMockito.mockStatic(DataSerializer.class);
@@ -858,6 +925,7 @@ public class DataTypeJUnitTest {
 
     assertThat(type).isEqualTo("java.lang.Enum:GEMFIRE_ENUM");
   }
+
   // TODO:PDX_INLINE_ENUM
   @Test
   public void testBigInteger() throws IOException {
@@ -869,6 +937,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.math.BigInteger", type);
   }
+
   @Test
   public void testBigDecimal() throws IOException {
     BigDecimal value = BigDecimal.ZERO;
@@ -879,6 +948,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.math.BigDecimal", type);
   }
+
   @Test
   public void testUUID() throws IOException {
     UUID value = new UUID(Long.MAX_VALUE, Long.MIN_VALUE);
@@ -889,6 +959,7 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.util.UUID", type);
   }
+
   @Test
   public void testSQLTimestamp() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -898,10 +969,11 @@ public class DataTypeJUnitTest {
     String type = DataType.getDataType(bytes);
     assertEquals("java.sql.Timestamp", type);
   }
+
   @Test
   public void testUnknownHeaderType() throws IOException {
     byte unknownType = 0;
-    
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     out.writeByte(unknownType);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DisconnectingOutOfOffHeapMemoryListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DisconnectingOutOfOffHeapMemoryListenerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DisconnectingOutOfOffHeapMemoryListenerJUnitTest.java
index 671a2b5..8e48863 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DisconnectingOutOfOffHeapMemoryListenerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DisconnectingOutOfOffHeapMemoryListenerJUnitTest.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class DisconnectingOutOfOffHeapMemoryListenerJUnitTest {
+
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
   
@@ -81,6 +82,7 @@ public class DisconnectingOutOfOffHeapMemoryListenerJUnitTest {
     listener.outOfOffHeapMemory(ex);
     verify(dm).setRootCause(ex);
   }
+
   @Test
   public void setRootCauseNotCalledWhenGetRootCauseReturnsNonNull() {
     DisconnectingOutOfOffHeapMemoryListener listener = new DisconnectingOutOfOffHeapMemoryListener(ids);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
index 28ff3ac..75eb73a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
@@ -38,6 +38,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class FreeListManagerTest {
+
   static {
     ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
index 71d33ab..1582d31 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
@@ -35,10 +35,10 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Tests LifecycleListener
- * 
  */
 @Category(UnitTest.class)
 public class LifecycleListenerJUnitTest {
+
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
index 582e8b9..4ed83bf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
@@ -36,12 +36,14 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class MemoryAllocatorJUnitTest {
+
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
   private static int round(int multiple, int v) {
     return ((v+multiple-1)/multiple)*multiple;
   }
+
   @Test
   public void testNullGetAllocator() {
     try {
@@ -50,6 +52,7 @@ public class MemoryAllocatorJUnitTest {
     } catch (CacheClosedException expected) {
     }
   }
+
   @Test
   public void testConstructor() {
     try {
@@ -58,6 +61,7 @@ public class MemoryAllocatorJUnitTest {
     } catch (IllegalArgumentException expected) {
     }
   }
+
   @Test
   public void testCreate() {
     System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
@@ -149,6 +153,7 @@ public class MemoryAllocatorJUnitTest {
       assertTrue(stats.isClosed());
     }
   }
+
   @Test
   public void testBasics() {
     int BATCH_SIZE = 1;
@@ -254,6 +259,7 @@ public class MemoryAllocatorJUnitTest {
     MemoryAllocatorImpl.debugLog("test debug log", false);
     MemoryAllocatorImpl.debugLog("test debug log", true);
   }
+
   @Test
   public void testGetLostChunks() {
     SlabImpl slab = new SlabImpl(1024*1024);
@@ -264,6 +270,7 @@ public class MemoryAllocatorJUnitTest {
       MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
+
   @Test
   public void testFindSlab() {
     final int SLAB_SIZE = 1024*1024;
@@ -286,6 +293,7 @@ public class MemoryAllocatorJUnitTest {
       MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
+
   @Test
   public void testValidateAddressAndSize() {
     final int SLAB_SIZE = 1024*1024;
@@ -323,6 +331,7 @@ public class MemoryAllocatorJUnitTest {
       MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
+
   @Test
   public void testMemoryInspection() {
     final int SLAB_SIZE = 1024*1024;
@@ -504,6 +513,7 @@ public class MemoryAllocatorJUnitTest {
   
   long expectedMemoryUsage;
   boolean memoryUsageEventReceived;
+
   @Test
   public void testUsageEventListener() {
     final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
@@ -551,6 +561,7 @@ public class MemoryAllocatorJUnitTest {
       MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
+
   private void checkMcs(ArrayList<StoredObject> mcs) {
     for (StoredObject mc: mcs) {
       assertEquals(8+8, mc.getSize());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
index f0563ad..b998e40 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
@@ -59,7 +59,8 @@ public class MemoryBlockNodeJUnitTest {
     ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
   }
   
-  @Rule public final ProvideSystemProperty myPropertyHasMyValue = new ProvideSystemProperty("gemfire.OFF_HEAP_DO_EXPENSIVE_VALIDATION", "true");
+  @Rule
+  public final ProvideSystemProperty myPropertyHasMyValue = new ProvideSystemProperty("gemfire.OFF_HEAP_DO_EXPENSIVE_VALIDATION", "true");
 
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryInspectorImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryInspectorImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryInspectorImplJUnitTest.java
index b3eb01c..60fa8fc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryInspectorImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryInspectorImplJUnitTest.java
@@ -27,14 +27,12 @@ import java.util.List;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
-
-/**
- * Created by sbawaskar on 1/15/16.
- */
 @Category(UnitTest.class)
 public class MemoryInspectorImplJUnitTest {
+
   private FreeListManager freeList;
   private MemoryInspector inspector;
+
   @Before
   public void setUp() {
     this.freeList = mock(FreeListManager.class);
@@ -55,6 +53,7 @@ public class MemoryInspectorImplJUnitTest {
   public void getAllocatedBlocksBeforeCreateSnapshotReturnsEmptyList() {
     assertTrue(inspector.getAllocatedBlocks().isEmpty());
   }
+
   @Test
   public void getFirstBlockBeforeCreateSnapshotReturnsNull() {
     assertNull(inspector.getFirstBlock());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
index 595e132..7881b08 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
@@ -31,8 +31,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.VMCachedDeserializable;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- */
 @Category(UnitTest.class)
 public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
 
@@ -47,7 +45,6 @@ public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
     OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
 
     ma = MemoryAllocatorImpl.create(ooohml, stats, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
-
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
index 77b1d8d..0d643f1 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
@@ -40,6 +40,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class OffHeapStorageJUnitTest {
+
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
@@ -58,19 +59,23 @@ public class OffHeapStorageJUnitTest {
   public void testParseOffHeapMemorySizeNegative() {
     assertEquals(0, OffHeapStorage.parseOffHeapMemorySize("-1"));
   }
+
   @Test
   public void testParseOffHeapMemorySizeNull() {
     assertEquals(0, OffHeapStorage.parseOffHeapMemorySize(null));
   }
+
   @Test
   public void testParseOffHeapMemorySizeEmpty() {
     assertEquals(0, OffHeapStorage.parseOffHeapMemorySize(""));
   }
+
   @Test
   public void testParseOffHeapMemorySizeBytes() {
     assertEquals(MEGABYTE, OffHeapStorage.parseOffHeapMemorySize("1"));
     assertEquals(Integer.MAX_VALUE * MEGABYTE, OffHeapStorage.parseOffHeapMemorySize("" + Integer.MAX_VALUE));
   }
+
   @Test
   public void testParseOffHeapMemorySizeKiloBytes() {
     try {
@@ -80,32 +85,35 @@ public class OffHeapStorageJUnitTest {
       // Expected
     }
   }
+
   @Test
   public void testParseOffHeapMemorySizeMegaBytes() {
     assertEquals(MEGABYTE, OffHeapStorage.parseOffHeapMemorySize("1m"));
     assertEquals(Integer.MAX_VALUE * MEGABYTE, OffHeapStorage.parseOffHeapMemorySize("" + Integer.MAX_VALUE + "m"));
   }
+
   @Test
   public void testParseOffHeapMemorySizeGigaBytes() {
     assertEquals(GIGABYTE, OffHeapStorage.parseOffHeapMemorySize("1g"));
     assertEquals(Integer.MAX_VALUE * GIGABYTE, OffHeapStorage.parseOffHeapMemorySize("" + Integer.MAX_VALUE + "g"));
   }
+
   @Test
   public void testCalcMaxSlabSize() {
     assertEquals(100, OffHeapStorage.calcMaxSlabSize(100L));
     assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcMaxSlabSize(Long.MAX_VALUE));
     try {
       System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "99");
-      assertEquals(99*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      assertEquals(99 * 1024 * 1024, OffHeapStorage.calcMaxSlabSize(100L * 1024 * 1024));
       assertEquals(88, OffHeapStorage.calcMaxSlabSize(88));
       System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "88m");
-      assertEquals(88*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      assertEquals(88 * 1024 * 1024, OffHeapStorage.calcMaxSlabSize(100L * 1024 * 1024));
       System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "77M");
-      assertEquals(77*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      assertEquals(77 * 1024 * 1024, OffHeapStorage.calcMaxSlabSize(100L * 1024 * 1024));
       System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "1g");
-      assertEquals(1*1024*1024*1024, OffHeapStorage.calcMaxSlabSize(2L*1024*1024*1024));
+      assertEquals(1 * 1024 * 1024 * 1024, OffHeapStorage.calcMaxSlabSize(2L * 1024 * 1024 * 1024));
       System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "1G");
-      assertEquals(1L*1024*1024*1024, OffHeapStorage.calcMaxSlabSize(2L*1024*1024*1024+1));
+      assertEquals(1L * 1024 * 1024 * 1024, OffHeapStorage.calcMaxSlabSize(2L * 1024 * 1024 * 1024 + 1));
       System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "foobarG");
       try {
         OffHeapStorage.calcMaxSlabSize(100);
@@ -119,34 +127,38 @@ public class OffHeapStorageJUnitTest {
       System.clearProperty("gemfire.OFF_HEAP_SLAB_SIZE");
     }
   }
+
   @Test
   public void createOffHeapStorageReturnsNullIfForceLocator() {
     System.setProperty(InternalLocator.FORCE_LOCATOR_DM_TYPE, "true");
     assertEquals(null, OffHeapStorage.createOffHeapStorage(null, 1, null));
   }
+
   @Test
   public void createOffHeapStorageReturnsNullIfMemorySizeIsZero() {
     assertEquals(null, OffHeapStorage.createOffHeapStorage(null, 0, null));
   }
+
   @Test
   public void exceptionIfSlabCountTooSmall() {
     StatisticsFactory statsFactory = mock(StatisticsFactory.class);
     try {
-      OffHeapStorage.createOffHeapStorage(statsFactory, OffHeapStorage.MIN_SLAB_SIZE-1, null);
+      OffHeapStorage.createOffHeapStorage(statsFactory, OffHeapStorage.MIN_SLAB_SIZE - 1, null);
     } catch (IllegalArgumentException expected) {
-      expected.getMessage().equals("The amount of off heap memory must be at least " + OffHeapStorage.MIN_SLAB_SIZE + " but it was set to " + (OffHeapStorage.MIN_SLAB_SIZE-1));
+      expected.getMessage().equals("The amount of off heap memory must be at least " + OffHeapStorage.MIN_SLAB_SIZE + " but it was set to " + (OffHeapStorage.MIN_SLAB_SIZE - 1));
     }
   }
+
   @Test
   public void exceptionIfDistributedSystemNull() {
     StatisticsFactory statsFactory = mock(StatisticsFactory.class);
     try {
-      OffHeapStorage.createOffHeapStorage(statsFactory, OffHeapStorage.MIN_SLAB_SIZE, (DistributedSystem)null);
+      OffHeapStorage.createOffHeapStorage(statsFactory, OffHeapStorage.MIN_SLAB_SIZE, (DistributedSystem) null);
     } catch (IllegalArgumentException expected) {
       expected.getMessage().equals("InternalDistributedSystem is null");
     }
   }
-  
+
   @Test
   public void createOffHeapStorageWorks() {
     StatisticsFactory localStatsFactory = new LocalStatisticsFactory(null);
@@ -160,30 +172,30 @@ public class OffHeapStorageJUnitTest {
   public void testCreateOffHeapStorage() {
     StatisticsFactory localStatsFactory = new LocalStatisticsFactory(null);
     OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
-    MemoryAllocator ma = OffHeapStorage.basicCreateOffHeapStorage(localStatsFactory, 1024*1024, ooohml);
+    MemoryAllocator ma = OffHeapStorage.basicCreateOffHeapStorage(localStatsFactory, 1024 * 1024, ooohml);
     try {
       OffHeapMemoryStats stats = ma.getStats();
       assertNotNull(stats.getStats());
-      assertEquals(1024*1024, stats.getFreeMemory());
-      assertEquals(1024*1024, stats.getMaxMemory());
+      assertEquals(1024 * 1024, stats.getFreeMemory());
+      assertEquals(1024 * 1024, stats.getMaxMemory());
       assertEquals(0, stats.getUsedMemory());
       assertEquals(0, stats.getDefragmentations());
       assertEquals(0, stats.getDefragmentationTime());
       assertEquals(0, stats.getFragmentation());
       assertEquals(1, stats.getFragments());
-      assertEquals(1024*1024, stats.getLargestFragment());
+      assertEquals(1024 * 1024, stats.getLargestFragment());
       assertEquals(0, stats.getObjects());
       assertEquals(0, stats.getReads());
 
       stats.incFreeMemory(100);
-      assertEquals(1024*1024+100, stats.getFreeMemory());
+      assertEquals(1024 * 1024 + 100, stats.getFreeMemory());
       stats.incFreeMemory(-100);
-      assertEquals(1024*1024, stats.getFreeMemory());
+      assertEquals(1024 * 1024, stats.getFreeMemory());
 
       stats.incMaxMemory(100);
-      assertEquals(1024*1024+100, stats.getMaxMemory());
+      assertEquals(1024 * 1024 + 100, stats.getMaxMemory());
       stats.incMaxMemory(-100);
-      assertEquals(1024*1024, stats.getMaxMemory());
+      assertEquals(1024 * 1024, stats.getMaxMemory());
 
       stats.incUsedMemory(100);
       assertEquals(100, stats.getUsedMemory());
@@ -210,8 +222,8 @@ public class OffHeapStorageJUnitTest {
 
       stats.setLargestFragment(100);
       assertEquals(100, stats.getLargestFragment());
-      stats.setLargestFragment(1024*1024);
-      assertEquals(1024*1024, stats.getLargestFragment());
+      stats.setLargestFragment(1024 * 1024);
+      assertEquals(1024 * 1024, stats.getLargestFragment());
 
       boolean originalEnableClockStats = DistributionStats.enableClockStats;
       DistributionStats.enableClockStats = true;
@@ -245,14 +257,14 @@ public class OffHeapStorageJUnitTest {
 
       OutOfOffHeapMemoryException ex = null;
       try {
-        ma.allocate(1024*1024+1);
+        ma.allocate(1024 * 1024 + 1);
         fail("expected OutOfOffHeapMemoryException");
       } catch (OutOfOffHeapMemoryException expected) {
         ex = expected;
       }
       verify(ooohml).outOfOffHeapMemory(ex);
       try {
-        ma.allocate(1024*1024+1);
+        ma.allocate(1024 * 1024 + 1);
         fail("expected OutOfOffHeapMemoryException");
       } catch (OutOfOffHeapMemoryException expected) {
         ex = expected;
@@ -268,16 +280,17 @@ public class OffHeapStorageJUnitTest {
       }
     }
   }
+
   @Test
   public void testCalcSlabCount() {
     final long MSS = OffHeapStorage.MIN_SLAB_SIZE;
-    assertEquals(100, OffHeapStorage.calcSlabCount(MSS*4, MSS*4*100));
-    assertEquals(100, OffHeapStorage.calcSlabCount(MSS*4, (MSS*4*100) + (MSS-1)));
-    assertEquals(101, OffHeapStorage.calcSlabCount(MSS*4, (MSS*4*100) + MSS));
+    assertEquals(100, OffHeapStorage.calcSlabCount(MSS * 4, MSS * 4 * 100));
+    assertEquals(100, OffHeapStorage.calcSlabCount(MSS * 4, (MSS * 4 * 100) + (MSS - 1)));
+    assertEquals(101, OffHeapStorage.calcSlabCount(MSS * 4, (MSS * 4 * 100) + MSS));
     assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, MSS * Integer.MAX_VALUE));
-    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, (MSS * Integer.MAX_VALUE) + MSS-1));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, (MSS * Integer.MAX_VALUE) + MSS - 1));
     try {
-      OffHeapStorage.calcSlabCount(MSS, (((long)MSS) * Integer.MAX_VALUE) + MSS);
+      OffHeapStorage.calcSlabCount(MSS, (((long) MSS) * Integer.MAX_VALUE) + MSS);
       fail("Expected IllegalArgumentException");
     } catch (IllegalArgumentException expected) {
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
index 1d19854..8e7f5b6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
@@ -34,6 +34,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class OffHeapStoredObjectAddressStackJUnitTest {
+
   static {
     ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
index 85c8d4c..0d7ea27 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.offheap;
 
 import static org.assertj.core.api.Assertions.assertThat;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
index 1b5bf26..2974d1a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.offheap;
 
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
index a763e76..8d7f49e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.offheap;
 
 import static org.junit.Assert.assertArrayEquals;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
index 681bec0..c9c9b1f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.offheap;
 
 import static org.junit.Assert.*;


[10/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
index b7b80ac..ed50a3a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
@@ -16,6 +16,23 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.membership;
 
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.*;
+
+import java.util.Properties;
+import java.util.concurrent.RejectedExecutionException;
+
+import org.jgroups.Event;
+import org.jgroups.Message;
+import org.jgroups.protocols.UNICAST3.Header;
+import org.jgroups.protocols.pbcast.NakAckHeader2;
+import org.jgroups.stack.Protocol;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -27,23 +44,6 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.JGroup
 import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.StatRecorder;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import org.jgroups.Event;
-import org.jgroups.Message;
-import org.jgroups.protocols.UNICAST3.Header;
-import org.jgroups.protocols.pbcast.NakAckHeader2;
-import org.jgroups.stack.Protocol;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.Properties;
-import java.util.concurrent.RejectedExecutionException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.*;
 
 /**
  * This class tests the GMS StatRecorder class, which records JGroups
@@ -51,18 +51,24 @@ import static org.mockito.Mockito.*;
  */
 @Category(UnitTest.class)
 public class StatRecorderJUnitTest {
-  Protocol mockDownProtocol, mockUpProtocol;
-  StatRecorder recorder;
-  MyStats stats = new MyStats();
-  Services services;
+
+  private Protocol mockDownProtocol;
+  private Protocol mockUpProtocol;
+  private StatRecorder recorder;
+  private MyStats stats;
+  private Services services;
   
   @Before
-  public void initMocks() throws Exception {
+  public void setUp() throws Exception {
+    stats = new MyStats();
+
     // create a StatRecorder that has mock up/down protocols and stats
     mockDownProtocol = mock(Protocol.class);
     mockUpProtocol = mock(Protocol.class);
+
     services = mock(Services.class);
     when(services.getStatistics()).thenReturn(stats);
+
     recorder = new StatRecorder();
     recorder.setServices(services);
     recorder.setUpProtocol(mockUpProtocol);
@@ -105,7 +111,6 @@ public class StatRecorderJUnitTest {
     when(msg.getHeader(any(Short.class))).thenReturn(Header.createDataHeader(1L, (short)1, true));
     when(msg.size()).thenReturn(150L);
 
-
     // GEODE-1178, the TP protocol may throw a RejectedExecutionException & StatRecorder should retry
     when(mockDownProtocol.down(any(Event.class))).thenThrow(new RejectedExecutionException());
 
@@ -161,8 +166,7 @@ public class StatRecorderJUnitTest {
     assertTrue("mcastRetransmits = " + stats.mcastRetransmits,
         stats.mcastRetransmits == 1);
   }
-  
-  
+
   /**
    * Ensure that the messenger JGroups configuration XML strings contain
    * the statistics recorder protocol
@@ -201,7 +205,8 @@ public class StatRecorderJUnitTest {
     assertTrue(jgroupsConfig.contains("gms.messenger.StatRecorder"));
   }
 
-  static class MyStats extends DummyDMStats {
+  private static class MyStats extends DummyDMStats {
+
     public int ucastMessagesReceived;
     public int ucastMessageBytesReceived;
     public int ucastMessagesSent;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
index 904a206..0e2a617 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
@@ -16,12 +16,11 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.messenger;
 
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.io.IOException;
-import java.net.UnknownHostException;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -33,7 +32,6 @@ import org.jgroups.Message;
 import org.jgroups.Receiver;
 import org.jgroups.stack.IpAddress;
 import org.jgroups.util.UUID;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -44,60 +42,17 @@ import org.mockito.stubbing.Answer;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
-import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class GMSQuorumCheckerJUnitTest {
 
   private InternalDistributedMember[] mockMembers;
-
-  private Services services;
-
   private JChannel channel;
-  
   private JGAddress address;
 
-  private class PingMessageAnswer implements Answer {
-    private int pingCount = 0;
-    private JChannel channel;
-    private GMSPingPonger pingPonger = new GMSPingPonger();
-    private Set<Integer> simulatedPongRespondersByPort;
-    
-    public PingMessageAnswer(JChannel channel, Set<Integer> simulatedPongRespondersByPort) {
-      this.channel = channel;
-      this.simulatedPongRespondersByPort = simulatedPongRespondersByPort;
-    }
-
-    @Override
-    public Object answer(InvocationOnMock invocation) throws Throwable {
-      Object[] args = invocation.getArguments();
-      for (int i = 0; i < args.length; i++) {
-        if (args[i] instanceof Message) {
-          Message msg = (Message) args[i];
-          Object content = null;
-          content = msg.getBuffer();
-          if (content instanceof byte[]) {
-            if (pingPonger.isPingMessage((byte[]) content)) {
-              pingCount++;              
-              if (simulatedPongRespondersByPort.contains(((JGAddress)msg.getDest()).getPort())) {
-                channel.getReceiver().receive(pingPonger.createPongMessage(msg.getDest(), msg.getSrc()));
-              }
-            }
-          }
-        }
-      }
-      return null;
-    }
-    
-    public int getPingCount() {
-      return pingCount;
-    }
-
-  }
-
   @Before
-  public void initMocks() throws UnknownHostException, Exception {
+  public void initMocks() throws Exception {
     mockMembers = new InternalDistributedMember[12];
     for (int i = 0; i < mockMembers.length; i++) {
       mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
@@ -111,22 +66,6 @@ public class GMSQuorumCheckerJUnitTest {
     Mockito.doReturn(address).when(channel).down(any(Event.class));
   }
   
-  private NetView prepareView() throws IOException {
-    return prepareView(mockMembers.length);
-  }
-
-  private NetView prepareView(int numMembers) throws IOException {
-    int viewId = 1;
-    List<InternalDistributedMember> mbrs = new LinkedList<InternalDistributedMember>();
-    for (int i = 0; i < numMembers; i++) {
-      mbrs.add(mockMembers[i]);
-    }
-
-    // prepare the view
-    NetView netView = new NetView(mockMembers[0], viewId, mbrs);
-    return netView;
-  }
-
   @Test
   public void testQuorumCheckerAllRespond() throws Exception {
     NetView view = prepareView();
@@ -140,10 +79,10 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertTrue(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
-    Assert.assertTrue(qc.checkForQuorum(500));
-    Assert.assertSame(qc.getMembershipInfo(), channel);
+    assertTrue(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertTrue(qc.checkForQuorum(500));
+    assertSame(qc.getMembershipInfo(), channel);
   }
   
   @Test
@@ -159,8 +98,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertTrue(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertTrue(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -174,8 +113,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertFalse(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertFalse(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -188,8 +127,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertFalse(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertFalse(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -202,6 +141,7 @@ public class GMSQuorumCheckerJUnitTest {
     for (int i = 0; i < mockMembers.length; i++) {
       pongResponders.add(mockMembers[i].getPort());
     }
+
     //remove 4 servers
     pongResponders.remove(mockMembers[8].getPort());
     pongResponders.remove(mockMembers[9].getPort());
@@ -214,8 +154,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertTrue(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertTrue(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -228,23 +168,24 @@ public class GMSQuorumCheckerJUnitTest {
     for (int i = 0; i < mockMembers.length; i++) {
       pongResponders.add(mockMembers[i].getPort());
     }
+
     //remove 4 servers
     pongResponders.remove(mockMembers[8].getPort());
     pongResponders.remove(mockMembers[9].getPort());
     pongResponders.remove(mockMembers[10].getPort());
     pongResponders.remove(mockMembers[11].getPort());
+
     //remove 1 locator
     pongResponders.remove(mockMembers[1].getPort());
 
-    
     PingMessageAnswer answerer = new PingMessageAnswer(channel, pongResponders);
     Mockito.doAnswer(answerer).when(channel).send(any(Message.class));
 
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertTrue(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertTrue(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -257,12 +198,14 @@ public class GMSQuorumCheckerJUnitTest {
     for (int i = 0; i < mockMembers.length; i++) {
       pongResponders.add(mockMembers[i].getPort());
     }
+
     //remove 5 servers
     pongResponders.remove(mockMembers[7].getPort());
     pongResponders.remove(mockMembers[8].getPort());
     pongResponders.remove(mockMembers[9].getPort());
     pongResponders.remove(mockMembers[10].getPort());
     pongResponders.remove(mockMembers[11].getPort());
+
     //remove locators
     pongResponders.remove(mockMembers[0].getPort());
     pongResponders.remove(mockMembers[1].getPort());
@@ -273,8 +216,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertFalse(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertFalse(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -303,8 +246,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertFalse(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertFalse(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -327,8 +270,8 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertTrue(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertTrue(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
   }
   
   @Test
@@ -352,8 +295,61 @@ public class GMSQuorumCheckerJUnitTest {
     GMSQuorumChecker qc = new GMSQuorumChecker(view, 51, channel);
     qc.initialize();
     boolean quorum = qc.checkForQuorum(500);
-    Assert.assertFalse(quorum);
-    Assert.assertSame(view.getMembers().size(), answerer.getPingCount());
+    assertFalse(quorum);
+    assertSame(view.getMembers().size(), answerer.getPingCount());
+  }
+
+  private NetView prepareView() throws IOException {
+    return prepareView(mockMembers.length);
+  }
+
+  private NetView prepareView(int numMembers) throws IOException {
+    int viewId = 1;
+    List<InternalDistributedMember> mbrs = new LinkedList<InternalDistributedMember>();
+    for (int i = 0; i < numMembers; i++) {
+      mbrs.add(mockMembers[i]);
+    }
+
+    // prepare the view
+    NetView netView = new NetView(mockMembers[0], viewId, mbrs);
+    return netView;
+  }
+
+  private static class PingMessageAnswer implements Answer {
+
+    private int pingCount = 0;
+    private JChannel channel;
+    private GMSPingPonger pingPonger = new GMSPingPonger();
+    private Set<Integer> simulatedPongRespondersByPort;
+
+    public PingMessageAnswer(JChannel channel, Set<Integer> simulatedPongRespondersByPort) {
+      this.channel = channel;
+      this.simulatedPongRespondersByPort = simulatedPongRespondersByPort;
+    }
+
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      Object[] args = invocation.getArguments();
+      for (int i = 0; i < args.length; i++) {
+        if (args[i] instanceof Message) {
+          Message msg = (Message) args[i];
+          Object content = null;
+          content = msg.getBuffer();
+          if (content instanceof byte[]) {
+            if (pingPonger.isPingMessage((byte[]) content)) {
+              pingCount++;
+              if (simulatedPongRespondersByPort.contains(((JGAddress)msg.getDest()).getPort())) {
+                channel.getReceiver().receive(pingPonger.createPongMessage(msg.getDest(), msg.getSrc()));
+              }
+            }
+          }
+        }
+      }
+      return null;
+    }
+
+    public int getPingCount() {
+      return pingCount;
+    }
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
index 952f347..a1d7a2d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
@@ -16,18 +16,11 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.mgr;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -77,6 +70,9 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class GMSMembershipManagerJUnitTest {
+
+  private static final long WAIT_FOR_REPLIES_MILLIS = 2000;
+
   private Services services;
   private ServiceConfig mockConfig;
   private DistributionConfig distConfig;
@@ -88,7 +84,7 @@ public class GMSMembershipManagerJUnitTest {
   private Messenger messenger;
   private JoinLeave joinLeave;
   private Stopper stopper;
-  DistributedMembershipListener listener;
+  private DistributedMembershipListener listener;
   private GMSMembershipManager manager;
   private List<InternalDistributedMember> members;
   private DirectChannel dc;
@@ -107,10 +103,8 @@ public class GMSMembershipManagerJUnitTest {
     nonDefault.put(DistributionConfig.LOCATORS_NAME, "localhost[10344]");
     distConfig = new DistributionConfigImpl(nonDefault);
     distProperties = nonDefault;
-    RemoteTransportConfig tconfig = new RemoteTransportConfig(distConfig,
-        DistributionManager.NORMAL_DM_TYPE);
+    RemoteTransportConfig tconfig = new RemoteTransportConfig(distConfig, DistributionManager.NORMAL_DM_TYPE);
     
-
     mockConfig = mock(ServiceConfig.class);
     when(mockConfig.getDistributionConfig()).thenReturn(distConfig);
     when(mockConfig.getTransport()).thenReturn(tconfig);
@@ -257,7 +251,6 @@ public class GMSMembershipManagerJUnitTest {
     manager.handleOrDeferSurpriseConnect(surpriseMember);
     assertEquals(5, manager.getStartupEvents().size());
 
-    
     // process a new view after we finish joining but before event processing has started
     manager.isJoining = false;
     mockMembers[4].setVmViewId(4);
@@ -296,26 +289,6 @@ public class GMSMembershipManagerJUnitTest {
     verify(listener).memberSuspect(suspectMember, mockMembers[0], "testing");
   }
   
-  /**
-   * Some tests require a DirectChannel mock
-   */
-  private void setUpDirectChannelMock() throws Exception {
-    dc = mock(DirectChannel.class);
-    when(dc.send(any(GMSMembershipManager.class), any(mockMembers.getClass()), any(DistributionMessage.class), anyInt(), anyInt()))
-      .thenReturn(100);
-
-    manager.start();
-    manager.started();
-    
-    manager.setDirectChannel(dc);
-
-    NetView view = new NetView(myMemberId, 1, members);
-    manager.installView(view);
-    when(joinLeave.getView()).thenReturn(view);
-    
-    manager.startEventProcessing();
-  }
-
   @Test
   public void testDirectChannelSend() throws Exception {
     setUpDirectChannelMock();
@@ -357,9 +330,8 @@ public class GMSMembershipManagerJUnitTest {
     when(stopper.cancelInProgress()).thenReturn("stopping for test");
     try {
       manager.directChannelSend(recipients, m, null);
-      throw new RuntimeException("expected directChannelSend to throw an exception");
-    } catch (DistributedSystemDisconnectedException e) {
-      // expected
+      fail("expected directChannelSend to throw an exception");
+    } catch (DistributedSystemDisconnectedException expected) {
     }
   }
   
@@ -427,10 +399,29 @@ public class GMSMembershipManagerJUnitTest {
     mbrs.add(mockMembers[0]);
     ReplyProcessor21 rp = new ReplyProcessor21(dm, mbrs);
     rp.enableSevereAlertProcessing();
-    boolean result = rp.waitForReplies(2000);
+    boolean result = rp.waitForReplies(WAIT_FOR_REPLIES_MILLIS);
     assertFalse(result);  // the wait should have timed out
     verify(healthMonitor, atLeastOnce()).checkIfAvailable(isA(InternalDistributedMember.class), isA(String.class), isA(Boolean.class));
   }
-  
-}
 
+  /**
+   * Some tests require a DirectChannel mock
+   */
+  private void setUpDirectChannelMock() throws Exception {
+    dc = mock(DirectChannel.class);
+    when(dc.send(any(GMSMembershipManager.class), any(mockMembers.getClass()), any(DistributionMessage.class), anyInt(), anyInt()))
+            .thenReturn(100);
+
+    manager.start();
+    manager.started();
+
+    manager.setDirectChannel(dc);
+
+    NetView view = new NetView(myMemberId, 1, members);
+    manager.installView(view);
+    when(joinLeave.getView()).thenReturn(view);
+
+    manager.startEventProcessing();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
index 8723505..5fd0f27 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
@@ -158,8 +158,8 @@ public class TcpServerBackwardCompatDUnitTest extends DistributedTestCase {
           TcpServer.OLDTESTVERSION -= 100;
           TcpServer.getGossipVersionMapForTestOnly().put(TcpServer.TESTVERSION, Version.CURRENT_ORDINAL);
           TcpServer.getGossipVersionMapForTestOnly().put(TcpServer.OLDTESTVERSION, Version.GFE_57.ordinal());
-//          assertEquals("Gossip Version and Test version are not same", TcpServer.GOSSIPVERSION, TcpServer.TESTVERSION);
-//          assertEquals("Previous Gossip Version and Test version are not same", TcpServer.OLDGOSSIPVERSION, TcpServer.OLDTESTVERSION);
+//          assertIndexDetailsEquals("Gossip Version and Test version are not same", TcpServer.GOSSIPVERSION, TcpServer.TESTVERSION);
+//          assertIndexDetailsEquals("Previous Gossip Version and Test version are not same", TcpServer.OLDGOSSIPVERSION, TcpServer.OLDTESTVERSION);
 
           Locator.startLocatorAndDS(port1, logFile1, props);
 
@@ -198,8 +198,8 @@ public class TcpServerBackwardCompatDUnitTest extends DistributedTestCase {
           TcpServer.OLDTESTVERSION -= 100;
           TcpServer.getGossipVersionMapForTestOnly().put(TcpServer.TESTVERSION, Version.CURRENT_ORDINAL);
           TcpServer.getGossipVersionMapForTestOnly().put(TcpServer.OLDTESTVERSION, Version.GFE_57.ordinal());
-//          assertEquals("Gossip Version and Test version are not same", TcpServer.GOSSIPVERSION, TcpServer.TESTVERSION);
-//          assertEquals("Previous Gossip Version and Test version are not same", TcpServer.OLDGOSSIPVERSION, TcpServer.OLDTESTVERSION);
+//          assertIndexDetailsEquals("Gossip Version and Test version are not same", TcpServer.GOSSIPVERSION, TcpServer.TESTVERSION);
+//          assertIndexDetailsEquals("Previous Gossip Version and Test version are not same", TcpServer.OLDGOSSIPVERSION, TcpServer.OLDTESTVERSION);
 
           Locator.startLocatorAndDS(port0, logFile0, props);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerJUnitTest.java
index 4594889..9b1b8d3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServerJUnitTest.java
@@ -39,11 +39,9 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.PoolStatHelper;
 import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
 import com.gemstone.gemfire.internal.AvailablePort;
-//import com.gemstone.org.jgroups.stack.GossipClient;
-//import com.gemstone.org.jgroups.stack.IpAddress;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class TcpServerJUnitTest {
   
   protected/*GemStoneAddition*/ InetAddress localhost;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
index 80c92e6..234d032 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
@@ -18,97 +18,74 @@ package com.gemstone.gemfire.internal;
 
 import static org.junit.Assert.*;
 
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
-
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.Map;
-import java.util.Properties;
-import junit.framework.TestCase;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class AbstractConfigJUnitTest extends TestCase  {
+public class AbstractConfigJUnitTest {
 
 	@Test
-	public void testDisplayPropertyValue(){
-		AbstractConfigTestClass actc=new AbstractConfigTestClass();
-        Method method=null;
-        try{
-            method=actc.getClass().getSuperclass().getDeclaredMethod("okToDisplayPropertyValue",String.class);
-            method.setAccessible(true);
-            assertFalse((Boolean) method.invoke(actc, "password"));
-            assertFalse((Boolean)method.invoke(actc,"cluster-ssl-truststore-password"));
-            assertTrue((Boolean) method.invoke(actc, "cluster-ssl-enabled"));
-            assertFalse((Boolean)method.invoke(actc,"gateway-ssl-truststore-password"));
-            assertFalse((Boolean)method.invoke(actc,"server-ssl-keystore-password"));
-            assertTrue((Boolean) method.invoke(actc, "ssl-enabled"));
-            assertTrue((Boolean)method.invoke(actc,"conserve-sockets"));
-            assertFalse((Boolean)method.invoke(actc,"javax.net.ssl.keyStorePassword"));
-            assertFalse((Boolean)method.invoke(actc,"javax.net.ssl.keyStoreType"));
-            assertFalse((Boolean)method.invoke(actc,"sysprop-value"));
-        } catch (NoSuchMethodException e) {
-            e.printStackTrace();
-        } catch (InvocationTargetException e) {
-            e.printStackTrace();
-        } catch (IllegalAccessException e) {
-            e.printStackTrace();
-        } catch(Exception e){
-            e.printStackTrace();
-        }
-    }
-
-}
-
-class AbstractConfigTestClass extends AbstractConfig{
+	public void testDisplayPropertyValue() throws Exception {
+		AbstractConfigTestClass actc = new AbstractConfigTestClass();
+		Method method = actc.getClass().getSuperclass().getDeclaredMethod("okToDisplayPropertyValue", String.class);
+		method.setAccessible(true);
+		assertFalse((Boolean) method.invoke(actc, "password"));
+		assertFalse((Boolean) method.invoke(actc, "cluster-ssl-truststore-password"));
+		assertTrue((Boolean) method.invoke(actc, "cluster-ssl-enabled"));
+		assertFalse((Boolean) method.invoke(actc, "gateway-ssl-truststore-password"));
+		assertFalse((Boolean) method.invoke(actc, "server-ssl-keystore-password"));
+		assertTrue((Boolean) method.invoke(actc, "ssl-enabled"));
+		assertTrue((Boolean) method.invoke(actc, "conserve-sockets"));
+		assertFalse((Boolean) method.invoke(actc, "javax.net.ssl.keyStorePassword"));
+		assertFalse((Boolean) method.invoke(actc, "javax.net.ssl.keyStoreType"));
+		assertFalse((Boolean) method.invoke(actc, "sysprop-value"));
+	}
 
+	private static class AbstractConfigTestClass extends AbstractConfig {
 
-	@Override
-	protected Map getAttDescMap() {
-		return null;
-	}
+		@Override
+		protected Map getAttDescMap() {
+			return null;
+		}
 
-	@Override
-	protected Map<String, ConfigSource> getAttSourceMap() {
-		return null;
-	}
+		@Override
+		protected Map<String, ConfigSource> getAttSourceMap() {
+			return null;
+		}
 
-	@Override
-	public Object getAttributeObject(String attName) {
-		return null;
-	}
+		@Override
+		public Object getAttributeObject(String attName) {
+			return null;
+		}
 
-	@Override
-	public void setAttributeObject(String attName, Object attValue, ConfigSource source) {
+		@Override
+		public void setAttributeObject(String attName, Object attValue, ConfigSource source) {
 
-	}
+		}
 
-	@Override
-	public boolean isAttributeModifiable(String attName) {
-		return false;
-	}
+		@Override
+		public boolean isAttributeModifiable(String attName) {
+			return false;
+		}
 
-	@Override
-	public Class getAttributeType(String attName) {
-		return null;
-	}
+		@Override
+		public Class getAttributeType(String attName) {
+			return null;
+		}
 
-	@Override
-	public String[] getAttributeNames() {
-		return new String[0];
-	}
+		@Override
+		public String[] getAttributeNames() {
+			return new String[0];
+		}
 
-	@Override
-	public String[] getSpecificAttributeNames() {
-		return new String[0];
+		@Override
+		public String[] getSpecificAttributeNames() {
+			return new String[0];
+		}
 	}
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/ArrayEqualsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ArrayEqualsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ArrayEqualsJUnitTest.java
deleted file mode 100644
index 714a7fe..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/ArrayEqualsJUnitTest.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.internal;
-
-import static org.junit.Assert.fail;
-
-import java.util.Properties;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-
-/**
- * See bug 52093.
- * Make sure that the contents of arrays are
- * used to test equality on regions ops.
- */
-@Category(UnitTest.class)
-public class ArrayEqualsJUnitTest {
-  private GemFireCacheImpl createCache() {
-    Properties props = new Properties();
-    props.setProperty("locators", "");
-    props.setProperty("mcast-port", "0");
-    GemFireCacheImpl result = (GemFireCacheImpl) new CacheFactory(props).create();
-    return result;
-  }
-  private void closeCache(GemFireCacheImpl gfc) {
-    gfc.close();
-  }
-  
-  private void doOps(Region r) {
-    {
-      byte[] bytesValue = new byte[]{1,2,3,4};
-      r.put("bytesValue", bytesValue.clone());
-      if (r.replace("bytesValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("bytesValue", bytesValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("bytesValue", bytesValue.clone());
-      if (r.remove("bytesValue", "")) fail("expected remove to fail");
-      if (!r.remove("bytesValue", bytesValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      boolean[] booleanValue = new boolean[]{true,false,true,false};
-      r.put("booleanValue", booleanValue.clone());
-      if (r.replace("booleanValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("booleanValue", booleanValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("booleanValue", booleanValue.clone());
-      if (r.remove("booleanValue", "")) fail("expected remove to fail");
-      if (!r.remove("booleanValue", booleanValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      short[] shortValue = new short[]{1,2,3,4};
-      r.put("shortValue", shortValue.clone());
-      if (r.replace("shortValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("shortValue", shortValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("shortValue", shortValue.clone());
-      if (r.remove("shortValue", "")) fail("expected remove to fail");
-      if (!r.remove("shortValue", shortValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      char[] charValue = new char[]{1,2,3,4};
-      r.put("charValue", charValue.clone());
-      if (r.replace("charValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("charValue", charValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("charValue", charValue.clone());
-      if (r.remove("charValue", "")) fail("expected remove to fail");
-      if (!r.remove("charValue", charValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      int[] intValue = new int[]{1,2,3,4};
-      r.put("intValue", intValue.clone());
-      if (r.replace("intValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("intValue", intValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("intValue", intValue.clone());
-      if (r.remove("intValue", "")) fail("expected remove to fail");
-      if (!r.remove("intValue", intValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      long[] longValue = new long[]{1,2,3,4};
-      r.put("longValue", longValue.clone());
-      if (r.replace("longValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("longValue", longValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("longValue", longValue.clone());
-      if (r.remove("longValue", "")) fail("expected remove to fail");
-      if (!r.remove("longValue", longValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      float[] floatValue = new float[]{1,2,3,4};
-      r.put("floatValue", floatValue.clone());
-      if (r.replace("floatValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("floatValue", floatValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("floatValue", floatValue.clone());
-      if (r.remove("floatValue", "")) fail("expected remove to fail");
-      if (!r.remove("floatValue", floatValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      double[] doubleValue = new double[]{1,2,3,4};
-      r.put("doubleValue", doubleValue.clone());
-      if (r.replace("doubleValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("doubleValue", doubleValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("doubleValue", doubleValue.clone());
-      if (r.remove("doubleValue", "")) fail("expected remove to fail");
-      if (!r.remove("doubleValue", doubleValue)) {
-        fail("expected remove to happen");
-      }
-    }
-    {
-      Object[] oaValue = new Object[]{new byte[]{1,2,3,4},new short[]{1,2,3,4},new int[]{1,2,3,4}, "hello sweet world!"};
-      r.put("oaValue", oaValue);
-      Object[] deepCloneOaValue = new Object[]{new byte[]{1,2,3,4},new short[]{1,2,3,4},new int[]{1,2,3,4}, "hello sweet world!"};
-      if (r.replace("oaValue", "", "")) fail("expected replace to fail");
-      if (!r.replace("oaValue", deepCloneOaValue, "")) {
-        fail("expected replace to happen");
-      }
-      r.put("oaValue", oaValue);
-      if (r.remove("oaValue", "")) fail("expected remove to fail");
-      if (!r.remove("oaValue", deepCloneOaValue)) {
-        fail("expected remove to happen");
-      }
-    }
-  }
-  
-  @Test
-  public void testPartition() {
-    GemFireCacheImpl gfc = createCache();
-    try {
-      Region r = gfc.createRegionFactory(RegionShortcut.PARTITION).create("ArrayEqualsJUnitTestPartitionRegion");
-      doOps(r);
-    } finally {
-      closeCache(gfc);
-    }
-  }
-  @Test
-  public void testLocal() {
-    GemFireCacheImpl gfc = createCache();
-    try {
-      Region r = gfc.createRegionFactory(RegionShortcut.LOCAL).create("ArrayEqualsJUnitTestLocalRegion");
-      doOps(r);
-    } finally {
-      closeCache(gfc);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
index 25184de..706702d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
@@ -17,11 +17,6 @@
 package com.gemstone.gemfire.internal;
 
 import static org.junit.Assert.*;
-import static org.junit.Assume.*;
-
-import com.gemstone.gemfire.admin.internal.InetAddressUtil;
-import com.gemstone.gemfire.internal.lang.SystemUtils;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 import java.io.IOException;
 import java.net.InetAddress;
@@ -32,9 +27,13 @@ import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.admin.internal.InetAddressUtil;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
 /**
+ * multicast availability is tested in JGroupsMessengerJUnitTest
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class AvailablePortJUnitTest {
   
   private ServerSocket socket;
@@ -69,5 +68,4 @@ public class AvailablePortJUnitTest {
     assertFalse(AvailablePort.isPortAvailable(port, AvailablePort.SOCKET));
   }
   
-  // multicast availability is tested in JGroupsMessengerJUnitTest
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderJUnitTest.java
deleted file mode 100755
index 2a1db52..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassPathLoaderJUnitTest.java
+++ /dev/null
@@ -1,1261 +0,0 @@
-/*
- * 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.internal;
-
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.Enumeration;
-import java.util.Vector;
-
-import junit.framework.TestCase;
-
-import org.apache.bcel.Constants;
-import org.apache.bcel.classfile.JavaClass;
-import org.apache.bcel.generic.ClassGen;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- * Test the {@link ClassPathLoader}.
- * 
- * @since 6.5.1.4
- */
-@Category(UnitTest.class)
-public class ClassPathLoaderJUnitTest extends TestCase {
-
-  private static final int TEMP_FILE_BYTES_COUNT = 256;
-  private static final int GENERATED_CLASS_BYTES_COUNT = 362;
-
-  private static final String ORIGINAL_EXCLUDE_TCCL_VALUE;
-  
-  private boolean deleteExtDir = false;
-  private boolean deleteLibDir = false;
-
-  static {
-    ORIGINAL_EXCLUDE_TCCL_VALUE = System.getProperty(ClassPathLoader.EXCLUDE_TCCL_PROPERTY, Boolean
-        .toString(ClassPathLoader.EXCLUDE_TCCL_DEFAULT_VALUE));
-
-    exploreClassLoaders(); // optional output for developer's convenience
-  }
-
-  private volatile File tempFile;
-  private volatile File tempFile2;
-
-  public ClassPathLoaderJUnitTest(String name) {
-    super(name);
-  }
-
-  @Override
-  public void setUp() throws Exception {
-    System.setProperty(ClassPathLoader.EXCLUDE_TCCL_PROPERTY, "false");
-
-    File workingDir = new File(System.getProperty("user.dir")).getAbsoluteFile();
-    this.tempFile = File.createTempFile("ClassPathLoaderJUnitTest.", null, workingDir);
-    FileOutputStream fos = new FileOutputStream(this.tempFile);
-    fos.write(new byte[TEMP_FILE_BYTES_COUNT]);
-    fos.close();
-
-    this.tempFile2 = File.createTempFile("ClassPathLoaderJUnitTest.", null, workingDir);
-    fos = new FileOutputStream(this.tempFile2);
-    fos.write(new byte[TEMP_FILE_BYTES_COUNT]);
-    fos.close();
-  }
-
-  @Override
-  public void tearDown() throws Exception {
-    System.setProperty(ClassPathLoader.EXCLUDE_TCCL_PROPERTY, ORIGINAL_EXCLUDE_TCCL_VALUE);
-    
-    // these deletions fail on windows
-//    if (this.deleteLibDir) {
-//      FileUtil.delete(ClassPathLoader.EXT_LIB_DIR.getParentFile());
-//    } else {
-//      if (this.deleteExtDir) {
-//        FileUtil.delete(ClassPathLoader.EXT_LIB_DIR);
-//      } else {
-//        FileUtil.delete(new File(ClassPathLoader.EXT_LIB_DIR, "ClassPathLoaderJUnit1.jar"));
-//        FileUtil.delete(new File(ClassPathLoader.EXT_LIB_DIR, "cplju"));
-//      }
-//    }
-    
-    assertTrue(this.tempFile.delete());
-  }
-
-  /**
-   * Verifies that {@link ClassPathLoader#getLatest()} is always initialized and returns a <tt>ClassPathLoader</tt>
-   * instance.
-   */
-  public void testLatestExists() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testLatestExists");
-
-    assertNotNull(ClassPathLoader.getLatest());
-  }
-
-  /**
-   * Verifies that {@link ClassPathLoader#getLatest()} throws <tt>ClassNotFoundException</tt> when class does not exist.
-   */
-  public void testForNameThrowsClassNotFoundException() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testForNameThrowsClassNotFoundException");
-
-    try {
-      String classToLoad = "com.nowhere.DoesNotExist";
-      ClassPathLoader.getLatest().forName(classToLoad);
-      fail();
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-  }
-
-  /**
-   * Verifies that {@link ClassPathLoader#getLatest()} finds and loads class via
-   * <tt>Class.forName(String, boolean, ClassLoader)</tt> when class does exist.
-   */
-  public void testForName() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testForName");
-
-    String classToLoad = "com.gemstone.gemfire.internal.classpathloaderjunittest.DoesExist";
-    Class<?> clazz = ClassPathLoader.getLatest().forName(classToLoad);
-    assertNotNull(clazz);
-  }
-
-  /**
-   * Verifies that {@link ClassPathLoader#getLatest()} can actually <tt>getResource</tt> when it exists.
-   */
-  public void testGetResource() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResource");
-
-    String resourceToGet = "com/gemstone/gemfire/internal/classpathloaderjunittest/DoesExist.class";
-    URL url = ClassPathLoader.getLatest().getResource(resourceToGet);
-    assertNotNull(url);
-
-    InputStream is = url != null ? url.openStream() : null;
-    assertNotNull(is);
-
-    int totalBytesRead = 0;
-    byte[] input = new byte[256];
-
-    BufferedInputStream bis = new BufferedInputStream(is);
-    for (int bytesRead = bis.read(input); bytesRead > -1;) {
-      totalBytesRead += bytesRead;
-      bytesRead = bis.read(input);
-    }
-    bis.close();
-
-    // if the following fails then maybe javac changed and DoesExist.class
-    // contains other than 374 bytes of data... consider updating this test
-    assertEquals(GENERATED_CLASS_BYTES_COUNT, totalBytesRead);
-  }
-
-  /**
-   * Verifies that {@link ClassPathLoader#getLatest()} can actually <tt>getResources</tt> when it exists.
-   */
-  public void testGetResources() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResources");
-
-    String resourceToGet = "com/gemstone/gemfire/internal/classpathloaderjunittest/DoesExist.class";
-    Enumeration<URL> urls = ClassPathLoader.getLatest().getResources(resourceToGet);
-    assertNotNull(urls);
-    assertTrue(urls.hasMoreElements());
-
-    URL url = urls.nextElement();
-    InputStream is = url != null ? url.openStream() : null;
-    assertNotNull(is);
-
-    int totalBytesRead = 0;
-    byte[] input = new byte[256];
-
-    BufferedInputStream bis = new BufferedInputStream(is);
-    for (int bytesRead = bis.read(input); bytesRead > -1;) {
-      totalBytesRead += bytesRead;
-      bytesRead = bis.read(input);
-    }
-    bis.close();
-
-    // if the following fails then maybe javac changed and DoesExist.class
-    // contains other than 374 bytes of data... consider updating this test
-    assertEquals(GENERATED_CLASS_BYTES_COUNT, totalBytesRead);
-  }
-
-  /**
-   * Verifies that {@link ClassPathLoader#getLatest()} can actually <tt>getResourceAsStream</tt> when it exists.
-   */
-  public void testGetResourceAsStream() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceAsStream");
-
-    String resourceToGet = "com/gemstone/gemfire/internal/classpathloaderjunittest/DoesExist.class";
-    InputStream is = ClassPathLoader.getLatest().getResourceAsStream(resourceToGet);
-    assertNotNull(is);
-
-    int totalBytesRead = 0;
-    byte[] input = new byte[256];
-
-    BufferedInputStream bis = new BufferedInputStream(is);
-    for (int bytesRead = bis.read(input); bytesRead > -1;) {
-      totalBytesRead += bytesRead;
-      bytesRead = bis.read(input);
-    }
-    bis.close();
-
-    // if the following fails then maybe javac changed and DoesExist.class
-    // contains other than 374 bytes of data... consider updating this test
-    assertEquals(GENERATED_CLASS_BYTES_COUNT, totalBytesRead);
-  }
-
-  /**
-   * Verifies that the {@link GeneratingClassLoader} works and always generates the named class. This is a control which
-   * ensures that tests depending on <tt>GeneratingClassLoader</tt> are valid.
-   */
-  public void testGeneratingClassLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGeneratingClassLoader");
-
-    ClassLoader gcl = new GeneratingClassLoader();
-    String classToLoad = "com.nowhere.TestGeneratingClassLoader";
-
-    Class<?> clazz = gcl.loadClass(classToLoad);
-    assertNotNull(clazz);
-    assertEquals(classToLoad, clazz.getName());
-
-    Object obj = clazz.newInstance();
-    assertEquals(clazz.getName(), obj.getClass().getName());
-
-    try {
-      Class.forName(classToLoad);
-      fail("Should have thrown ClassNotFoundException");
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-
-    Class<?> clazzForName = Class.forName(classToLoad, true, gcl);
-    assertNotNull(clazzForName);
-    assertEquals(clazz, clazzForName);
-
-    Object objForName = clazzForName.newInstance();
-    assertEquals(classToLoad, objForName.getClass().getName());
-  }
-
-  /**
-   * Verifies that custom loader is used to load class.
-   */
-  public void testForNameWithCustomLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testForNameWithCustomLoader");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-
-    String classToLoad = "com.nowhere.TestForNameWithCustomLoader";
-    Class<?> clazz = dcl.forName(classToLoad);
-    assertNotNull(clazz);
-    assertEquals(classToLoad, clazz.getName());
-
-    Object obj = clazz.newInstance();
-    assertEquals(classToLoad, obj.getClass().getName());
-  }
-
-  /**
-   * Verifies that {@link Class#forName(String, boolean, ClassLoader)} used with {@link ClassPathLoader} works as
-   * expected with named object arrays, while {@link ClassLoader#loadClass(String)} throws ClassNotFoundException for
-   * named object arrays.
-   */
-  public void testForNameWithObjectArray() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testForNameWithObjectArray");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-
-    String classToLoad = "[Ljava.lang.String;";
-    Class<?> clazz = null;
-    clazz = dcl.forName(classToLoad);
-    assertEquals(classToLoad, clazz.getName());
-  }
-
-  /**
-   * Verifies that TCCL finds the class when {@link Class#forName(String, boolean, ClassLoader)} uses
-   * {@link ClassPathLoader}.
-   */
-  public void testForNameWithTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testForNameWithTCCL");
-
-    final ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    final String classToLoad = "com.nowhere.TestForNameWithTCCL";
-
-    try {
-      dcl.forName(classToLoad);
-      fail("Should have thrown ClassNotFoundException");
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // ensure that TCCL is only CL that can find this class
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      Class<?> clazz = dcl.forName(classToLoad);
-      assertNotNull(clazz);
-      Object instance = clazz.newInstance();
-      assertNotNull(instance);
-      assertEquals(classToLoad, instance.getClass().getName());
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-
-    try {
-      dcl.forName(classToLoad);
-      fail("Should have thrown ClassNotFoundException");
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-
-  }
-
-  /**
-   * Verifies that the {@link NullClassLoader} works and never finds the named class. This is a control which ensures
-   * that tests depending on <tt>NullClassLoader</tt> are valid.
-   */
-  public void testNullClassLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testNullClassLoader");
-
-    ClassLoader cl = new NullClassLoader();
-    String classToLoad = "java.lang.String";
-
-    try {
-      Class.forName(classToLoad, true, cl);
-      fail();
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-
-    String resourceToGet = "java/lang/String.class";
-
-    URL url = cl.getResource(resourceToGet);
-    assertNull(url);
-
-    InputStream is = cl.getResourceAsStream(resourceToGet);
-    assertNull(is);
-  }
-
-  /**
-   * Verifies that the {@link SimpleClassLoader} works and finds classes that the parent can find. This is a control
-   * which ensures that tests depending on <tt>SimpleClassLoader</tt> are valid.
-   */
-  public void testSimpleClassLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testSimpleClassLoader");
-
-    ClassLoader cl = new SimpleClassLoader(getClass().getClassLoader());
-    String classToLoad = "java.lang.String";
-
-    Class<?> clazz = Class.forName(classToLoad, true, cl);
-    assertNotNull(clazz);
-
-    String resourceToGet = "java/lang/String.class";
-
-    URL url = cl.getResource(resourceToGet);
-    assertNotNull(url);
-
-    InputStream is = cl.getResourceAsStream(resourceToGet);
-    assertNotNull(is);
-  }
-
-  /**
-   * Verifies that the {@link BrokenClassLoader} is broken and always throws errors. This is a control which ensures
-   * that tests depending on <tt>BrokenClassLoader</tt> are valid.
-   */
-  public void testBrokenClassLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testBrokenClassLoader");
-
-    ClassLoader cl = new BrokenClassLoader();
-
-    String classToLoad = "java.lang.String";
-    try {
-      Class.forName(classToLoad, true, cl);
-      fail();
-    } catch (ClassNotFoundException e) {
-      throw e;
-    } catch (BrokenError expected) {
-      // Expected
-    }
-
-    String resourceToGet = "java/lang/String.class";
-    try {
-      cl.getResource(resourceToGet);
-      fail();
-    } catch (BrokenError expected) {
-      // Expected
-    }
-    try {
-      cl.getResourceAsStream(resourceToGet);
-      fail();
-    } catch (BrokenError expected) {
-      // Expected
-    }
-  }
-
-  /**
-   * Verifies that the {@link BrokenClassLoader} is broken and always throws errors even when used as a TCCL from
-   * {@link ClassPathLoader}. This is primarily a control which ensures that tests depending on
-   * <tt>BrokenClassLoader</tt> are valid, but it also verifies that TCCL is included by default by
-   * <tt>ClassPathLoader</tt>.
-   */
-  public void testBrokenTCCLThrowsErrors() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testBrokenTCCLThrowsErrors");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl.addOrReplace(new NullClassLoader());
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set the TCCL to throw errors
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      String classToLoad = "java.lang.String";
-      try {
-        dcl.forName(classToLoad);
-        fail();
-      } catch (ClassNotFoundException e) {
-        throw e;
-      } catch (BrokenError expected) {
-        // Expected
-      }
-
-      String resourceToGet = "java/lang/String.class";
-      try {
-        dcl.getResource(resourceToGet);
-        fail();
-      } catch (BrokenError expected) {
-        // Expected
-      }
-
-      try {
-        dcl.getResourceAsStream(resourceToGet);
-        fail();
-      } catch (BrokenError expected) {
-        // Expected
-      }
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that the class classloader or system classloader will find the class or resource. Parent is a
-   * {@link NullClassLoader} while the TCCL is an excluded {@link BrokenClassLoader}.
-   */
-  public void testEverythingWithDefaultLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testEverythingWithDefaultLoader");
-
-    // create DCL such that parent cannot find anything
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
-    dcl.addOrReplace(new NullClassLoader());
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set the TCCL to never find anything
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      String classToLoad = "java.lang.String";
-      Class<?> clazz = dcl.forName(classToLoad);
-      assertNotNull(clazz);
-
-      String resourceToGet = "java/lang/String.class";
-      URL url = dcl.getResource(resourceToGet);
-      assertNotNull(url);
-      InputStream is = dcl.getResourceAsStream(resourceToGet);
-      assertNotNull(is);
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that the 3rd custom loader will find the class. Parent cannot find it and TCCL is broken. This verifies
-   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
-   */
-  public void testForNameWithMultipleCustomLoaders() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testForNameWithMultipleCustomLoaders");
-
-    // create DCL such that the 3rd loader should find the class
-    // first custom loader becomes parent which won't find anything
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    final GeneratingClassLoader generatingClassLoader = new GeneratingClassLoader();
-    dcl = dcl.addOrReplace(generatingClassLoader);
-    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
-    dcl = dcl.addOrReplace(new NullClassLoader());
-
-    String classToLoad = "com.nowhere.TestForNameWithMultipleCustomLoaders";
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set TCCL to throw errors which makes sure we find before checking TCCL
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      Class<?> clazz = dcl.forName(classToLoad);
-      assertNotNull(clazz);
-      assertEquals(classToLoad, clazz.getName());
-      assertTrue("Class not loaded by a GeneratingClassLoader.", clazz.getClassLoader() instanceof GeneratingClassLoader);
-      assertEquals("Class not loaded by generatingClassLoader.", generatingClassLoader, clazz.getClassLoader());
-
-      Object obj = clazz.newInstance();
-      assertEquals(classToLoad, obj.getClass().getName());
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that the 3rd custom loader will get the resource. Parent cannot find it and TCCL is broken. This verifies
-   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
-   */
-  public void testGetResourceWithMultipleCustomLoaders() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceWithMultipleCustomLoaders");
-
-    // create DCL such that the 3rd loader should find the resource
-    // first custom loader becomes parent which won't find anything
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
-    dcl = dcl.addOrReplace(new NullClassLoader());
-
-    String resourceToGet = "com/nowhere/testGetResourceWithMultipleCustomLoaders.rsc";
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set TCCL to throw errors which makes sure we find before checking TCCL
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      URL url = dcl.getResource(resourceToGet);
-      assertNotNull(url);
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that the 3rd custom loader will get the resources. Parent cannot find it and TCCL is broken. This verifies
-   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
-   */
-  public void testGetResourcesWithMultipleCustomLoaders() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceWithMultipleCustomLoaders");
-
-    // create DCL such that the 3rd loader should find the resource
-    // first custom loader becomes parent which won't find anything
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-    dcl = dcl.addOrReplace(new GeneratingClassLoader2());
-    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
-    dcl = dcl.addOrReplace(new NullClassLoader());
-
-    String resourceToGet = "com/nowhere/testGetResourceWithMultipleCustomLoaders.rsc";
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set TCCL to throw errors which makes sure we find before checking TCCL
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      Enumeration<URL> urls = dcl.getResources(resourceToGet);
-      assertNotNull(urls);
-      assertTrue(urls.hasMoreElements());
-      
-      URL url = urls.nextElement();
-      assertNotNull(url);
-      
-      // Should find two with unique URLs
-      assertTrue("Did not find all resources.", urls.hasMoreElements());
-      URL url2 = urls.nextElement();
-      assertNotNull(url2);
-      assertTrue("Resource URLs should be unique.", !url.equals(url2));
-      
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that the 3rd custom loader will get the resource. Parent cannot find it and TCCL is broken. This verifies
-   * that all custom loaders are checked and that the custom loaders are all checked before TCCL.
-   */
-  public void testGetResourceAsStreamWithMultipleCustomLoaders() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceAsStreamWithMultipleCustomLoaders");
-
-    // create DCL such that the 3rd loader should find the resource
-    // first custom loader becomes parent which won't find anything
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
-    dcl = dcl.addOrReplace(new NullClassLoader());
-
-    String resourceToGet = "com/nowhere/testGetResourceAsStreamWithMultipleCustomLoaders.rsc";
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set TCCL to throw errors which makes sure we find before checking TCCL
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      InputStream is = dcl.getResourceAsStream(resourceToGet);
-      assertNotNull(is);
-      is.close();
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that setting <tt>excludeThreadContextClassLoader</tt> to true will indeed exclude the TCCL.
-   */
-  public void testExcludeTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testExcludeTCCL");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
-
-    String classToLoad = "com.nowhere.TestExcludeTCCL";
-
-    try {
-      dcl.forName(classToLoad);
-      fail("Should have thrown ClassNotFoundException");
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // ensure that TCCL is only CL that can find this class
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      dcl.forName(classToLoad);
-      fail("Should have thrown ClassNotFoundException");
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-
-  }
-
-  /**
-   * Verifies that <tt>getResource</tt> works with custom loader from {@link ClassPathLoader}.
-   */
-  public void testGetResourceWithCustomLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceWithCustomLoader");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-
-    String resourceToGet = "com/nowhere/testGetResourceWithCustomLoader.rsc";
-    URL url = dcl.getResource(resourceToGet);
-    assertNotNull(url);
-
-    InputStream is = url != null ? url.openStream() : null;
-    assertNotNull(is);
-
-    int totalBytesRead = 0;
-    byte[] input = new byte[128];
-
-    BufferedInputStream bis = new BufferedInputStream(is);
-    for (int bytesRead = bis.read(input); bytesRead > -1;) {
-      totalBytesRead += bytesRead;
-      bytesRead = bis.read(input);
-    }
-    bis.close();
-
-    assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
-  }
-
-  /**
-   * Verifies that <tt>getResources</tt> works with custom loader from {@link ClassPathLoader}.
-   */
-  public void testGetResourcesWithCustomLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceWithCustomLoader");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-
-    String resourceToGet = "com/nowhere/testGetResourceWithCustomLoader.rsc";
-    Enumeration<URL> urls = dcl.getResources(resourceToGet);
-    assertNotNull(urls);
-    assertTrue(urls.hasMoreElements());
-
-    URL url = urls.nextElement();
-    InputStream is = url != null ? url.openStream() : null;
-    assertNotNull(is);
-
-    int totalBytesRead = 0;
-    byte[] input = new byte[128];
-
-    BufferedInputStream bis = new BufferedInputStream(is);
-    for (int bytesRead = bis.read(input); bytesRead > -1;) {
-      totalBytesRead += bytesRead;
-      bytesRead = bis.read(input);
-    }
-    bis.close();
-
-    assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
-  }
-
-  /**
-   * Verifies that <tt>getResourceAsStream</tt> works with custom loader from {@link ClassPathLoader}.
-   */
-  public void testGetResourceAsStreamWithCustomLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceAsStreamWithCustomLoader");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-
-    String resourceToGet = "com/nowhere/testGetResourceAsStreamWithCustomLoader.rsc";
-    InputStream is = dcl.getResourceAsStream(resourceToGet);
-    assertNotNull(is);
-
-    int totalBytesRead = 0;
-    byte[] input = new byte[128];
-
-    BufferedInputStream bis = new BufferedInputStream(is);
-    for (int bytesRead = bis.read(input); bytesRead > -1;) {
-      totalBytesRead += bytesRead;
-      bytesRead = bis.read(input);
-    }
-    bis.close();
-
-    assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
-  }
-
-  /**
-   * Verifies that <tt>getResource</tt> works with TCCL from {@link ClassPathLoader}.
-   */
-  public void testGetResourceWithTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceWithTCCL");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-
-    String resourceToGet = "com/nowhere/testGetResourceWithTCCL.rsc";
-    assertNull(dcl.getResource(resourceToGet));
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      URL url = dcl.getResource(resourceToGet);
-      assertNotNull(url);
-
-      InputStream is = url.openStream();
-      assertNotNull(is);
-
-      int totalBytesRead = 0;
-      byte[] input = new byte[128];
-
-      BufferedInputStream bis = new BufferedInputStream(is);
-      for (int bytesRead = bis.read(input); bytesRead > -1;) {
-        totalBytesRead += bytesRead;
-        bytesRead = bis.read(input);
-      }
-      bis.close();
-
-      assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that <tt>getResources</tt> works with TCCL from {@link ClassPathLoader}.
-   */
-  public void testGetResourcesWithTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceWithTCCL");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-
-    String resourceToGet = "com/nowhere/testGetResourceWithTCCL.rsc";
-    Enumeration<URL> urls = dcl.getResources(resourceToGet);
-    assertNotNull(urls);
-    assertFalse(urls.hasMoreElements());
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      urls = dcl.getResources(resourceToGet);
-      assertNotNull(urls);
-
-      URL url = urls.nextElement();
-      InputStream is = url.openStream();
-      assertNotNull(is);
-
-      int totalBytesRead = 0;
-      byte[] input = new byte[128];
-
-      BufferedInputStream bis = new BufferedInputStream(is);
-      for (int bytesRead = bis.read(input); bytesRead > -1;) {
-        totalBytesRead += bytesRead;
-        bytesRead = bis.read(input);
-      }
-      bis.close();
-
-      assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that <tt>getResourceAsStream</tt> works with TCCL from {@link ClassPathLoader}.
-   */
-  public void testGetResourceAsStreamWithTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceAsStreamWithTCCL");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-
-    String resourceToGet = "com/nowhere/testGetResourceAsStreamWithTCCL.rsc";
-    assertNull(dcl.getResourceAsStream(resourceToGet));
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // ensure that TCCL is only CL that can find this resource
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      InputStream is = dcl.getResourceAsStream(resourceToGet);
-      assertNotNull(is);
-
-      int totalBytesRead = 0;
-      byte[] input = new byte[128];
-
-      BufferedInputStream bis = new BufferedInputStream(is);
-      for (int bytesRead = bis.read(input); bytesRead > -1;) {
-        totalBytesRead += bytesRead;
-        bytesRead = bis.read(input);
-      }
-      bis.close();
-
-      assertEquals(TEMP_FILE_BYTES_COUNT, totalBytesRead);
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that <tt>getResource</tt> will skip TCCL if <tt>excludeThreadContextClassLoader</tt> has been set to true.
-   */
-  public void testGetResourceExcludeTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceExcludeTCCL");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
-
-    String resourceToGet = "com/nowhere/testGetResourceExcludeTCCL.rsc";
-    assertNull(dcl.getResource(resourceToGet));
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // ensure that TCCL is only CL that can find this resource
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      assertNull(dcl.getResource(resourceToGet));
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  /**
-   * Verifies that <tt>getResourceAsStream</tt> will skip TCCL if <tt>excludeThreadContextClassLoader</tt> has been set
-   * to true.
-   */
-  public void testGetResourceAsStreamExcludeTCCL() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testGetResourceAsStreamExcludeTCCL");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(true);
-
-    String resourceToGet = "com/nowhere/testGetResourceAsStreamExcludeTCCL.rsc";
-    assertNull(dcl.getResourceAsStream(resourceToGet));
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // ensure that TCCL is only CL that can find this resource
-      Thread.currentThread().setContextClassLoader(new GeneratingClassLoader());
-      assertNull(dcl.getResourceAsStream(resourceToGet));
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  public void testAddFindsLatestClassLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testAddFindsLatestClassLoader");
-
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    dcl = dcl.addOrReplace(new GeneratingClassLoader());
-
-    String classToLoad = "com.nowhere.TestAddFindsLatestClassLoader";
-    Class<?> clazz = dcl.forName(classToLoad);
-    assertNotNull(clazz);
-
-    dcl = dcl.addOrReplace(new BrokenClassLoader());
-
-    try {
-      dcl.forName(classToLoad);
-      fail();
-    } catch (BrokenError expected) {
-      // Expected
-    }
-  }
-
-  /**
-   * Verifies removing a ClassLoader.
-   */
-  public void testRemoveClassLoader() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testRemoveClassLoader");
-
-    GeneratingClassLoader genClassLoader = new GeneratingClassLoader();
-    ClassPathLoader cpl = ClassPathLoader.createWithDefaults(false);
-    cpl = cpl.addOrReplace(genClassLoader);
-
-    String classToLoad = "com.nowhere.TestRemoveClassLoader";
-    Class<?> clazz = cpl.forName(classToLoad);
-    assertNotNull(clazz);
-
-    cpl = cpl.remove(genClassLoader);
-
-    try {
-      clazz = cpl.forName(classToLoad);
-      fail();
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-  }
-
-  /**
-   * Verifies that a ClassLoader will be replaced when added more than once.
-   */
-  public void testClassLoaderReplace() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testClassLoaderReplace");
-
-    String class1ToLoad = "ClassA";
-    String class2ToLoad = "ClassB";
-
-    ClassPathLoader cpl = ClassPathLoader.createWithDefaults(false);
-    cpl = cpl.addOrReplace(new OneClassClassLoader(class1ToLoad));
-
-    try {
-      @SuppressWarnings("unused")
-      Class<?> clazz = cpl.forName(class1ToLoad);
-    } catch (ClassNotFoundException unexpected) {
-      fail();
-    }
-
-    try {
-      @SuppressWarnings("unused")
-      Class<?> clazz = cpl.forName(class2ToLoad);
-      fail();
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-
-    cpl = cpl.addOrReplace(new OneClassClassLoader(class2ToLoad));
-    try {
-      @SuppressWarnings("unused")
-      Class<?> clazz = cpl.forName(class2ToLoad);
-    } catch (ClassNotFoundException unexpected) {
-      fail();
-    }
-
-    try {
-      @SuppressWarnings("unused")
-      Class<?> clazz = cpl.forName(class1ToLoad);
-      fail();
-    } catch (ClassNotFoundException expected) {
-      // Expected
-    }
-  }
-
-  /**
-   * Verifies that JAR files found in the extlib directory will be correctly
-   * added to the {@link ClassPathLoader}.
-   */
-  public void testJarsInExtLib() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testJarsInExtLib");
-
-    this.deleteLibDir = ClassPathLoader.EXT_LIB_DIR.getParentFile().mkdirs();
-    this.deleteExtDir = ClassPathLoader.EXT_LIB_DIR.mkdirs();
-    
-    File subdir = new File(ClassPathLoader.EXT_LIB_DIR, "cplju");
-    subdir.mkdir();
-    
-    final ClassBuilder classBuilder = new ClassBuilder();
-    
-    writeJarBytesToFile(new File(ClassPathLoader.EXT_LIB_DIR, "ClassPathLoaderJUnit1.jar"),
-        classBuilder.createJarFromClassContent("com/cpljunit1/ClassPathLoaderJUnit1", "package com.cpljunit1; public class ClassPathLoaderJUnit1 {}"));
-    writeJarBytesToFile(new File(subdir, "ClassPathLoaderJUnit2.jar"),
-        classBuilder.createJarFromClassContent("com/cpljunit2/ClassPathLoaderJUnit2", "package com.cpljunit2; public class ClassPathLoaderJUnit2 {}"));
-    
-    ClassPathLoader classPathLoader = ClassPathLoader.createWithDefaults(false);
-    try {
-      classPathLoader.forName("com.cpljunit1.ClassPathLoaderJUnit1");
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath");
-    }
-    
-    try {
-      classPathLoader.forName("com.cpljunit2.ClassPathLoaderJUnit2");
-    } catch (ClassNotFoundException cnfex) {
-      fail("JAR file not correctly added to Classpath");
-    }
-    
-    assertNotNull(classPathLoader.getResource("com/cpljunit2/ClassPathLoaderJUnit2.class"));
-    
-    Enumeration<URL> urls = classPathLoader.getResources("com/cpljunit1");
-    if  (!urls.hasMoreElements()) {
-      fail("Resources should return one element");
-    }
-  }
-  
-  @Test
-  public void testAsClassLoaderLoadClassWithMultipleCustomLoaders() throws Exception {
-    System.out.println("\nStarting ClassPathLoaderJUnitTest#testAsClassLoaderLoadClassWithMultipleCustomLoaders");
-
-    // create DCL such that the 3rd loader should find the class
-    // first custom loader becomes parent which won't find anything
-    ClassPathLoader dcl = ClassPathLoader.createWithDefaults(false);
-    final GeneratingClassLoader generatingClassLoader = new GeneratingClassLoader();
-    dcl = dcl.addOrReplace(generatingClassLoader);
-    dcl = dcl.addOrReplace(new SimpleClassLoader(getClass().getClassLoader()));
-    dcl = dcl.addOrReplace(new NullClassLoader());
-
-    final String classToLoad = "com.nowhere.TestForNameWithMultipleCustomLoaders";
-
-    ClassLoader cl = Thread.currentThread().getContextClassLoader();
-    try {
-      // set TCCL to throw errors which makes sure we find before checking TCCL
-      Thread.currentThread().setContextClassLoader(new BrokenClassLoader());
-
-      final ClassLoader classLoader = dcl.asClassLoader();
-      final Class<?> clazz = classLoader.loadClass(classToLoad);
-      assertNotNull(clazz);
-      assertEquals(classToLoad, clazz.getName());
-      assertTrue(clazz.getClassLoader() instanceof GeneratingClassLoader);
-      assertEquals(generatingClassLoader, clazz.getClassLoader());
-
-      final Object obj = clazz.newInstance();
-      assertEquals(classToLoad, obj.getClass().getName());
-      
-      final Class<?> clazz2 = dcl.forName(classToLoad);
-      assertSame("Should load same class as calling classLoader.", clazz, clazz2);
-
-      final Class<?> clazz3 = Class.forName(classToLoad, true, classLoader);
-      assertSame("Should load same class as calling classLoader.", clazz, clazz3);
-
-    } finally {
-      Thread.currentThread().setContextClassLoader(cl);
-    }
-  }
-
-  private void writeJarBytesToFile(File jarFile, byte[] jarBytes) throws IOException {
-    final OutputStream outStream = new FileOutputStream(jarFile);
-    outStream.write(jarBytes);
-    outStream.close();
-  }
-  
-  private static void exploreClassLoaders() {
-    System.out.println("Thread.currentThread().getContextClassLoader()...");
-    exploreClassLoader(Thread.currentThread().getContextClassLoader(), 1);
-
-    System.out.println("class.getClassLoader()...");
-    exploreClassLoader(ClassPathLoaderJUnitTest.class.getClassLoader(), 1);
-
-    System.out.println("ClassLoader.getSystemClassLoader()...");
-    exploreClassLoader(ClassLoader.getSystemClassLoader(), 1);
-  }
-
-  private static void exploreClassLoader(ClassLoader cl, int indent) {
-    String prefix = "";
-    for (int i = 0; i < indent; i++) {
-      prefix += "\t";
-    }
-    System.out.println(prefix + "ClassLoader toString() = " + cl);
-
-    Class<?> clazz = cl.getClass();
-    System.out.println(prefix + "ClassLoader getClass().getName() = " + clazz.getName());
-    exploreClassLoaderSuperClass(prefix, clazz);
-
-    try {
-      URL[] urls = ((URLClassLoader) cl).getURLs();
-      StringBuilder sb = new StringBuilder(prefix).append("ClassLoader getURLs = [");
-      for (int i = 0; i < urls.length; i++) {
-        if (i > 0)
-          sb.append(", ");
-        sb.append(urls[i].toString());
-      }
-      sb.append("]");
-      System.out.println(sb.toString());
-    } catch (Exception e) {
-      System.out.println(prefix + "ClassLoader is not a URLClassLoader");
-    }
-
-    ClassLoader parent = cl.getParent();
-    if (parent != null) {
-      System.out.println(prefix + "ClassLoader has parent...");
-      exploreClassLoader(parent, ++indent);
-    }
-  }
-
-  private static void exploreClassLoaderSuperClass(String prefix, Class<?> clazz) {
-    Class<?> superClazz = clazz.getSuperclass();
-    if (superClazz != null) {
-      System.out.println(prefix + "                       getSuperclass().getName() = " + superClazz.getName());
-      exploreClassLoaderSuperClass(prefix, superClazz);
-    }
-  }
-  
-  /**
-   * Custom class loader which uses BCEL to always dynamically generate a class for any class name it tries to load.
-   */
-  private class GeneratingClassLoader extends ClassLoader {
-
-    /**
-     * Currently unused but potentially useful for some future test. This causes this loader to only generate a class
-     * that the parent could not find.
-     * 
-     * @param parent
-     *          the parent class loader to check with first
-     */
-    @SuppressWarnings("unused")
-    public GeneratingClassLoader(ClassLoader parent) {
-      super(parent);
-    }
-
-    /**
-     * Specifies no parent to ensure that this loader generates the named class.
-     */
-    public GeneratingClassLoader() {
-      super(null); // no parent!!
-    }
-
-    @Override
-    protected Class<?> findClass(String name) throws ClassNotFoundException {
-      ClassGen cg = new ClassGen(name, "java.lang.Object", "<generated>", Constants.ACC_PUBLIC | Constants.ACC_SUPER, null);
-      cg.addEmptyConstructor(Constants.ACC_PUBLIC);
-      JavaClass jClazz = cg.getJavaClass();
-      byte[] bytes = jClazz.getBytes();
-      return defineClass(jClazz.getClassName(), bytes, 0, bytes.length);
-    }
-
-    @Override
-    protected URL findResource(String name) {
-      URL url = null;
-      try {
-        url = getTempFile().getAbsoluteFile().toURI().toURL();
-        System.out.println("GeneratingClassLoader#findResource returning " + url);
-      } catch (IOException e) {
-      }
-      return url;
-    }
-
-    @Override
-    protected Enumeration<URL> findResources(String name) throws IOException {
-      URL url = null;
-      try {
-        url = getTempFile().getAbsoluteFile().toURI().toURL();
-        System.out.println("GeneratingClassLoader#findResources returning " + url);
-      } catch (IOException e) {
-      }
-      Vector<URL> urls = new Vector<URL>();
-      urls.add(url);
-      return urls.elements();
-    }
-
-    protected File getTempFile() {
-      return ClassPathLoaderJUnitTest.this.tempFile;
-    }
-  }
-
-  /**
-   * Custom class loader which uses BCEL to always dynamically generate a class for any class name it tries to load.
-   */
-  private class GeneratingClassLoader2 extends GeneratingClassLoader {
-    @Override
-    protected File getTempFile() {
-      return ClassPathLoaderJUnitTest.this.tempFile2;
-    }
-  }
-  
-  /**
-   * Custom class loader which will never find any class or resource.
-   */
-  private class NullClassLoader extends ClassLoader {
-    public NullClassLoader() {
-      super(null); // no parent!!
-    }
-
-    @Override
-    public Class<?> loadClass(String name) throws ClassNotFoundException {
-      throw new ClassNotFoundException(name);
-    }
-
-    @Override
-    public URL getResource(String name) {
-      return null;
-    }
-  }
-
-  /**
-   * Custom class loader which will find anything the parent can find.
-   */
-  private class SimpleClassLoader extends ClassLoader {
-    public SimpleClassLoader(ClassLoader parent) {
-      super(parent);
-    }
-  }
-
-  private class OneClassClassLoader extends ClassLoader {
-    private final GeneratingClassLoader genClassLoader = new GeneratingClassLoader();
-    private String className;
-
-    public OneClassClassLoader(final String className) {
-      super(null); // no parent!!
-      this.className = className;
-    }
-    
-    @Override
-    public Class<?> findClass(String name) throws ClassNotFoundException {
-      if (!name.equals(className))
-        throw new ClassNotFoundException();
-
-      else
-        return this.genClassLoader.findClass(name);
-    }
-    
-    @Override
-    public boolean equals(final Object other) {
-      return (other instanceof OneClassClassLoader);
-    }
-  }
-  
-  /**
-   * Custom class loader which is broken and always throws errors.
-   */
-  private class BrokenClassLoader extends ClassLoader {
-    public BrokenClassLoader() {
-      super(null); // no parent!!
-    }
-
-    @Override
-    public Class<?> loadClass(String name) throws ClassNotFoundException {
-      throw new BrokenError();
-    }
-
-    @Override
-    public URL getResource(String name) {
-      throw new BrokenError();
-    }
-  }
-
-  @SuppressWarnings("serial")
-  private class BrokenError extends Error {
-  }
-}



[07/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/RegionOperationsEqualityShouldUseArrayEqualsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/RegionOperationsEqualityShouldUseArrayEqualsIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/RegionOperationsEqualityShouldUseArrayEqualsIntegrationTest.java
new file mode 100644
index 0000000..aaa6a3c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/RegionOperationsEqualityShouldUseArrayEqualsIntegrationTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.internal;
+
+import static org.junit.Assert.fail;
+
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * See bug 52093.
+ * Make sure that the contents of arrays are
+ * used to test equality on regions ops.
+ */
+@Category(IntegrationTest.class)
+public class RegionOperationsEqualityShouldUseArrayEqualsIntegrationTest {
+
+  private GemFireCacheImpl createCache() {
+    Properties props = new Properties();
+    props.setProperty("locators", "");
+    props.setProperty("mcast-port", "0");
+    GemFireCacheImpl result = (GemFireCacheImpl) new CacheFactory(props).create();
+    return result;
+  }
+
+  private void closeCache(GemFireCacheImpl gfc) {
+    gfc.close();
+  }
+  
+  private void doOps(Region r) {
+    {
+      byte[] bytesValue = new byte[]{1,2,3,4};
+      r.put("bytesValue", bytesValue.clone());
+      if (r.replace("bytesValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("bytesValue", bytesValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("bytesValue", bytesValue.clone());
+      if (r.remove("bytesValue", "")) fail("expected remove to fail");
+      if (!r.remove("bytesValue", bytesValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      boolean[] booleanValue = new boolean[]{true,false,true,false};
+      r.put("booleanValue", booleanValue.clone());
+      if (r.replace("booleanValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("booleanValue", booleanValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("booleanValue", booleanValue.clone());
+      if (r.remove("booleanValue", "")) fail("expected remove to fail");
+      if (!r.remove("booleanValue", booleanValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      short[] shortValue = new short[]{1,2,3,4};
+      r.put("shortValue", shortValue.clone());
+      if (r.replace("shortValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("shortValue", shortValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("shortValue", shortValue.clone());
+      if (r.remove("shortValue", "")) fail("expected remove to fail");
+      if (!r.remove("shortValue", shortValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      char[] charValue = new char[]{1,2,3,4};
+      r.put("charValue", charValue.clone());
+      if (r.replace("charValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("charValue", charValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("charValue", charValue.clone());
+      if (r.remove("charValue", "")) fail("expected remove to fail");
+      if (!r.remove("charValue", charValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      int[] intValue = new int[]{1,2,3,4};
+      r.put("intValue", intValue.clone());
+      if (r.replace("intValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("intValue", intValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("intValue", intValue.clone());
+      if (r.remove("intValue", "")) fail("expected remove to fail");
+      if (!r.remove("intValue", intValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      long[] longValue = new long[]{1,2,3,4};
+      r.put("longValue", longValue.clone());
+      if (r.replace("longValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("longValue", longValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("longValue", longValue.clone());
+      if (r.remove("longValue", "")) fail("expected remove to fail");
+      if (!r.remove("longValue", longValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      float[] floatValue = new float[]{1,2,3,4};
+      r.put("floatValue", floatValue.clone());
+      if (r.replace("floatValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("floatValue", floatValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("floatValue", floatValue.clone());
+      if (r.remove("floatValue", "")) fail("expected remove to fail");
+      if (!r.remove("floatValue", floatValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      double[] doubleValue = new double[]{1,2,3,4};
+      r.put("doubleValue", doubleValue.clone());
+      if (r.replace("doubleValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("doubleValue", doubleValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("doubleValue", doubleValue.clone());
+      if (r.remove("doubleValue", "")) fail("expected remove to fail");
+      if (!r.remove("doubleValue", doubleValue)) {
+        fail("expected remove to happen");
+      }
+    }
+    {
+      Object[] oaValue = new Object[]{new byte[]{1,2,3,4},new short[]{1,2,3,4},new int[]{1,2,3,4}, "hello sweet world!"};
+      r.put("oaValue", oaValue);
+      Object[] deepCloneOaValue = new Object[]{new byte[]{1,2,3,4},new short[]{1,2,3,4},new int[]{1,2,3,4}, "hello sweet world!"};
+      if (r.replace("oaValue", "", "")) fail("expected replace to fail");
+      if (!r.replace("oaValue", deepCloneOaValue, "")) {
+        fail("expected replace to happen");
+      }
+      r.put("oaValue", oaValue);
+      if (r.remove("oaValue", "")) fail("expected remove to fail");
+      if (!r.remove("oaValue", deepCloneOaValue)) {
+        fail("expected remove to happen");
+      }
+    }
+  }
+  
+  @Test
+  public void testPartition() {
+    GemFireCacheImpl gfc = createCache();
+    try {
+      Region r = gfc.createRegionFactory(RegionShortcut.PARTITION).create("ArrayEqualsJUnitTestPartitionRegion");
+      doOps(r);
+    } finally {
+      closeCache(gfc);
+    }
+  }
+  @Test
+  public void testLocal() {
+    GemFireCacheImpl gfc = createCache();
+    try {
+      Region r = gfc.createRegionFactory(RegionShortcut.LOCAL).create("ArrayEqualsJUnitTestLocalRegion");
+      doOps(r);
+    } finally {
+      closeCache(gfc);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserJUnitTest.java
index 83f4a08..cc71738 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserJUnitTest.java
@@ -16,15 +16,13 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
 
-import java.io.IOException;
 import java.net.Socket;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.junit.Assert.*;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -150,7 +148,7 @@ public class SocketCloserJUnitTest {
    * closed socket is a noop.
    */
   @Test
-  public void testClosedSocket() throws IOException {
+  public void testClosedSocket() throws Exception {
     final AtomicBoolean runnableCalled = new AtomicBoolean();
     Runnable r = new Runnable() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserWithWaitJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserWithWaitJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserWithWaitJUnitTest.java
index 52a367d..0ba41ba 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserWithWaitJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/SocketCloserWithWaitJUnitTest.java
@@ -28,6 +28,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  */
 @Category(UnitTest.class)
 public class SocketCloserWithWaitJUnitTest extends SocketCloserJUnitTest {
+
   @Override
   protected SocketCloser createSocketCloser() {
     return new SocketCloser(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
index b35c851..90bd869 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
@@ -29,6 +29,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
@@ -39,21 +40,25 @@ import com.gemstone.gemfire.StatisticsType;
 import com.gemstone.gemfire.internal.StatArchiveReader.StatValue;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.util.StopWatch;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * @since 7.0
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class StatSamplerJUnitTest {
+
   private static final Logger logger = LogService.getLogger();
-  
-  @Rule
-  public TestName testName = new TestName();
 
   private Map<String,String> statisticTypes;
   private Map<String,Map<String,Number>> allStatistics;
 
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  @Rule
+  public TestName testName = new TestName();
+
   @Before
   public void setUp() {
     this.statisticTypes = new HashMap<String,String>();
@@ -62,13 +67,6 @@ public class StatSamplerJUnitTest {
   
   @After
   public void tearDown() {
-    System.clearProperty("stats.log-level");
-    System.clearProperty("stats.disable");
-    System.clearProperty("stats.name");
-    System.clearProperty("stats.archive-file");
-    System.clearProperty("stats.file-size-limit");
-    System.clearProperty("stats.disk-space-limit");
-    System.clearProperty("stats.sample-rate");
     this.statisticTypes = null;
     this.allStatistics = null;
     StatisticsTypeFactoryImpl.clear();
@@ -114,18 +112,16 @@ public class StatSamplerJUnitTest {
     final Statistics st1_1 = factory.createAtomicStatistics(ST1, "st1_1", 1);
     
     boolean done = false;
-    try {
-      Statistics[] samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
-      for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 4000; done = (samplerStatsInstances != null && samplerStatsInstances.length > 0)) {
-        Thread.sleep(10);
-        samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
+
+    Statistics[] samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
+    for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 4000; done = (samplerStatsInstances != null && samplerStatsInstances.length > 0)) {
+      Thread.sleep(10);
+      samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
     }
+
     assertTrue("Waiting for statSampler stats", done);
 
-    Statistics[] samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
+    samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
     assertNotNull(samplerStatsInstances);
     assertEquals(1, samplerStatsInstances.length);
     final Statistics samplerStats = samplerStatsInstances[0];
@@ -265,10 +261,8 @@ public class StatSamplerJUnitTest {
     final StatArchiveReader reader = new StatArchiveReader(
         new File[]{archiveFile}, null, false);
 
-    @SuppressWarnings("rawtypes")
     List resources = reader.getResourceInstList();
-    for (@SuppressWarnings("rawtypes")
-    Iterator iter = resources.iterator(); iter.hasNext();) {
+    for (Iterator iter = resources.iterator(); iter.hasNext();) {
       StatArchiveReader.ResourceInst ri = (StatArchiveReader.ResourceInst) iter.next();
       String resourceName = ri.getName();
       assertNotNull(resourceName);
@@ -303,15 +297,11 @@ public class StatSamplerJUnitTest {
     }
   }
   
-  private void waitForStatSample(final Statistics samplerStats) {
+  private void waitForStatSample(final Statistics samplerStats) throws InterruptedException {
     int startSampleCount = samplerStats.getInt("sampleCount");
     boolean done = false;
-    try {
-      for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 3000; done = (samplerStats.getInt("sampleCount") > startSampleCount)) {
-        Thread.sleep(10);
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
+    for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 3000; done = (samplerStats.getInt("sampleCount") > startSampleCount)) {
+      Thread.sleep(10);
     }
     assertTrue("Waiting for statSampler sampleCount to increment", done);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/UniqueIdGeneratorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/UniqueIdGeneratorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/UniqueIdGeneratorJUnitTest.java
index 16bf32c..222d46a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/UniqueIdGeneratorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/UniqueIdGeneratorJUnitTest.java
@@ -16,11 +16,11 @@
  */
 package com.gemstone.gemfire.internal;
 
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.internal.UniqueIdGenerator;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
@@ -28,17 +28,9 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * @since 5.0.2 (cbb5x_PerfScale)
  */
 @Category(UnitTest.class)
-public class UniqueIdGeneratorJUnitTest extends TestCase {
-  
-  public UniqueIdGeneratorJUnitTest() {
-  }
-  
-  public void setup() {
-  }
-  
-  public void tearDown() {
-  }
+public class UniqueIdGeneratorJUnitTest {
   
+  @Test
   public void testBasics() throws Exception {
     UniqueIdGenerator uig = new UniqueIdGenerator(1);
     assertEquals(0, uig.obtain());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
index 8eaeeb5..ac060eb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
@@ -16,10 +16,9 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.*;
 
 import org.apache.logging.log4j.Logger;
 import org.junit.Test;
@@ -34,9 +33,8 @@ import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.fake.Fakes;
 
-import junit.framework.TestCase;
+public abstract class AbstractDistributedRegionJUnitTest {
 
-public abstract class AbstractDistributedRegionJUnitTest extends TestCase {
   protected static final Logger logger = LogService.getLogger();
   
   private RegionAttributes createRegionAttributes(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionJUnitTest.java
index ca486cd..f17b6c5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionJUnitTest.java
@@ -54,7 +54,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link AbstractRegion}.
- * 
  *
  * @since 8.1
  */
@@ -66,7 +65,6 @@ public class AbstractRegionJUnitTest {
    * 
    * Assert that method returns a {@link SimpleExtensionPoint} instance and
    * assume that {@link com.gemstone.gemfire.internal.cache.extension.SimpleExtensionPointJUnitTest} has covered the rest.
-   * 
    */
   @Test
   public void testGetExtensionPoint() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionMapTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionMapTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionMapTest.java
index b54bbe2..e8591e4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionMapTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractRegionMapTest.java
@@ -19,10 +19,6 @@ package com.gemstone.gemfire.internal.cache;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -34,22 +30,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class AbstractRegionMapTest {
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-  }
-
-  @Before
-  public void setUp() throws Exception {
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
   @Test
   public void invalidateOfNonExistentRegionThrowsEntryNotFound() {
     TestableAbstractRegionMap arm = new TestableAbstractRegionMap();
@@ -172,7 +152,8 @@ public class AbstractRegionMapTest {
     }
   }
 
-  public static class TestableAbstractRegionMap extends AbstractRegionMap {
+  private static class TestableAbstractRegionMap extends AbstractRegionMap {
+
     public LocalRegion owner;
 
     protected TestableAbstractRegionMap() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
index a5a3cc0..bb14b22 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
@@ -16,26 +16,25 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.anyLong;
 import static org.mockito.Mockito.*;
 
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-/**
- *
- */
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 @Category(UnitTest.class)
 public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
 
+  @Override
   protected void setInternalRegionArguments(InternalRegionArguments ira) {
     // PR specific
     PartitionedRegion pr = mock(PartitionedRegion.class);
@@ -49,13 +48,14 @@ public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
       .setPartitionedRegionBucketRedundancy(1)
       .setBucketAdvisor(ba);
   }
-  
+
+  @Override
   protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
       RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
     BucketRegion br = new BucketRegion("testRegion", ra, null, cache, ira);
 
     // since br is a real bucket region object, we need to tell mockito to monitor it
-    br = Mockito.spy(br);
+    br = spy(br);
 
 //    doNothing().when(dm).addMembershipListener(any());
     doNothing().when(br).distributeUpdateOperation(any(), anyLong());
@@ -69,6 +69,7 @@ public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
     return br;
   }
 
+  @Override
   protected void verifyDistributeUpdate(DistributedRegion region, EntryEventImpl event, int cnt) {
     assertTrue(region instanceof BucketRegion);
     BucketRegion br = (BucketRegion)region;
@@ -80,7 +81,8 @@ public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
       verify(br, never()).distributeUpdateOperation(eq(event), eq(12345L));
     }
   }
-  
+
+  @Override
   protected void verifyDistributeDestroy(DistributedRegion region, EntryEventImpl event, int cnt) {
     assertTrue(region instanceof BucketRegion);
     BucketRegion br = (BucketRegion)region;
@@ -92,7 +94,8 @@ public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
       verify(br, never()).distributeDestroyOperation(eq(event));
     }
   }
-  
+
+  @Override
   protected void verifyDistributeInvalidate(DistributedRegion region, EntryEventImpl event, int cnt) {
     assertTrue(region instanceof BucketRegion);
     BucketRegion br = (BucketRegion)region;
@@ -104,7 +107,8 @@ public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
       verify(br, never()).distributeInvalidateOperation(eq(event));
     }
   }
-    
+
+  @Override
   protected void verifyDistributeUpdateEntryVersion(DistributedRegion region, EntryEventImpl event, int cnt) {
     assertTrue(region instanceof BucketRegion);
     BucketRegion br = (BucketRegion)region;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaFaultInDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaFaultInDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaFaultInDUnitTest.java
index a1fca4c..78855fd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaFaultInDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaFaultInDUnitTest.java
@@ -119,7 +119,7 @@ public class DeltaFaultInDUnitTest extends CacheTestCase {
         
         long entriesEvicted = ((AbstractLRURegionMap)((PartitionedRegion)region).entries)._getLruList().stats()
         .getEvictions();
-//        assertEquals(1, entriesEvicted);
+//        assertIndexDetailsEquals(1, entriesEvicted);
         
         TestDelta result = region.get(new Integer(0));
         assertEquals("initial_plus_some_more_data", result.info);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaSizingDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaSizingDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaSizingDUnitTest.java
index 94aeccd..7d53098 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaSizingDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaSizingDUnitTest.java
@@ -155,7 +155,7 @@ public class DeltaSizingDUnitTest extends CacheTestCase {
 //              throws CacheWriterException {
 //            assertTrue(event.getOldValue() instanceof MyClass);
 //            assertTrue(event.getNewValue() instanceof MyClass);
-//            assertEquals(event.getOldValue(), event.getNewValue());
+//            assertIndexDetailsEquals(event.getOldValue(), event.getNewValue());
 //          }
 //          
 //        });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskIdJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskIdJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskIdJUnitTest.java
index 14c25ef..4a11280 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskIdJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskIdJUnitTest.java
@@ -16,29 +16,24 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 /**
- * 
- * Test verifies the setting and getting of disk id values are correctly 
- * 
- *
+ * Test verifies the setting and getting of disk id values are correctly
  */
 @Category(UnitTest.class)
-public class DiskIdJUnitTest extends TestCase
-{
+public class DiskIdJUnitTest {
 
   /**
    * Test the getOplogId returns what has been set
-   * 
-   * @throws Exception
    */
-  public void testGetSetOplogId() throws Exception
-  {
+  @Test
+  public void testGetSetOplogId() throws Exception {
     DiskId did = getDiskId();
     did.setOplogId(-1);
     assertEquals(-1, did.getOplogId());
@@ -52,12 +47,9 @@ public class DiskIdJUnitTest extends TestCase
 
   /**
    * Test the getUserbits returns what has been set
-   * 
-   * @throws Exception
    */
- 
-  public void testGetSetUserBits() throws Exception
-  {
+  @Test
+  public void testGetSetUserBits() throws Exception {
     DiskId did = getDiskId();
     byte userBits = 0;
     userBits = EntryBits.setSerialized(userBits, true);
@@ -119,9 +111,8 @@ public class DiskIdJUnitTest extends TestCase
   /**
    * Test the whether setting of one set of values does not affect another set of values
    */
- 
-  public void testAllOperationsValidatingResult1()
-  {
+  @Test
+  public void testAllOperationsValidatingResult1() {
     DiskId did = getDiskId();
     for (int i = -16777215; i < 16777215; i++) {
       boolean boolValuePerIteration = false;
@@ -168,15 +159,14 @@ public class DiskIdJUnitTest extends TestCase
         break;
       }
     }
-
   }
   
   /**
    * Tests that an instance of 'PersistenceIntOplogOffsetDiskId' is created when
    * max-oplog-size (in bytes) passed is smaller than Integer.MAX_VALUE
    */
-  public void testPersistIntDiskIdInstance()
-  {
+  @Test
+  public void testPersistIntDiskIdInstance() {
     int maxOplogSizeinMB = 2;
 
     DiskId diskId = DiskId.createDiskId(maxOplogSizeinMB, true /*is persistence type*/, true);
@@ -189,8 +179,8 @@ public class DiskIdJUnitTest extends TestCase
    * Tests that an instance of 'LongOplogOffsetDiskId' is created when
    * max-oplog-size (in bytes) passed is greater than Integer.MAX_VALUE
    */
-  public void testPersistLongDiskIdInstance()
-  {
+  @Test
+  public void testPersistLongDiskIdInstance() {
     long maxOplogSizeInBytes = (long)Integer.MAX_VALUE + 1;
     int maxOplogSizeinMB = (int)(maxOplogSizeInBytes / (1024 * 1024));
 
@@ -204,8 +194,8 @@ public class DiskIdJUnitTest extends TestCase
    * Tests that an instance of 'PersistenceIntOplogOffsetDiskId' is created when
    * max-oplog-size (in bytes) passed is smaller than Integer.MAX_VALUE
    */
-  public void testOverflowIntDiskIdInstance()
-  {
+  @Test
+  public void testOverflowIntDiskIdInstance() {
     int maxOplogSizeinMB = 2;
 
     DiskId diskId = DiskId.createDiskId(maxOplogSizeinMB, false /*is overflow type*/, true);
@@ -218,8 +208,8 @@ public class DiskIdJUnitTest extends TestCase
    * Tests that an instance of 'LongOplogOffsetDiskId' is created when
    * max-oplog-size (in bytes) passed is greater than Integer.MAX_VALUE
    */
-  public void testOverflowLongDiskIdInstance()
-  {
+  @Test
+  public void testOverflowLongDiskIdInstance() {
     long maxOplogSizeInBytes = (long)Integer.MAX_VALUE + 1;
     int maxOplogSizeinMB = (int)(maxOplogSizeInBytes / (1024 * 1024));
 
@@ -229,8 +219,7 @@ public class DiskIdJUnitTest extends TestCase
         DiskId.isInstanceofOverflowOnlyWithLongOffset(diskId));
   }
 
-  private DiskId getDiskId()
-  {
+  private DiskId getDiskId() {
     return DiskId.createDiskId(1024, true /* is persistence type*/, true);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskInitFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskInitFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskInitFileJUnitTest.java
index c9f3961..5a4e622 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskInitFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskInitFileJUnitTest.java
@@ -16,43 +16,44 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import java.io.File;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import java.io.File;
+import java.util.Collections;
 
 import org.jmock.Expectations;
 import org.jmock.Mockery;
 import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
-import java.util.Collections;
+import org.junit.rules.TemporaryFolder;
 
 import com.gemstone.gemfire.StatisticsFactory;
-import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.cache.persistence.DiskRegionView;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
-public class DiskInitFileJUnitTest extends TestCase {
+@Category(IntegrationTest.class)
+public class DiskInitFileJUnitTest {
   
   private File testDirectory;
   private Mockery context = new Mockery() {{
     setImposteriser(ClassImposteriser.INSTANCE);
   }};
-  
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
   public void setUp() throws Exception {
-    testDirectory = new File("_DiskInitFileJUnitTest");
-    FileUtil.delete(testDirectory);
-    FileUtil.mkdirs(testDirectory);
-  }
-  
-  public void tearDown() throws Exception {
-    FileUtil.delete(testDirectory);
+    testDirectory = temporaryFolder.newFolder("_" + getClass().getSimpleName());
   }
-  
+
   /**
    * Test the behavior of canonical ids in the init file.
    */
+  @Test
   public void testCanonicalIds() {
     //create a mock statistics factory for creating directory holders
     final StatisticsFactory sf = context.mock(StatisticsFactory.class);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesJUnitTest.java
index d7d07f9..4c6710c 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesJUnitTest.java
@@ -16,136 +16,117 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.DiskWriteAttributes;
 import com.gemstone.gemfire.cache.DiskWriteAttributesFactory;
-import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 /**
  * Tests if DiskWriteAttributeFactory returns the correct DWA object with the
  * desired values
- * 
- *  
  */
 @Category(UnitTest.class)
-public class DiskWriteAttributesJUnitTest extends TestCase
-{
+public class DiskWriteAttributesJUnitTest {
 
-  public DiskWriteAttributesJUnitTest(String arg0) {
-    super(arg0);
-  }
-
-  protected void setUp() throws Exception
-  {
-    super.setUp();
-  }
-
-  protected void tearDown() throws Exception
-  {
-    super.tearDown();
-  }
-
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultInstance()'
    */
-  public void testGetDefaultInstance()
-  {
+  @Test
+  public void testGetDefaultInstance() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     DiskWriteAttributes dwa = dwaf.create();
-    Assert.assertTrue(!dwa.isSynchronous());
-    Assert.assertTrue(dwa.isRollOplogs());
+    assertTrue(!dwa.isSynchronous());
+    assertTrue(dwa.isRollOplogs());
   }
 
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultSync()'
    */
-  public void testGetDefaultSync()
-  {
+  @Test
+  public void testGetDefaultSync() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     dwaf.setSynchronous(true);
     DiskWriteAttributes dwa = dwaf.create();
-    Assert.assertTrue(dwa.isSynchronous());
-    Assert.assertTrue(dwa.isRollOplogs());
+    assertTrue(dwa.isSynchronous());
+    assertTrue(dwa.isRollOplogs());
   }
 
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultAsync()'
    */
-  public void testGetDefaultAsync()
-  {
+  @Test
+  public void testGetDefaultAsync() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     DiskWriteAttributes dwa = dwaf.create();
-    Assert.assertTrue(!dwa.isSynchronous());
-    Assert.assertTrue(dwa.isRollOplogs());
+    assertTrue(!dwa.isSynchronous());
+    assertTrue(dwa.isRollOplogs());
   }
 
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultRollingSync()'
    */
-  public void testGetDefaultRollingSync()
-  {
+  @Test
+  public void testGetDefaultRollingSync() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     dwaf.setSynchronous(true);
     DiskWriteAttributes dwa = dwaf.create();
 
-    Assert.assertTrue(dwa.isSynchronous());
-    Assert.assertTrue(dwa.isRollOplogs());
+    assertTrue(dwa.isSynchronous());
+    assertTrue(dwa.isRollOplogs());
   }
 
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultRollingAsync()'
    */
-  public void testGetDefaultRollingAsync()
-  {
+  @Test
+  public void testGetDefaultRollingAsync() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     DiskWriteAttributes dwa = dwaf.create();
-    Assert.assertTrue(!dwa.isSynchronous());
-    Assert.assertTrue(dwa.isRollOplogs());
-
+    assertTrue(!dwa.isSynchronous());
+    assertTrue(dwa.isRollOplogs());
   }
 
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultNonRollingSync()'
    */
-  public void testGetDefaultNonRollingSync()
-  {
-
+  @Test
+  public void testGetDefaultNonRollingSync() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     dwaf.setRollOplogs(false);
     dwaf.setSynchronous(true);
     DiskWriteAttributes dwa = dwaf.create();
-    Assert.assertTrue(dwa.isSynchronous());
-    Assert.assertTrue(!dwa.isRollOplogs());
-
+    assertTrue(dwa.isSynchronous());
+    assertTrue(!dwa.isRollOplogs());
   }
 
-  /*
+  /**
    * Test method for
    * 'com.gemstone.gemfire.cache.DiskWriteAttributes.getDefaultNonRollingAsync()'
    */
-  public void testGetDefaultNonRollingAsync()
-  {
+  @Test
+  public void testGetDefaultNonRollingAsync() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     dwaf.setRollOplogs(false);
     DiskWriteAttributes dwa = dwaf.create();
-    Assert.assertTrue(!dwa.isSynchronous());
-    Assert.assertTrue(!dwa.isRollOplogs());
+    assertTrue(!dwa.isSynchronous());
+    assertTrue(!dwa.isRollOplogs());
   }
   
   /**
    * Tests the behaviour of DiskWriteAttributesFactory & DiskWritesAttrbutes with
    * various combinations of  time interval & buffer size.
    */
+  @Test
   public void testDiskWriteAttributesCreation() {
     DiskWriteAttributesFactory dwaf = new DiskWriteAttributesFactory();
     dwaf.setSynchronous(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
index cf2525a..d5e1b9a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
@@ -16,28 +16,25 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.TestCase;
-
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.anyBoolean;
 import static org.mockito.Mockito.*;
 
-/**
- *
- */
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 @Category(UnitTest.class)
 public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTest {
 
+  @Override
   protected void setInternalRegionArguments(InternalRegionArguments ira) {
   }
-  
+
+  @Override
   protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
       RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
     DistributedRegion region = new DistributedRegion("testRegion", ra, null, cache, ira);
@@ -46,7 +43,7 @@ public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTe
     }
     
     // since it is a real region object, we need to tell mockito to monitor it
-    region = Mockito.spy(region);
+    region = spy(region);
 
     doNothing().when(region).distributeUpdate(any(), anyLong(), anyBoolean(), anyBoolean(), any(), anyBoolean());
     doNothing().when(region).distributeDestroy(any(), any());
@@ -55,7 +52,8 @@ public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTe
     
     return region;
   }
-  
+
+  @Override
   protected void verifyDistributeUpdate(DistributedRegion region, EntryEventImpl event, int cnt) {
     region.virtualPut(event, false, false, null, false, 12345L, false);
     // verify the result
@@ -66,6 +64,7 @@ public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTe
     }
   }
 
+  @Override
   protected void verifyDistributeDestroy(DistributedRegion region, EntryEventImpl event, int cnt) {
     region.basicDestroy(event, false, null);
     // verify the result
@@ -76,6 +75,7 @@ public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTe
     }
   }
 
+  @Override
   protected void verifyDistributeInvalidate(DistributedRegion region, EntryEventImpl event, int cnt) {
     region.basicInvalidate(event);
     // verify the result
@@ -86,6 +86,7 @@ public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTe
     }
   }
 
+  @Override
   protected void verifyDistributeUpdateEntryVersion(DistributedRegion region, EntryEventImpl event, int cnt) {
     region.basicUpdateEntryVersion(event);
     // verify the result

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
index c3e057c..800527f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
@@ -17,23 +17,30 @@
 package com.gemstone.gemfire.internal.cache;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 
 import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import static org.mockito.Mockito.*;
-
 @Category(UnitTest.class)
 public class EntryEventImplTest {
 
-  String expectedRegionName = "ExpectedFullRegionPathName";
-  String key = "key1";
-  String value = "value1";
-  KeyInfo keyInfo = new KeyInfo(key, value, null);
+  private String expectedRegionName;
+  private String key;
+  private String value;
+  private KeyInfo keyInfo;
+
+  @Before
+  public void setUp() throws Exception {
+    expectedRegionName = "ExpectedFullRegionPathName";
+    key = "key1";
+    value = "value1";
+    keyInfo = new KeyInfo(key, value, null);
+  }
 
   @Test
   public void verifyToStringOutputHasRegionName() {
@@ -50,7 +57,7 @@ public class EntryEventImplTest {
     assertTrue("String " + expectedRegionName + " was not in toString text: " + toStringValue, toStringValue.indexOf(expectedRegionName) > 0);
 
     // verify that toString called getFullPath method of region object
-    verify(region, Mockito.times(1)).getFullPath();
+    verify(region, times(1)).getFullPath();
   }
 
   private EntryEventImpl createEntryEvent(LocalRegion l) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EnumListenerEventJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EnumListenerEventJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EnumListenerEventJUnitTest.java
index 1687db7..9a58987 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EnumListenerEventJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EnumListenerEventJUnitTest.java
@@ -16,26 +16,22 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
-/**
- * 
- */
 @Category(UnitTest.class)
-public class EnumListenerEventJUnitTest extends TestCase
-{
+public class EnumListenerEventJUnitTest {
+
   /**
    * tests whether EnumListenerEvent.getEnumListenerEvent(int cCode) returns the
    * right result
-   * 
    */
-  public void testGetEnumListEvent()
-  {
+  @Test
+  public void testGetEnumListEvent() {
     checkAndAssert(0,  null);
     checkAndAssert(1,  EnumListenerEvent.AFTER_CREATE);
     checkAndAssert(2,  EnumListenerEvent.AFTER_UPDATE);
@@ -63,9 +59,9 @@ public class EnumListenerEventJUnitTest extends TestCase
   // check that the code and object both match
   private void checkAndAssert(int code, EnumListenerEvent event) {
     EnumListenerEvent localEvent = EnumListenerEvent.getEnumListenerEvent(code); 
-    Assert.assertTrue( localEvent == event);
+    assertTrue( localEvent == event);
     if (localEvent != null) {
-      Assert.assertTrue( localEvent.getEventCode() == code);
+      assertTrue( localEvent.getEventCode() == code);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
index 550d133..362962d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
@@ -16,8 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -27,11 +26,11 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.OffHeapValueWrapper;
 import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.Flushable;
+import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.OffHeapValueWrapper;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
 import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
-import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
@@ -39,7 +38,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class OffHeapValueWrapperJUnitTest {
 
-  private static OffHeapValueWrapper createChunkValueWrapper(byte[] bytes, boolean isSerialized) {
+  private OffHeapValueWrapper createChunkValueWrapper(byte[] bytes, boolean isSerialized) {
     StoredObject c = MemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, false);
     return new OffHeapValueWrapper(c);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldVLJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldVLJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldVLJUnitTest.java
index 9841ec5..3f919ed 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldVLJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldVLJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -24,15 +26,15 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class OldVLJUnitTest extends TestCase {
+public class OldVLJUnitTest {
+
   private ByteArrayOutputStream baos;
   private DataOutputStream dos;
 
@@ -44,46 +46,53 @@ public class OldVLJUnitTest extends TestCase {
 
   private DataInput createDIS() throws IOException {
     this.dos.close();
-    ByteArrayInputStream bais = new ByteArrayInputStream(this.baos
-        .toByteArray());
+    ByteArrayInputStream bais = new ByteArrayInputStream(this.baos.toByteArray());
     return new DataInputStream(bais);
   }
 
+  @Test
   public void testMinByte() throws IOException {
     InternalDataSerializer.writeVLOld(1, createDOS());
     assertEquals(1, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMaxByte() throws IOException {
     InternalDataSerializer.writeVLOld(125, createDOS());
     assertEquals(125, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMinShort() throws IOException {
     InternalDataSerializer.writeVLOld(126, createDOS());
     assertEquals(126, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMaxShort() throws IOException {
     InternalDataSerializer.writeVLOld(0x7fff, createDOS());
     assertEquals(0x7fff, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMinInt() throws IOException {
     InternalDataSerializer.writeVLOld(0x7fff + 1, createDOS());
     assertEquals(0x7fff + 1, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMaxInt() throws IOException {
     InternalDataSerializer.writeVLOld(0x7fffffff, createDOS());
     assertEquals(0x7fffffff, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMinLong() throws IOException {
     InternalDataSerializer.writeVLOld(0x7fffffffL + 1, createDOS());
     assertEquals(0x7fffffffL + 1, InternalDataSerializer.readVLOld(createDIS()));
   }
 
+  @Test
   public void testMaxLong() throws IOException {
     InternalDataSerializer.writeVLOld(Long.MAX_VALUE, createDOS());
     assertEquals(Long.MAX_VALUE, InternalDataSerializer.readVLOld(createDIS()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
index 6dbe100..7c8beec 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
@@ -20,36 +20,27 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
-import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
-import com.gemstone.gemfire.internal.offheap.TinyStoredObject;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
 import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
-import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
 import com.gemstone.gemfire.internal.offheap.SlabImpl;
+import com.gemstone.gemfire.internal.offheap.TinyStoredObject;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 
 public abstract class OldValueImporterTestBase {
-  @Before
-  public void setUp() throws Exception {
-  }
 
-  @After
-  public void tearDown() throws Exception {
-  }
-  
   protected abstract OldValueImporter createImporter();
   protected abstract Object getOldValueFromImporter(OldValueImporter ovi);
   protected abstract void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException;
   protected abstract void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException;
 
   @Test
-  public void testValueSerialization() throws IOException, ClassNotFoundException {
+  public void testValueSerialization() throws Exception {
     byte[] bytes = new byte[1024];
     HeapDataOutputStream hdos = new HeapDataOutputStream(bytes);
     OldValueImporter imsg = createImporter();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdMapJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdMapJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdMapJUnitTest.java
index 9cd0cf7..9e76edf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdMapJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdMapJUnitTest.java
@@ -16,8 +16,9 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.cache.Oplog.OplogEntryIdMap;
@@ -25,16 +26,11 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Tests DiskStoreImpl.OplogEntryIdMap
- * 
- *  
  */
 @Category(UnitTest.class)
-public class OplogEntryIdMapJUnitTest extends TestCase
-{
-  public OplogEntryIdMapJUnitTest(String arg0) {
-    super(arg0);
-  }
+public class OplogEntryIdMapJUnitTest {
 
+  @Test
   public void testBasics() {
     OplogEntryIdMap m = new OplogEntryIdMap();
     for (long i=1; i <= 777777; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdSetJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdSetJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdSetJUnitTest.java
index 9413c3b..12e5a3d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdSetJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogEntryIdSetJUnitTest.java
@@ -16,8 +16,9 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.cache.DiskStoreImpl.OplogEntryIdSet;
@@ -25,16 +26,11 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Tests DiskStoreImpl.OplogEntryIdSet
- * 
- *  
  */
 @Category(UnitTest.class)
-public class OplogEntryIdSetJUnitTest extends TestCase
-{
-  public OplogEntryIdSetJUnitTest(String arg0) {
-    super(arg0);
-  }
+public class OplogEntryIdSetJUnitTest {
 
+  @Test
   public void testBasics() {
     OplogEntryIdSet s = new OplogEntryIdSet();
     for (long i=1; i < 777777; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogJUnitTest.java
index f4f652e..d1be04d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogJUnitTest.java
@@ -609,8 +609,8 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //     region.remove(new Integer(2));
 //     assertNull(writer
 //         .getAsynchOpForEntryFromPendingFlushMap(entry2.getDiskId()));
-//     assertEquals(opSizeBeforeCreateRemove, dr.getChild().getOplogSize());
-//     assertEquals(pendingFlushSize, dr.getChild().getAsynchWriter()
+//     assertIndexDetailsEquals(opSizeBeforeCreateRemove, dr.getChild().getOplogSize());
+//     assertIndexDetailsEquals(pendingFlushSize, dr.getChild().getAsynchWriter()
 //         .getCurrentBufferedBytesSize());
 
 //     closeDown();
@@ -1111,7 +1111,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //       fail("exception not expected" + e);
 //       fail("The test failed as the oplog could not eb synched to disk");
 //     }
-//     assertEquals((this.numCreate + this.numDel + this.numModify),
+//     assertIndexDetailsEquals((this.numCreate + this.numDel + this.numModify),
 //         this.totalSuccessfulOperations);
 //     assertTrue(" The expected oplog size =" + inMemOplogSize
 //         + " Actual Oplog file size =" + actFileSize,
@@ -1296,7 +1296,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 // //       if (j < currentOplogID) {
 // //         // oplogs are now closed to save memory and file descriptors
 // //         // once they are no longer needed
-// //         assertEquals(null, oplog);
+// //         assertIndexDetailsEquals(null, oplog);
 // //       } else {
 //         inMemOplogSize += oplog.getOplogSize();
 //         logWriter.info(" Oplog size="+ oplog.getOplogSize() + " Max Oplog size acceptable="+MAX_OPLOG_SIZE );
@@ -1319,7 +1319,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //         assertTrue(
 //                    " The max Oplog Size limit is violated when taken the actual file size",
 //                    oplog.getActualFileLength() <= MAX_OPLOG_SIZE);
-//         assertEquals(oplog.getOplogSize(), oplog.getActualFileLength());
+//         assertIndexDetailsEquals(oplog.getOplogSize(), oplog.getActualFileLength());
 // //       }
 //     }
 
@@ -1329,7 +1329,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //         + this.expectedOplogSize + " Actual sizes of all oplogs ="
 //         + inMemOplogSize, this.expectedOplogSize == inMemOplogSize);
 
-//     assertEquals((this.numCreate + this.numDel + this.numModify),
+//     assertIndexDetailsEquals((this.numCreate + this.numDel + this.numModify),
 //         this.totalSuccessfulOperations);
 //     closeDown();
 
@@ -1412,7 +1412,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 // //       if (j < currentOplogID) {
 // //         // oplogs are now closed to save memory and file descriptors
 // //         // once they are no longer needed
-// //         assertEquals(null, oplog);
+// //         assertIndexDetailsEquals(null, oplog);
 // //       } else {
 //         inMemOplogSize += oplog.getOplogSize();
 //         //oplog.forceFlush();
@@ -1433,7 +1433,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //         assertTrue(
 //                    " The max Oplog Size limit is violated when taken the actual file size",
 //                    oplog.getActualFileLength() <= MAX_OPLOG_SIZE);
-//         assertEquals(oplog.getOplogSize(), oplog.getActualFileLength());
+//         assertIndexDetailsEquals(oplog.getOplogSize(), oplog.getActualFileLength());
 // //       }
 //     }
 
@@ -1442,7 +1442,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //     assertTrue(" The sum of all oplogs size as expected  ="
 //         + this.expectedOplogSize + " Actual sizes of all oplogs ="
 //         + inMemOplogSize, this.expectedOplogSize == inMemOplogSize);
-//     assertEquals((this.numCreate + this.numDel + this.numModify),
+//     assertIndexDetailsEquals((this.numCreate + this.numDel + this.numModify),
 //         this.totalSuccessfulOperations);
 //     assertFalse(failureCause, testFailed);
 //     closeDown();
@@ -1575,7 +1575,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //     long currentOplogID = currOplog.getOplogId();
 //     long expectedSize = currOplog.getOplogSize();
 //     // Ensure that now switching has happned during the operations
-//     assertEquals(1, currentOplogID);
+//     assertIndexDetailsEquals(1, currentOplogID);
 //     assertTrue(
 //         "The number of operations did not cause asynch writer to run atleast once , the expected file size = "
 //             + expectedSize, expectedSize > 1000);
@@ -2174,11 +2174,11 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //     Oplog.ByteBufferPool pool = (Oplog.ByteBufferPool)bbPools.get(1);
 //     ByteBuffer bb1 = pool.getBufferFromPool();
 //     ByteBuffer bb2 = pool.getBufferFromPool();
-//     assertEquals(2, pool.getTotalBuffers());
-//     assertEquals(2, pool.getBuffersInUse());
+//     assertIndexDetailsEquals(2, pool.getTotalBuffers());
+//     assertIndexDetailsEquals(2, pool.getBuffersInUse());
 //     ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb1);
 //     ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb2);
-//     assertEquals(0, pool.getBuffersInUse());
+//     assertIndexDetailsEquals(0, pool.getBuffersInUse());
 //     region.close();
 
 //     System.setProperty("/testRegion_MAX_POOL_SIZE", "1");
@@ -2191,11 +2191,11 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //     pool = (Oplog.ByteBufferPool)bbPools.get(1);
 //     bb1 = pool.getBufferFromPool();
 //     bb2 = pool.getBufferFromPool();
-//     assertEquals(1, pool.getTotalBuffers());
-//     assertEquals(1, pool.getBuffersInUse());
+//     assertIndexDetailsEquals(1, pool.getTotalBuffers());
+//     assertIndexDetailsEquals(1, pool.getBuffersInUse());
 //     ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb1);
 //     ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb2);
-//     assertEquals(0, pool.getBuffersInUse());
+//     assertIndexDetailsEquals(0, pool.getBuffersInUse());
 //     closeDown();
 
 //   }
@@ -2553,7 +2553,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     //((LocalRegion)region).getDiskRegion().getChild().forceFlush();
 //     int x = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter()
 //         .getApproxFreeBuffers();
-//     assertEquals(10, x);
+//     assertIndexDetailsEquals(10, x);
   }
 
   // we no longer have a pendingFlushMap
@@ -2621,7 +2621,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //         if (!proceedForValidation) {
 //           try {
 //             OplogJUnitTest.class.wait(9000);
-//             assertEquals(true, proceedForValidation);
+//             assertIndexDetailsEquals(true, proceedForValidation);
 //           }
 //           catch (InterruptedException e) {
 //             fail("interrupted");
@@ -2631,7 +2631,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //     }
 
 //     cache.getLogger().info("valueRead : " + valueRead);
-//     assertEquals("valueRead is stale, doesnt match with latest PUT", NEW_VALUE,
+//     assertIndexDetailsEquals("valueRead is stale, doesnt match with latest PUT", NEW_VALUE,
 //         valueRead);
 //     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
 //     CacheObserverHolder.setInstance(old);
@@ -3631,7 +3631,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 //         size2 += dh.getDirStatsDiskSpaceUsage();
 //       }
 //       System.out.println("Size after recreation= "+ size2);
-//       assertEquals(size1, size2);
+//       assertIndexDetailsEquals(size1, size2);
 //       region.close();
       
 //     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogRVVJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogRVVJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogRVVJUnitTest.java
index b08efbf..404a1c9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogRVVJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OplogRVVJUnitTest.java
@@ -23,12 +23,15 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
-
 import org.jmock.Expectations;
 import org.jmock.Mockery;
 import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
 
 import com.gemstone.gemfire.StatisticsFactory;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
@@ -38,27 +41,31 @@ import com.gemstone.gemfire.internal.cache.DiskStoreImpl.OplogEntryIdSet;
 import com.gemstone.gemfire.internal.cache.persistence.DiskRecoveryStore;
 import com.gemstone.gemfire.internal.cache.persistence.DiskStoreID;
 import com.gemstone.gemfire.internal.cache.versions.DiskRegionVersionVector;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class OplogRVVJUnitTest {
 
-@Category(UnitTest.class)
-public class OplogRVVJUnitTest extends TestCase {
   private File testDirectory;
   private Mockery context = new Mockery() {{
     setImposteriser(ClassImposteriser.INSTANCE);
   }};
-  
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
   public void setUp() throws Exception {
-    testDirectory = new File("_DiskStoreImplJUnitTest");
-    FileUtil.delete(testDirectory);
-    FileUtil.mkdirs(testDirectory);
+    testDirectory = temporaryFolder.newFolder("_" + getClass().getSimpleName());
     DiskStoreImpl.SET_IGNORE_PREALLOCATE = true;
   }
-  
+
+  @After
   public void tearDown() throws Exception {
-    super.tearDown();
     DiskStoreImpl.SET_IGNORE_PREALLOCATE = false;
   }
-  
+
+  @Test
   public void testRecoverRVV() throws UnknownHostException {
     final DiskInitFile df = context.mock(DiskInitFile.class);
     final LogWriterI18n logger = context.mock(LogWriterI18n.class);
@@ -149,7 +156,6 @@ public class OplogRVVJUnitTest extends TestCase {
     Oplog oplog = new Oplog(1, oplogSet, dirHolder);
     oplog.close();
     
-    
     context.checking(new Expectations() {{
       one(drs).recordRecoveredGCVersion(m1, 1);
       one(drs).recordRecoveredGCVersion(m2, 0);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OrderedTombstoneMapJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OrderedTombstoneMapJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OrderedTombstoneMapJUnitTest.java
index bd50d05..29e26e1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OrderedTombstoneMapJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OrderedTombstoneMapJUnitTest.java
@@ -16,17 +16,19 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.cache.persistence.DiskStoreID;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class OrderedTombstoneMapJUnitTest extends TestCase {
-  
+public class OrderedTombstoneMapJUnitTest {
+
+  @Test
   public void test() {
     OrderedTombstoneMap<String> map = new OrderedTombstoneMap<String>();
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImplJUnitTest.java
index e75e17f..a8d8720 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImplJUnitTest.java
@@ -16,11 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 import java.util.Properties;
 
@@ -38,7 +34,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit test suite for PartitionAttributesImpl.
- * 
  */
 @Category(UnitTest.class)
 public class PartitionAttributesImplJUnitTest {
@@ -153,8 +148,8 @@ public class PartitionAttributesImplJUnitTest {
     destination.merge(instance);
     
     assertEquals(this.colocatedRegionFullPath, destination.getColocatedWith());
-    //assertEquals(this.globalProps, destination.getGlobalProperties());
-    //assertEquals(this.localProps, destination.getLocalProperties());
+    //assertIndexDetailsEquals(this.globalProps, destination.getGlobalProperties());
+    //assertIndexDetailsEquals(this.localProps, destination.getLocalProperties());
     assertEquals(this.partitionResolver, destination.getPartitionResolver());
     assertEquals(this.recoveryDelay, destination.getRecoveryDelay());
     assertEquals(this.redundancy, destination.getRedundancy());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCacheCloseDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCacheCloseDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCacheCloseDUnitTest.java
index 43252a2..17085c7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCacheCloseDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCacheCloseDUnitTest.java
@@ -237,7 +237,7 @@ public class PartitionedRegionCacheCloseDUnitTest extends
 //            VersionedArrayList nodeList = (VersionedArrayList)itr.next();
 //            logger.info("Size of nodeList for b2n entries for region = "
 //                + regionName + " = " + nodeList.size());
-//            assertEquals("Node list: " + nodeList, 1, nodeList.size());
+//            assertIndexDetailsEquals("Node list: " + nodeList, 1, nodeList.size());
 //          }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
index f401e0d..d964e5d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDUnitTestCase.java
@@ -235,7 +235,7 @@ public class PartitionedRegionDUnitTestCase extends CacheTestCase
           rootRegion.size());
 //        Region allPR = rootRegion.getSubregion(PartitionedRegionHelper.PARTITIONED_REGION_CONFIG_NAME);
 //        assertNotNull(allPR);
-//        assertEquals("allPR size is not correct", innerEndIndexForRegion,
+//        assertIndexDetailsEquals("allPR size is not correct", innerEndIndexForRegion,
 //      allPR.size());
         assertEquals("prIdToPR size is not correct", innerEndIndexForRegion,
             PartitionedRegion.prIdToPR.size());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDestroyDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDestroyDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDestroyDUnitTest.java
index 498203d..abfb7d2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDestroyDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDestroyDUnitTest.java
@@ -239,7 +239,7 @@ public class PartitionedRegionDestroyDUnitTest extends
         while (itr.hasNext()) {
           Region rg = (Region)itr.next();
           LogWriterUtils.getLogWriter().info("Root Region SubRegionName = " + rg.getName());
-//          assertEquals("REGION NAME FOUND:"+rg.getName(),-1, rg.getName().indexOf(
+//          assertIndexDetailsEquals("REGION NAME FOUND:"+rg.getName(),-1, rg.getName().indexOf(
 //              PartitionedRegionHelper.BUCKET_2_NODE_TABLE_PREFIX));
           assertEquals("regionFound that should be gone!:"+rg.getName(),-1, rg.getName().indexOf(
               PartitionedRegionHelper.BUCKET_REGION_PREFIX));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionEvictionDUnitTest.java
index 0319171..ce10abb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionEvictionDUnitTest.java
@@ -1109,7 +1109,7 @@ public class PartitionedRegionEvictionDUnitTest extends CacheTestCase {
 //          factory.setEvictionAttributes(okHeapLRUea);
 //          final Region pr = createRootRegion(name, factory.create());
 //          assertNotNull(pr);
-//          assertEquals(okHeapLRUea, pr.getAttributes().getEvictionAttributes());
+//          assertIndexDetailsEquals(okHeapLRUea, pr.getAttributes().getEvictionAttributes());
 //          pr.localDestroyRegion();
 //        }
 
@@ -1123,7 +1123,7 @@ public class PartitionedRegionEvictionDUnitTest extends CacheTestCase {
 //          factory.setEvictionAttributes(okHeapLRUea);
 //          final Region pr = createRootRegion(name, factory.create());
 //          assertNotNull(pr);
-//          assertEquals(okHeapLRUea, pr.getAttributes().getEvictionAttributes());
+//          assertIndexDetailsEquals(okHeapLRUea, pr.getAttributes().getEvictionAttributes());
 //          pr.localDestroyRegion();
 //        }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelperJUnitTest.java
index a904478..c1dbd78 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelperJUnitTest.java
@@ -16,18 +16,17 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class PartitionedRegionHelperJUnitTest extends TestCase {
-  
+public class PartitionedRegionHelperJUnitTest {
+
+  @Test
   public void testEscapeUnescape() {
     {
       String bucketName = PartitionedRegionHelper.getBucketName("/root/region", 5);
@@ -41,7 +40,5 @@ public class PartitionedRegionHelperJUnitTest extends TestCase {
       assertEquals("/root/region_one" , PartitionedRegionHelper.getPRPath(bucketName));
     }
   }
-  
-  
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionQueryEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionQueryEvaluatorTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionQueryEvaluatorTest.java
index fa4a26a..e3e8d4e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionQueryEvaluatorTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionQueryEvaluatorTest.java
@@ -16,12 +16,9 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -54,19 +51,19 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class PartitionedRegionQueryEvaluatorTest {
 
-  InternalDistributedMember localNode;
-  InternalDistributedMember remoteNodeA;
-  InternalDistributedMember remoteNodeB;
-  GemFireCacheImpl cache;
-  InternalDistributedSystem system;
-  PartitionedRegion pr;
-  DefaultQuery query;
+  private InternalDistributedMember localNode;
+  private InternalDistributedMember remoteNodeA;
+  private InternalDistributedMember remoteNodeB;
+  private GemFireCacheImpl cache;
+  private InternalDistributedSystem system;
+  private PartitionedRegion pr;
+  private DefaultQuery query;
   //Needed to help mock out certain scenarios
-  ExtendedPartitionedRegionDataStore dataStore;
+  private ExtendedPartitionedRegionDataStore dataStore;
   //This is the set of nodes that remain after a failure
-  ArrayList allNodes = new ArrayList();
+  private ArrayList allNodes = new ArrayList();
   //convenience list for empty set
-  Set<InternalDistributedMember> noFailingMembers = new HashSet<>();
+  private Set<InternalDistributedMember> noFailingMembers = new HashSet<>();
     
   @Before
   public void setup() throws Exception {
@@ -196,7 +193,7 @@ public class PartitionedRegionQueryEvaluatorTest {
     
   }
   
-  private  Map<InternalDistributedMember, List<Integer>> createFakeBucketMap() {
+  private Map<InternalDistributedMember, List<Integer>> createFakeBucketMap() {
     Map<InternalDistributedMember, List<Integer>> bucketToNodeMap = new HashMap<>();
     bucketToNodeMap.put(localNode, createBucketList(1, 2, 3));
     bucketToNodeMap.put(remoteNodeA, createBucketList(4, 5, 6));
@@ -205,7 +202,7 @@ public class PartitionedRegionQueryEvaluatorTest {
   }
   
   //fake bucket map to use after we fake a node failure
-  private  Map<InternalDistributedMember, List<Integer>> createFakeBucketMapFailedNodesToLocalMember() {
+  private Map<InternalDistributedMember, List<Integer>> createFakeBucketMapFailedNodesToLocalMember() {
     Map<InternalDistributedMember, List<Integer>> bucketToNodeMap = new HashMap<>();
     bucketToNodeMap.put(localNode, createBucketList(1, 2, 3, 7, 8, 9));
     bucketToNodeMap.put(remoteNodeA, createBucketList(4, 5, 6));
@@ -227,8 +224,16 @@ public class PartitionedRegionQueryEvaluatorTest {
     }
     return results;
   }
-  
-  class ExtendedPartitionedRegionDataStore extends PartitionedRegionDataStore {
+
+  private Queue<PartitionedQueryScenario> createScenariosQueue(PartitionedQueryScenario... scenarios) {
+    Queue<PartitionedQueryScenario> queue = new LinkedList<>();
+    for (PartitionedQueryScenario scenario: scenarios) {
+      queue.add(scenario);
+    }
+    return queue;
+  }
+
+  private static class ExtendedPartitionedRegionDataStore extends PartitionedRegionDataStore {
     //Must be the same referenced queue as that used by the ExtendedPartitionedRegionQueryEvaluator
     //That way they will be synched to the same scenario;
     Queue<PartitionedQueryScenario> scenarios;
@@ -243,8 +248,10 @@ public class PartitionedRegionQueryEvaluatorTest {
     }
   }
 
-  class ExtendedPartitionedRegionQueryEvaluator extends PartitionedRegionQueryEvaluator {
+  private static class ExtendedPartitionedRegionQueryEvaluator extends PartitionedRegionQueryEvaluator {
+
     Queue<PartitionedQueryScenario> scenarios;
+
     //pass through so we can fake out the executeQuery locally
     PRQueryProcessor extendedPRQueryProcessor;
     
@@ -260,8 +267,8 @@ public class PartitionedRegionQueryEvaluatorTest {
     }
 
     // (package access for unit test purposes)
-    Map<InternalDistributedMember, List<Integer>> buildNodeToBucketMap() throws QueryException
-    {
+    @Override
+    Map<InternalDistributedMember, List<Integer>> buildNodeToBucketMap() throws QueryException {
       return currentScenario().bucketMap;
     }
 
@@ -291,8 +298,8 @@ public class PartitionedRegionQueryEvaluatorTest {
     protected Set<InternalDistributedMember> getBucketOwners(Integer bid) {
       return currentScenario().getBucketOwners(bid);
     }
-    
-    class ExtendedPRQueryProcessor extends PRQueryProcessor {
+
+    private class ExtendedPRQueryProcessor extends PRQueryProcessor {
 
       public ExtendedPRQueryProcessor(PartitionedRegion pr, DefaultQuery query, Object[] parameters, List buckets) {
         super(pr, query, parameters, buckets);
@@ -309,7 +316,7 @@ public class PartitionedRegionQueryEvaluatorTest {
       }
     }
 
-    class FakeNumFailStreamingQueryPartitionResponse extends StreamingQueryPartitionResponse {
+    private class FakeNumFailStreamingQueryPartitionResponse extends StreamingQueryPartitionResponse {
       private PartitionedRegionQueryEvaluator processor;
       Queue<PartitionedQueryScenario> scenarios;
       
@@ -336,13 +343,13 @@ public class PartitionedRegionQueryEvaluatorTest {
     }
   }
   
-  interface ProcessDataFaker {
+  private interface ProcessDataFaker {
     void processData(PartitionedRegionQueryEvaluator processor);
     void executeQueryLocally(Collection resultsCollector);
   }
   
   //holds information on how the PRQE is to behave and what responses are "returned"
-  private class PartitionedQueryScenario {
+  private static class PartitionedQueryScenario {
     private InternalDistributedMember localNode;
     private ArrayList allNodes;
     private Set<InternalDistributedMember> failingNodes;
@@ -378,12 +385,4 @@ public class PartitionedRegionQueryEvaluatorTest {
     }
   }
   
-  private Queue<PartitionedQueryScenario> createScenariosQueue(PartitionedQueryScenario... scenarios) {
-    Queue<PartitionedQueryScenario> queue = new LinkedList<>();
-    for (PartitionedQueryScenario scenario: scenarios) {
-      queue.add(scenario);
-    }
-    return queue;
-    
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionRedundancyZoneDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionRedundancyZoneDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionRedundancyZoneDUnitTest.java
index 027ae30..a2dafbf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionRedundancyZoneDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionRedundancyZoneDUnitTest.java
@@ -85,9 +85,9 @@ public class PartitionedRegionRedundancyZoneDUnitTest extends CacheTestCase {
     //member to bring it's primary count up to two.
     assertTrue(counts, vm0Count >= 2 && vm1Count >=2);
     
-//    assertEquals(counts, 3, vm0Count);
-//    assertEquals(counts, 3, vm1Count);
-//    assertEquals(counts, 6, vm2Count);
+//    assertIndexDetailsEquals(counts, 3, vm0Count);
+//    assertIndexDetailsEquals(counts, 3, vm1Count);
+//    assertIndexDetailsEquals(counts, 6, vm2Count);
   }
   
   protected void checkBucketCount(VM vm0, final int numLocalBuckets) {



[03/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
index 94559d6..56a93db 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
@@ -14,340 +14,334 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.offheap;
 
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
 import com.gemstone.gemfire.internal.cache.CachePerfStats;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.RegionEntryContext;
-import com.gemstone.gemfire.internal.offheap.TinyStoredObject;
-
-import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mock;
-
-import java.nio.ByteBuffer;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.*;
 
 @Category(UnitTest.class)
 public class TinyStoredObjectJUnitTest extends AbstractStoredObjectTestBase {
 
-    @Override
-    public Object getValue() {
-        return Integer.valueOf(123456789);
+  @Override
+  public Object getValue() {
+    return Integer.valueOf(123456789);
+  }
+
+  @Override
+  public byte[] getValueAsByteArray() {
+    return convertValueToByteArray(getValue());
+  }
+
+  private byte[] convertValueToByteArray(Object value) {
+    return ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt((Integer) value).array();
+  }
+
+  @Override
+  public Object convertByteArrayToObject(byte[] valueInByteArray) {
+    return ByteBuffer.wrap(valueInByteArray).getInt();
+  }
+
+  @Override
+  public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
+    return EntryEventImpl.deserialize(valueInSerializedByteArray);
+  }
+
+  @Override
+  public TinyStoredObject createValueAsUnserializedStoredObject(Object value) {
+    byte[] valueInByteArray;
+    if (value instanceof Integer) {
+      valueInByteArray = convertValueToByteArray(value);
+    } else {
+      valueInByteArray = (byte[]) value;
     }
+    //encode a non-serialized entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInByteArray, false, false);
+    return new TinyStoredObject(encodedAddress);
+  }
+
+  @Override
+  public TinyStoredObject createValueAsSerializedStoredObject(Object value) {
+    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+    //encode a serialized entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
+    return new TinyStoredObject(encodedAddress);
+  }
+
+  public TinyStoredObject createValueAsCompressedStoredObject(Object value) {
+    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+    //encode a serialized, compressed entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, true);
+    return new TinyStoredObject(encodedAddress);
+  }
+
+  public TinyStoredObject createValueAsUncompressedStoredObject(Object value) {
+    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+    //encode a serialized, uncompressed entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
+    return new TinyStoredObject(encodedAddress);
+  }
+
+  @Test
+  public void shouldReturnCorrectEncodingAddress() {
+
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    assertNotNull(address1);
+    assertEquals("Encoding address should be:", 10001, address1.getAddress());
+
+    TinyStoredObject address2 = new TinyStoredObject(10002L);
+    assertNotNull(address2);
+    assertEquals("Returning always 10001 expected 10002", 10002, address2.getAddress());
+  }
 
-    @Override
-    public byte[] getValueAsByteArray() {
-        return convertValueToByteArray(getValue());
-    }
+  @Test
+  public void twoAddressesShouldBeEqualIfEncodingAddressIsSame() {
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    TinyStoredObject address2 = new TinyStoredObject(10001L);
+
+    assertEquals("Two addresses are equal if encoding address is same", true, address1.equals(address2));
+  }
 
-    private byte[] convertValueToByteArray(Object value) {
-        return ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt((Integer) value).array();
-    }
+  @Test
+  public void twoAddressesShouldNotBeEqualIfEncodingAddressIsNotSame() {
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    TinyStoredObject address2 = new TinyStoredObject(10002L);
+
+    assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
+  }
 
-    @Override
-    public Object convertByteArrayToObject(byte[] valueInByteArray) {
-        return ByteBuffer.wrap(valueInByteArray).getInt();
-    }
+  @Test
+  public void twoAddressesAreNotEqualIfTheyAreNotTypeDataAsAddress() {
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    Long address2 = new Long(10002L);
+
+    assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
+  }
 
-    @Override
-    public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
-       return EntryEventImpl.deserialize(valueInSerializedByteArray);
-    }
+  @Test
+  public void addressHashCodeShouldBe() {
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    assertEquals("", 10001, address1.hashCode());
+  }
 
-    @Override
-    public TinyStoredObject createValueAsUnserializedStoredObject(Object value) {
-        byte[] valueInByteArray;
-        if(value instanceof Integer) {
-            valueInByteArray = convertValueToByteArray(value);
-        } else {
-            valueInByteArray = (byte[]) value;
-        }
-        //encode a non-serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInByteArray, false, false);
-        return new TinyStoredObject(encodedAddress);
-    }
+  @Test
+  public void getSizeInBytesAlwaysReturnsZero() {
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    TinyStoredObject address2 = new TinyStoredObject(10002L);
 
-    @Override
-    public TinyStoredObject createValueAsSerializedStoredObject(Object value) {
-        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-        //encode a serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
-        return new TinyStoredObject(encodedAddress);
-    }
+    assertEquals("getSizeInBytes", 0, address1.getSizeInBytes());
+    assertEquals("getSizeInBytes", 0, address2.getSizeInBytes());
+  }
 
-    public TinyStoredObject createValueAsCompressedStoredObject(Object value) {
-        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-        //encode a serialized, compressed entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, true);
-        return new TinyStoredObject(encodedAddress);
-    }
+  @Test
+  public void getValueSizeInBytesAlwaysReturnsZero() {
+    TinyStoredObject address1 = new TinyStoredObject(10001L);
+    TinyStoredObject address2 = new TinyStoredObject(10002L);
 
-    public TinyStoredObject createValueAsUncompressedStoredObject(Object value) {
-        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-        //encode a serialized, uncompressed entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
-        return new TinyStoredObject(encodedAddress);
-    }
+    assertEquals("getSizeInBytes", 0, address1.getValueSizeInBytes());
+    assertEquals("getSizeInBytes", 0, address2.getValueSizeInBytes());
+  }
 
-    @Test
-    public void shouldReturnCorrectEncodingAddress() {
+  @Test
+  public void isCompressedShouldReturnTrueIfCompressed() {
+    Object regionEntryValue = getValue();
 
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        assertNotNull(address1);
-        assertEquals("Encoding address should be:", 10001, address1.getAddress());
+    TinyStoredObject offheapAddress = createValueAsCompressedStoredObject(regionEntryValue);
 
-        TinyStoredObject address2 = new TinyStoredObject(10002L);
-        assertNotNull(address2);
-        assertEquals("Returning always 10001 expected 10002", 10002, address2.getAddress());
-    }
+    assertEquals("Should return true as it is compressed", true, offheapAddress.isCompressed());
+  }
 
-    @Test
-    public void twoAddressesShouldBeEqualIfEncodingAddressIsSame() {
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        TinyStoredObject address2 = new TinyStoredObject(10001L);
+  @Test
+  public void isCompressedShouldReturnFalseIfNotCompressed() {
+    Object regionEntryValue = getValue();
 
-        assertEquals("Two addresses are equal if encoding address is same", true, address1.equals(address2));
-    }
+    TinyStoredObject offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
 
-    @Test
-    public void twoAddressesShouldNotBeEqualIfEncodingAddressIsNotSame() {
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        TinyStoredObject address2 = new TinyStoredObject(10002L);
+    assertEquals("Should return false as it is compressed", false, offheapAddress.isCompressed());
+  }
 
-        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
-    }
+  @Test
+  public void isSerializedShouldReturnTrueIfSeriazlied() {
+    Object regionEntryValue = getValue();
 
-    @Test
-    public void twoAddressesAreNotEqualIfTheyAreNotTypeDataAsAddress() {
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        Long address2 = new Long(10002L);
+    TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
 
-        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
-    }
+    assertEquals("Should return true as it is serialized", true, offheapAddress.isSerialized());
+  }
 
-    @Test
-    public void addressHashCodeShouldBe() {
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        assertEquals("", 10001, address1.hashCode());
-    }
+  @Test
+  public void isSerializedShouldReturnFalseIfNotSeriazlied() {
+    Object regionEntryValue = getValue();
 
-    @Test
-    public void getSizeInBytesAlwaysReturnsZero() {
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        TinyStoredObject address2 = new TinyStoredObject(10002L);
+    TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValue);
 
-        assertEquals("getSizeInBytes", 0, address1.getSizeInBytes());
-        assertEquals("getSizeInBytes", 0, address2.getSizeInBytes());
-    }
+    assertEquals("Should return false as it is serialized", false, offheapAddress.isSerialized());
+  }
 
-    @Test
-    public void getValueSizeInBytesAlwaysReturnsZero() {
-        TinyStoredObject address1 = new TinyStoredObject(10001L);
-        TinyStoredObject address2 = new TinyStoredObject(10002L);
+  @Test
+  public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
 
-        assertEquals("getSizeInBytes", 0, address1.getValueSizeInBytes());
-        assertEquals("getSizeInBytes", 0, address2.getValueSizeInBytes());
-    }
-
-    @Test
-    public void isCompressedShouldReturnTrueIfCompressed() {
-        Object regionEntryValue = getValue();
-
-        TinyStoredObject offheapAddress = createValueAsCompressedStoredObject(regionEntryValue);
-
-        assertEquals("Should return true as it is compressed", true, offheapAddress.isCompressed());
-    }
-
-    @Test
-    public void isCompressedShouldReturnFalseIfNotCompressed() {
-        Object regionEntryValue = getValue();
-
-        TinyStoredObject offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
-
-        assertEquals("Should return false as it is compressed", false, offheapAddress.isCompressed());
-    }
-
-    @Test
-    public void isSerializedShouldReturnTrueIfSeriazlied() {
-        Object regionEntryValue = getValue();
-
-        TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
-
-        assertEquals("Should return true as it is serialized", true, offheapAddress.isSerialized());
-    }
-
-    @Test
-    public void isSerializedShouldReturnFalseIfNotSeriazlied() {
-        Object regionEntryValue = getValue();
+    //encode a non-serialized and compressed entry value to address - last argument is to let that it is compressed
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, true);
+    TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
 
-        TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValue);
+    RegionEntryContext regionContext = mock(RegionEntryContext.class);
+    CachePerfStats cacheStats = mock(CachePerfStats.class);
+    Compressor compressor = mock(Compressor.class);
 
-        assertEquals("Should return false as it is serialized", false, offheapAddress.isSerialized());
-    }
-
-    @Test
-    public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
-        Object regionEntryValue = getValue();
-        byte[] regionEntryValueAsBytes =  convertValueToByteArray(regionEntryValue);
+    long startTime = 10000L;
 
-        //encode a non-serialized and compressed entry value to address - last argument is to let that it is compressed
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, true);
-        TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
+    //mock required things
+    when(regionContext.getCompressor()).thenReturn(compressor);
+    when(compressor.decompress(regionEntryValueAsBytes)).thenReturn(regionEntryValueAsBytes);
+    when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
+    when(cacheStats.startDecompression()).thenReturn(startTime);
 
-        RegionEntryContext regionContext = mock(RegionEntryContext.class);
-        CachePerfStats cacheStats = mock(CachePerfStats.class);
-        Compressor compressor = mock(Compressor.class);
+    //invoke the thing
+    byte[] bytes = offheapAddress.getDecompressedBytes(regionContext);
 
-        long startTime = 10000L;
+    //verify the thing happened
+    verify(cacheStats, atLeastOnce()).startDecompression();
+    verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
+    verify(cacheStats, atLeastOnce()).endDecompression(startTime);
 
-        //mock required things
-        when(regionContext.getCompressor()).thenReturn(compressor);
-        when(compressor.decompress(regionEntryValueAsBytes)).thenReturn(regionEntryValueAsBytes);
-        when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
-        when(cacheStats.startDecompression()).thenReturn(startTime);
+    assertArrayEquals(regionEntryValueAsBytes, bytes);
+  }
 
-        //invoke the thing
-        byte[] bytes = offheapAddress.getDecompressedBytes(regionContext);
+  @Test
+  public void getDecompressedBytesShouldNotTryToDecompressIfNotCompressed() {
+    Object regionEntryValue = getValue();
 
-        //verify the thing happened
-        verify(cacheStats, atLeastOnce()).startDecompression();
-        verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
-        verify(cacheStats, atLeastOnce()).endDecompression(startTime);
-
-        assertArrayEquals(regionEntryValueAsBytes, bytes);
-    }
+    TinyStoredObject offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
 
-    @Test
-    public void getDecompressedBytesShouldNotTryToDecompressIfNotCompressed() {
-        Object regionEntryValue = getValue();
+    //mock the thing
+    RegionEntryContext regionContext = mock(RegionEntryContext.class);
+    Compressor compressor = mock(Compressor.class);
+    when(regionContext.getCompressor()).thenReturn(compressor);
 
-        TinyStoredObject offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
+    //invoke the thing
+    byte[] actualValueInBytes = offheapAddress.getDecompressedBytes(regionContext);
 
-        //mock the thing
-        RegionEntryContext regionContext = mock(RegionEntryContext.class);
-        Compressor compressor = mock(Compressor.class);
-        when(regionContext.getCompressor()).thenReturn(compressor);
+    //createValueAsUncompressedStoredObject does uses a serialized value - so convert it to object
+    Object actualRegionValue = convertSerializedByteArrayToObject(actualValueInBytes);
 
-        //invoke the thing
-        byte[] actualValueInBytes = offheapAddress.getDecompressedBytes(regionContext);
-
-        //createValueAsUncompressedStoredObject does uses a serialized value - so convert it to object
-        Object actualRegionValue = convertSerializedByteArrayToObject(actualValueInBytes);
-
-        //verify the thing happened
-        verify(regionContext, never()).getCompressor();
-        assertEquals(regionEntryValue, actualRegionValue);
-    }
+    //verify the thing happened
+    verify(regionContext, never()).getCompressor();
+    assertEquals(regionEntryValue, actualRegionValue);
+  }
 
-    @Test
-    public void getRawBytesShouldReturnAByteArray() {
-        byte[] regionEntryValueAsBytes = getValueAsByteArray();
+  @Test
+  public void getRawBytesShouldReturnAByteArray() {
+    byte[] regionEntryValueAsBytes = getValueAsByteArray();
 
-        TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
-        byte[] actual = offheapAddress.getRawBytes();
+    TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
+    byte[] actual = offheapAddress.getRawBytes();
 
-        assertArrayEquals(regionEntryValueAsBytes, actual);
-    }
+    assertArrayEquals(regionEntryValueAsBytes, actual);
+  }
 
-    @Test
-    public void getSerializedValueShouldReturnASerializedByteArray() {
-        Object regionEntryValue = getValue();
+  @Test
+  public void getSerializedValueShouldReturnASerializedByteArray() {
+    Object regionEntryValue = getValue();
 
-        TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+    TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
 
-        byte[] actualSerializedValue = offheapAddress.getSerializedValue();
+    byte[] actualSerializedValue = offheapAddress.getSerializedValue();
 
-        Object actualRegionEntryValue = convertSerializedByteArrayToObject(actualSerializedValue);
+    Object actualRegionEntryValue = convertSerializedByteArrayToObject(actualSerializedValue);
 
-        assertEquals(regionEntryValue, actualRegionEntryValue);
-    }
+    assertEquals(regionEntryValue, actualRegionEntryValue);
+  }
 
-    @Test
-    public void getDeserializedObjectShouldReturnADeserializedObject() {
-        Object regionEntryValue = getValue();
+  @Test
+  public void getDeserializedObjectShouldReturnADeserializedObject() {
+    Object regionEntryValue = getValue();
 
-        TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+    TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
 
-        Integer actualRegionEntryValue = (Integer) offheapAddress.getDeserializedValue(null, null);
+    Integer actualRegionEntryValue = (Integer) offheapAddress.getDeserializedValue(null, null);
 
-        assertEquals(regionEntryValue, actualRegionEntryValue);
-    }
+    assertEquals(regionEntryValue, actualRegionEntryValue);
+  }
 
-    @Test
-    public void getDeserializedObjectShouldReturnAByteArrayAsIsIfNotSerialized() {
-        byte[] regionEntryValueAsBytes = getValueAsByteArray();
+  @Test
+  public void getDeserializedObjectShouldReturnAByteArrayAsIsIfNotSerialized() {
+    byte[] regionEntryValueAsBytes = getValueAsByteArray();
 
-        TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
+    TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
 
-        byte[] deserializeValue = (byte[]) offheapAddress.getDeserializedValue(null, null);
+    byte[] deserializeValue = (byte[]) offheapAddress.getDeserializedValue(null, null);
 
-        assertArrayEquals(regionEntryValueAsBytes, deserializeValue);
-    }
+    assertArrayEquals(regionEntryValueAsBytes, deserializeValue);
+  }
 
-    @Test
-    public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
-        Object regionEntryValue = getValue();
-        byte[] serializedRegionEntryValue = EntryEventImpl.serialize(regionEntryValue);
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
+    Object regionEntryValue = getValue();
+    byte[] serializedRegionEntryValue = EntryEventImpl.serialize(regionEntryValue);
 
-        //encode a serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedRegionEntryValue, true, false);
+    //encode a serialized entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedRegionEntryValue, true, false);
 
-        TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
+    TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
 
-        //mock the things
-        BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+    //mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
 
-        byte userBits = 1;
-        offheapAddress.fillSerializedValue(wrapper, userBits);
+    byte userBits = 1;
+    offheapAddress.fillSerializedValue(wrapper, userBits);
 
-        verify(wrapper, times(1)).setData(serializedRegionEntryValue, userBits, serializedRegionEntryValue.length, true);
-    }
+    verify(wrapper, times(1)).setData(serializedRegionEntryValue, userBits, serializedRegionEntryValue.length, true);
+  }
 
-    @Test
-    public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
-        Object regionEntryValue = getValue();
-        byte[] regionEntryValueAsBytes =  convertValueToByteArray(regionEntryValue);
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
 
-        //encode a un serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, false);
+    //encode a un serialized entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, false);
 
-        TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
+    TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
 
-        //mock the things
-        BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+    //mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
 
-        byte userBits = 1;
-        offheapAddress.fillSerializedValue(wrapper, userBits);
+    byte userBits = 1;
+    offheapAddress.fillSerializedValue(wrapper, userBits);
 
-        verify(wrapper, times(1)).setData(regionEntryValueAsBytes, userBits, regionEntryValueAsBytes.length, true);
-    }
+    verify(wrapper, times(1)).setData(regionEntryValueAsBytes, userBits, regionEntryValueAsBytes.length, true);
+  }
 
-    @Test
-    public void getStringFormShouldCatchExceptionAndReturnErrorMessageAsString() {
-        Object regionEntryValueAsBytes = getValue();
+  @Test
+  public void getStringFormShouldCatchExceptionAndReturnErrorMessageAsString() {
+    Object regionEntryValueAsBytes = getValue();
 
-        byte[] serializedValue = EntryEventImpl.serialize(regionEntryValueAsBytes);
+    byte[] serializedValue = EntryEventImpl.serialize(regionEntryValueAsBytes);
 
-        //store -127 (DSCODE.ILLEGAL) - in order the deserialize to throw exception
-        serializedValue[0] = -127;
+    //store -127 (DSCODE.ILLEGAL) - in order the deserialize to throw exception
+    serializedValue[0] = -127;
 
-        //encode a un serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedValue, true, false);
+    //encode a un serialized entry value to address
+    long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedValue, true, false);
 
-        TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
+    TinyStoredObject offheapAddress = new TinyStoredObject(encodedAddress);
 
-        String errorMessage = offheapAddress.getStringForm();
+    String errorMessage = offheapAddress.getStringForm();
 
-        assertEquals(true, errorMessage.contains("Could not convert object to string because "));
-    }
+    assertEquals(true, errorMessage.contains("Could not convert object to string because "));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/process/LocalProcessLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/process/LocalProcessLauncherJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/process/LocalProcessLauncherJUnitTest.java
index 78dc6bb..3ff9936 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/process/LocalProcessLauncherJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/process/LocalProcessLauncherJUnitTest.java
@@ -24,26 +24,36 @@ import java.io.FileReader;
 import java.io.FileWriter;
 
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
 
 import com.gemstone.gemfire.internal.OSProcess;
-import com.gemstone.gemfire.internal.process.LocalProcessLauncher;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Unit tests for ProcessLauncher.
  * 
  * @since 7.0
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class LocalProcessLauncherJUnitTest {
 
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  private File pidFile;
+
   @Before
-  public void createDirectory() throws Exception {
-    new File(getClass().getSimpleName()).mkdir();
+  public void setUp() throws Exception {
+    this.pidFile = new File(this.temporaryFolder.getRoot(), testName.getMethodName() + ".pid");
   }
-  
+
   @Test
   public void testPidAccuracy() throws PidUnavailableException {
     int pid = ProcessUtils.identifyPid();
@@ -58,8 +68,6 @@ public class LocalProcessLauncherJUnitTest {
   
   @Test
   public void testPidFileIsCreated() throws Exception {
-    final File pidFile = new File(getClass().getSimpleName() 
-        + File.separator + "testPidFileIsCreated.pid");
     assertFalse(pidFile.exists());
     new LocalProcessLauncher(pidFile, false);
     assertTrue(pidFile.exists());
@@ -67,8 +75,6 @@ public class LocalProcessLauncherJUnitTest {
   
   @Test
   public void testPidFileContainsPid() throws Exception {
-    final File pidFile = new File(getClass().getSimpleName() 
-        + File.separator + "testPidFileContainsPid.pid");
     final LocalProcessLauncher launcher = new LocalProcessLauncher(pidFile, false);
     assertNotNull(launcher);
     assertTrue(pidFile.exists());
@@ -85,8 +91,6 @@ public class LocalProcessLauncherJUnitTest {
   
   @Test
   public void testPidFileIsCleanedUp() throws Exception {
-    final File pidFile = new File(getClass().getSimpleName() 
-        + File.separator + "testPidFileIsCleanedUp.pid");
     final LocalProcessLauncher launcher = new LocalProcessLauncher(pidFile, false);
     assertTrue(pidFile.exists());
     launcher.close(); // TODO: launch an external JVM and then close it nicely
@@ -95,8 +99,6 @@ public class LocalProcessLauncherJUnitTest {
   
   @Test
   public void testExistingPidFileThrows() throws Exception {
-    final File pidFile = new File(getClass().getSimpleName() 
-        + File.separator + "testExistingPidFileThrows.pid");
     assertTrue(pidFile.createNewFile());
     assertTrue(pidFile.exists());
     
@@ -115,8 +117,6 @@ public class LocalProcessLauncherJUnitTest {
 
   @Test
   public void testStalePidFileIsReplaced() throws Exception {
-    final File pidFile = new File(getClass().getSimpleName() 
-        + File.separator + "testStalePidFileIsReplaced.pid");
     assertTrue(pidFile.createNewFile());
     assertTrue(pidFile.exists());
     
@@ -143,9 +143,7 @@ public class LocalProcessLauncherJUnitTest {
   public void testForceReplacesExistingPidFile() throws Exception {
     assertTrue("testForceReplacesExistingPidFile is broken if PID == Integer.MAX_VALUE", 
         ProcessUtils.identifyPid() != Integer.MAX_VALUE);
-    
-    final File pidFile = new File(getClass().getSimpleName() 
-        + File.separator + "testForceReplacesExistingPidFile.pid");
+
     assertTrue(pidFile.createNewFile());
     assertTrue(pidFile.exists());
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
index e74365e..bd9bf62 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
@@ -41,7 +41,7 @@ import org.junit.rules.ExpectedException;
 import org.junit.rules.TemporaryFolder;
 
 import com.gemstone.gemfire.internal.util.StopWatch;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.gemstone.gemfire.test.junit.rules.ExpectedTimeoutRule;
 
 /**
@@ -49,7 +49,7 @@ import com.gemstone.gemfire.test.junit.rules.ExpectedTimeoutRule;
  * 
  * @since 8.2
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class PidFileJUnitTest {
 
   @Rule

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
index 91af22c..0852c9d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
@@ -16,26 +16,19 @@
  */
 package com.gemstone.gemfire.internal.size;
 
+import static com.gemstone.gemfire.internal.size.SizeTestUtil.*;
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-import static com.gemstone.gemfire.internal.size.SizeTestUtil.*;
-
 @Category(UnitTest.class)
-public class ObjectSizerJUnitTest extends TestCase {
-  
-  
-  public ObjectSizerJUnitTest() {
-    super();
-  }
-
-  public ObjectSizerJUnitTest(String name) {
-    super(name);
-  }
+public class ObjectSizerJUnitTest {
 
-  public void test() throws IllegalArgumentException, IllegalAccessException {
+  @Test
+  public void test() throws Exception {
     assertEquals(roundup(OBJECT_SIZE), ObjectGraphSizer.size(new Object()));
     
     assertEquals(roundup(OBJECT_SIZE + 4), ObjectGraphSizer.size(new TestObject1()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectTraverserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectTraverserJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectTraverserJUnitTest.java
index bd0bf9d..87ca34f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectTraverserJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectTraverserJUnitTest.java
@@ -16,24 +16,25 @@
  */
 package com.gemstone.gemfire.internal.size;
 
+import static org.junit.Assert.*;
+
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.Map;
 import java.util.Set;
 
+import org.junit.Ignore;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class ObjectTraverserJUnitTest extends TestCase {
-  
-  public void testBasic() throws IllegalArgumentException, IllegalAccessException {
-    
+public class ObjectTraverserJUnitTest {
+
+  @Test
+  public void testBasic() throws Exception {
     Set testData = new HashSet();
     Object one = new Object();
     testData.add(one);
@@ -46,36 +47,35 @@ public class ObjectTraverserJUnitTest extends TestCase {
     TestVisitor visitor = new TestVisitor();
     ObjectTraverser.breadthFirstSearch(testData, visitor, false);
     
-    Assert.assertNotNull(visitor.visited.remove(testData));
-    Assert.assertNotNull(visitor.visited.remove(one));
-    Assert.assertNotNull(visitor.visited.remove(two));
-    Assert.assertNotNull(visitor.visited.remove(three));
+    assertNotNull(visitor.visited.remove(testData));
+    assertNotNull(visitor.visited.remove(one));
+    assertNotNull(visitor.visited.remove(two));
+    assertNotNull(visitor.visited.remove(three));
   }
-  
-  public void testStatics() throws IllegalArgumentException, IllegalAccessException {
-   
+
+  @Test
+  public void testStatics() throws Exception {
     final Object staticObject = new Object();
     TestObject1.test2 = staticObject;
     TestObject1 test1 = new TestObject1();
     
     TestVisitor visitor = new TestVisitor();
     ObjectTraverser.breadthFirstSearch(test1, visitor, false);
-    Assert.assertNull(visitor.visited.get(staticObject));
+    assertNull(visitor.visited.get(staticObject));
     
     visitor = new TestVisitor();
     ObjectTraverser.breadthFirstSearch(test1, visitor, true);
-    Assert.assertNotNull(visitor.visited.get(staticObject));
+    assertNotNull(visitor.visited.get(staticObject));
   }
-  
-  public void testStop() throws IllegalArgumentException, IllegalAccessException {
+
+  @Test
+  public void testStop() throws Exception {
     Set set1 = new HashSet();
     final Set set2 = new HashSet();
     Object object3 = new Object();
     set1.add(set2);
     set2.add(object3);
     
-    
-    
     TestVisitor visitor = new TestVisitor();
     visitor = new TestVisitor() {
       public boolean visit(Object parent, Object object) {
@@ -86,14 +86,15 @@ public class ObjectTraverserJUnitTest extends TestCase {
     
     ObjectTraverser.breadthFirstSearch(set1, visitor, true);
     
-    Assert.assertNotNull(visitor.visited.get(set1));
-    Assert.assertNotNull(visitor.visited.get(set2));
-    Assert.assertNull(visitor.visited.get(object3));
+    assertNotNull(visitor.visited.get(set1));
+    assertNotNull(visitor.visited.get(set2));
+    assertNull(visitor.visited.get(object3));
   }
   
   /** This test is commented out because it needs to be verified manually */
-  public void z_testHistogram() throws IllegalArgumentException, IllegalAccessException {
-    
+  @Ignore("commented out because it needs to be verified manually")
+  @Test
+  public void testHistogram() throws Exception {
     Set set1 = new HashSet();
     final Set set2 = new HashSet();
     Object object3 = new Object();
@@ -110,7 +111,7 @@ public class ObjectTraverserJUnitTest extends TestCase {
     public Map visited = new IdentityHashMap();
 
     public boolean visit(Object parent, Object object) {
-      Assert.assertNull(visited.put(object, VALUE));
+      assertNull(visited.put(object, VALUE));
       return true;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ReflectionObjectSizerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ReflectionObjectSizerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ReflectionObjectSizerJUnitTest.java
index 4489433..7618e36 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ReflectionObjectSizerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/ReflectionObjectSizerJUnitTest.java
@@ -17,29 +17,26 @@
 package com.gemstone.gemfire.internal.size;
 
 import static org.junit.Assert.*;
-
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.mockito.Mockito.*;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import io.codearte.catchexception.shade.mockito.Mockito;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class ReflectionObjectSizerJUnitTest {
 
   @Test
   public void skipsSizingDistributedSystem() {
-
-    Object referenceObject = Mockito.mock(InternalDistributedSystem.class);
+    Object referenceObject = mock(InternalDistributedSystem.class);
     checkSizeDoesNotChange(referenceObject);
   }
 
   @Test
   public void skipsSizingClassLoader() {
-
     checkSizeDoesNotChange(Thread.currentThread().getContextClassLoader());
   }
 
@@ -59,7 +56,7 @@ public class ReflectionObjectSizerJUnitTest {
     assertNotEquals(sizeWithoutReference, sizer.sizeof(stringReference));
   }
 
-  public class TestObject {
+  private static class TestObject {
 
     public TestObject(final Object reference) {
       this.reference = reference;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/size/SizeClassOnceObjectSizerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/SizeClassOnceObjectSizerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/SizeClassOnceObjectSizerJUnitTest.java
index 8f41b21..5f7bb1b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/SizeClassOnceObjectSizerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/SizeClassOnceObjectSizerJUnitTest.java
@@ -16,20 +16,19 @@
  */
 package com.gemstone.gemfire.internal.size;
 
-import org.junit.experimental.categories.Category;
-
-import junit.framework.TestCase;
 import static com.gemstone.gemfire.internal.size.SizeTestUtil.*;
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
-public class SizeClassOnceObjectSizerJUnitTest extends TestCase{
-  
+public class SizeClassOnceObjectSizerJUnitTest {
+
+  @Test
   public void test() {
     byte[] b1 = new byte[5];
     byte[] b2 = new byte[15];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/size/WellKnownClassSizerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/WellKnownClassSizerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/WellKnownClassSizerJUnitTest.java
index 76655fb..e25ae4c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/size/WellKnownClassSizerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/size/WellKnownClassSizerJUnitTest.java
@@ -16,20 +16,19 @@
  */
 package com.gemstone.gemfire.internal.size;
 
+import static com.gemstone.gemfire.internal.size.SizeTestUtil.*;
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-import static com.gemstone.gemfire.internal.size.SizeTestUtil.*;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class WellKnownClassSizerJUnitTest extends TestCase {
-  
+public class WellKnownClassSizerJUnitTest {
+
+  @Test
   public void testByteArrays() {
     byte[] test1 = new byte[5];
     byte[] test2 = new byte[8];
@@ -41,7 +40,8 @@ public class WellKnownClassSizerJUnitTest extends TestCase {
     
     assertEquals(0, WellKnownClassSizer.sizeof(new Object()));
   }
-  
+
+  @Test
   public void testStrings() {
     String test1 = "123";
     String test2 = "012345678";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/SampleCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/SampleCollectorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/SampleCollectorJUnitTest.java
index dcf9420..040fc85 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/SampleCollectorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/SampleCollectorJUnitTest.java
@@ -16,22 +16,18 @@
  */
 package com.gemstone.gemfire.internal.statistics;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.io.File;
 import java.util.List;
 
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
 
 import com.gemstone.gemfire.StatisticDescriptor;
 import com.gemstone.gemfire.Statistics;
@@ -51,40 +47,24 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class SampleCollectorJUnitTest {
 
-  private static final String dir = "SampleCollectorJUnitTest";
-
-  private Mockery mockContext;
-  private TestStatisticsManager manager; 
+  private TestStatisticsManager manager;
   private SampleCollector sampleCollector;
   
   @Before
   public void setUp() throws Exception {
-    new File(dir).mkdir();
-    this.mockContext = new Mockery() {{
-      setImposteriser(ClassImposteriser.INSTANCE);
-    }};
     final long startTime = System.currentTimeMillis();
-    this.manager = new TestStatisticsManager(1, "SampleCollectorJUnitTest", startTime);
-    
-    final StatArchiveHandlerConfig mockStatArchiveHandlerConfig = this.mockContext.mock(StatArchiveHandlerConfig.class, "SampleCollectorJUnitTest$StatArchiveHandlerConfig");
-    this.mockContext.checking(new Expectations() {{
-      allowing(mockStatArchiveHandlerConfig).getArchiveFileName();
-      will(returnValue(new File("")));
-      allowing(mockStatArchiveHandlerConfig).getArchiveFileSizeLimit();
-      will(returnValue(0));
-      allowing(mockStatArchiveHandlerConfig).getArchiveDiskSpaceLimit();
-      will(returnValue(0));
-      allowing(mockStatArchiveHandlerConfig).getSystemId();
-      will(returnValue(1));
-      allowing(mockStatArchiveHandlerConfig).getSystemStartTime();
-      will(returnValue(startTime));
-      allowing(mockStatArchiveHandlerConfig).getSystemDirectoryPath();
-      will(returnValue(""));
-      allowing(mockStatArchiveHandlerConfig).getProductDescription();
-      will(returnValue("SampleCollectorJUnitTest"));
-    }});
+    this.manager = new TestStatisticsManager(1, getClass().getSimpleName(), startTime);
+
+    final StatArchiveHandlerConfig mockStatArchiveHandlerConfig = mock(StatArchiveHandlerConfig.class, getClass().getSimpleName() + "$" + StatArchiveHandlerConfig.class.getSimpleName());
+    when(mockStatArchiveHandlerConfig.getArchiveFileName()).thenReturn(new File(""));
+    when(mockStatArchiveHandlerConfig.getArchiveFileSizeLimit()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getArchiveDiskSpaceLimit()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getSystemId()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getSystemStartTime()).thenReturn(startTime);
+    when(mockStatArchiveHandlerConfig.getSystemDirectoryPath()).thenReturn("");
+    when(mockStatArchiveHandlerConfig.getProductDescription()).thenReturn(getClass().getSimpleName());
 
-    StatisticsSampler sampler = new TestStatisticsSampler(manager);
+    final StatisticsSampler sampler = new TestStatisticsSampler(manager);
     this.sampleCollector = new SampleCollector(sampler);
     this.sampleCollector.initialize(mockStatArchiveHandlerConfig, NanoTimer.getTime());
   }
@@ -95,8 +75,6 @@ public class SampleCollectorJUnitTest {
       this.sampleCollector.close();
       this.sampleCollector = null;
     }
-    this.mockContext.assertIsSatisfied();
-    this.mockContext = null;
     this.manager = null;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatMonitorHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatMonitorHandlerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatMonitorHandlerJUnitTest.java
index 6c88fd4..0266e86 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatMonitorHandlerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatMonitorHandlerJUnitTest.java
@@ -27,29 +27,19 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.NanoTimer;
-import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.internal.logging.PureLogWriter;
 import com.gemstone.gemfire.internal.statistics.StatMonitorHandler.StatMonitorNotifier;
 import com.gemstone.gemfire.internal.util.StopWatch;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
- * Unit test for the StatMonitorHandler and its inner classes.
+ * Unit tests for the StatMonitorHandler and its inner classes.
  *   
  * @since 7.0
  */
 @Category(UnitTest.class)
 public class StatMonitorHandlerJUnitTest {
 
-  private LogWriterI18n log = null;
-
-  @Before
-  public void setUp() throws Exception {
-    this.log = new PureLogWriter(LogWriterImpl.levelNameToCode("config"));
-  }
-
   @Test
   public void testAddNewMonitor() throws Exception {
     StatMonitorHandler handler = new StatMonitorHandler();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatisticsMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatisticsMonitorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatisticsMonitorJUnitTest.java
index f8cf043..4f4e6bd 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatisticsMonitorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/StatisticsMonitorJUnitTest.java
@@ -17,68 +17,43 @@
 package com.gemstone.gemfire.internal.statistics;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.io.File;
 import java.util.List;
 
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.NanoTimer;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.internal.logging.PureLogWriter;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
- * Unit and integration tests for the StatisticsMonitor.
+ * Unit tests for the StatisticsMonitor class.
  *   
  * @since 7.0
  */
 @Category(UnitTest.class)
 public class StatisticsMonitorJUnitTest {
   
-  private Mockery mockContext;
-  private InternalLogWriter log;
-  private TestStatisticsManager manager; 
+  private TestStatisticsManager manager;
   private SampleCollector sampleCollector;
 
   @Before
   public void setUp() throws Exception {
-    this.mockContext = new Mockery() {{
-      setImposteriser(ClassImposteriser.INSTANCE);
-    }};
-    
-    this.log = new PureLogWriter(LogWriterImpl.levelNameToCode("config"));
-    
     final long startTime = System.currentTimeMillis();
-    this.manager = new TestStatisticsManager(
-        1, 
-        "StatisticsMonitorJUnitTest", 
-        startTime);
+    this.manager = new TestStatisticsManager(1, getClass().getSimpleName(), startTime);
     
-    final StatArchiveHandlerConfig mockStatArchiveHandlerConfig = this.mockContext.mock(StatArchiveHandlerConfig.class, "StatisticsMonitorJUnitTest$StatArchiveHandlerConfig");
-    this.mockContext.checking(new Expectations() {{
-      allowing(mockStatArchiveHandlerConfig).getArchiveFileName();
-      will(returnValue(new File("")));
-      allowing(mockStatArchiveHandlerConfig).getArchiveFileSizeLimit();
-      will(returnValue(0));
-      allowing(mockStatArchiveHandlerConfig).getArchiveDiskSpaceLimit();
-      will(returnValue(0));
-      allowing(mockStatArchiveHandlerConfig).getSystemId();
-      will(returnValue(1));
-      allowing(mockStatArchiveHandlerConfig).getSystemStartTime();
-      will(returnValue(startTime));
-      allowing(mockStatArchiveHandlerConfig).getSystemDirectoryPath();
-      will(returnValue(""));
-      allowing(mockStatArchiveHandlerConfig).getProductDescription();
-      will(returnValue("StatisticsMonitorJUnitTest"));
-    }});
+    final StatArchiveHandlerConfig mockStatArchiveHandlerConfig = mock(StatArchiveHandlerConfig.class, getClass().getSimpleName() + "$" + StatArchiveHandlerConfig.class.getSimpleName());
+    when(mockStatArchiveHandlerConfig.getArchiveFileName()).thenReturn(new File(""));
+    when(mockStatArchiveHandlerConfig.getArchiveFileSizeLimit()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getArchiveDiskSpaceLimit()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getSystemId()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getSystemStartTime()).thenReturn(0L);
+    when(mockStatArchiveHandlerConfig.getSystemDirectoryPath()).thenReturn("");
+    when(mockStatArchiveHandlerConfig.getProductDescription()).thenReturn(getClass().getSimpleName());
 
     StatisticsSampler sampler = new TestStatisticsSampler(manager);
     this.sampleCollector = new SampleCollector(sampler);
@@ -91,8 +66,6 @@ public class StatisticsMonitorJUnitTest {
       this.sampleCollector.close();
       this.sampleCollector = null;
     }
-    this.mockContext.assertIsSatisfied();
-    this.mockContext = null;
     this.manager = null;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/ArrayUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/ArrayUtilsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/ArrayUtilsJUnitTest.java
index 0a78a19..dde21ca 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/ArrayUtilsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/ArrayUtilsJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.util;
 
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BytesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BytesJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BytesJUnitTest.java
index e5a7f4d..997548c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BytesJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BytesJUnitTest.java
@@ -16,18 +16,21 @@
  */
 package com.gemstone.gemfire.internal.util;
 
+import static org.junit.Assert.*;
+
 import java.nio.ByteBuffer;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class BytesJUnitTest extends TestCase {
+public class BytesJUnitTest {
+
   private ByteBuffer buf = ByteBuffer.allocate(8);
-  
+
+  @Test
   public void testShort() {
     short[] val = { 666, -1, Short.MIN_VALUE, 0, 12, Short.MAX_VALUE };
     for (int i = 0; i < val.length; i++) {
@@ -37,7 +40,8 @@ public class BytesJUnitTest extends TestCase {
       buf.rewind();
     }
   }
-  
+
+  @Test
   public void testChar() {
     char[] val = { 'a', 'b', 'c' };
     for (int i = 0; i < val.length; i++) {
@@ -47,7 +51,8 @@ public class BytesJUnitTest extends TestCase {
       buf.rewind();
     }
   }
-  
+
+  @Test
   public void testUnsignedShort() {
     int[] val = { 0, 1, Short.MAX_VALUE + 1, 2 * Short.MAX_VALUE };
     for (int i = 0; i < val.length; i++) {
@@ -57,7 +62,8 @@ public class BytesJUnitTest extends TestCase {
       buf.rewind();
     }
   }
-  
+
+  @Test
   public void testInt() {
     int[] val = { 666, -1, Integer.MIN_VALUE, 0, 1, Integer.MAX_VALUE };
     for (int i = 0; i < val.length; i++) {
@@ -71,7 +77,8 @@ public class BytesJUnitTest extends TestCase {
       assertEquals(val[i], Bytes.toInt(bytes[0], bytes[1], bytes[2], bytes[3]));
     }
   }
-  
+
+  @Test
   public void testLong() {
     long[] val = { 666, -1, Long.MIN_VALUE, 0, 1, Long.MAX_VALUE };
     for (int i = 0; i < val.length; i++) {
@@ -82,8 +89,8 @@ public class BytesJUnitTest extends TestCase {
       buf.rewind();
     }
   }
-  
-  
+
+  @Test
   public void testVarint() {
     ByteBuffer buf = ByteBuffer.allocate(5);
     checkVarint(0, buf);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/CollectionUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/CollectionUtilsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/CollectionUtilsJUnitTest.java
index 0748055..242d2f2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/CollectionUtilsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/CollectionUtilsJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.util;
 
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/DelayedActionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/DelayedActionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/DelayedActionJUnitTest.java
index c6aa1a3..3a2d660 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/DelayedActionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/DelayedActionJUnitTest.java
@@ -16,19 +16,22 @@
  */
 package com.gemstone.gemfire.internal.util;
 
+import static org.junit.Assert.*;
+
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class DelayedActionJUnitTest extends TestCase {
+public class DelayedActionJUnitTest {
+
+  @Test
   public void testDelay() throws InterruptedException {
     final AtomicBoolean hit = new AtomicBoolean(false);
     final CountDownLatch complete = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/CompactConcurrentHashSetJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/CompactConcurrentHashSetJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/CompactConcurrentHashSetJUnitTest.java
index 216a85c..7c63dd4 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/CompactConcurrentHashSetJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/CompactConcurrentHashSetJUnitTest.java
@@ -16,8 +16,7 @@
  */
 package com.gemstone.gemfire.internal.util.concurrent;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.util.HashSet;
 import java.util.Iterator;
@@ -32,9 +31,10 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class CompactConcurrentHashSetJUnitTest {
-  static int RANGE = 100000;
-  static int SET_SIZE = 1000;
-  Random ran = new Random();
+
+  private static final int RANGE = 100000;
+
+  private Random random = new Random();
   
   @Test
   public void testEquals() {
@@ -42,7 +42,7 @@ public class CompactConcurrentHashSetJUnitTest {
     s1 = new CompactConcurrentHashSet2();
     s2 = new HashSet();
     for (int i=0; i<10000; i++) {
-      int nexti = ran.nextInt(RANGE);
+      int nexti = random.nextInt(RANGE);
       s1.add(nexti);
       s2.add(nexti);
       assertTrue("expected s1 and s2 to be equal", s1.equals(s2));
@@ -60,7 +60,7 @@ public class CompactConcurrentHashSetJUnitTest {
     Set<Integer> s1;
     s1 = new CompactConcurrentHashSet2<Integer>();
     for (int i=0; i<10000; i++) {
-      int nexti = ran.nextInt(RANGE);
+      int nexti = random.nextInt(RANGE);
       s1.add(nexti);
     }
     for (Iterator<Integer> it=s1.iterator(); it.hasNext(); ) {
@@ -80,7 +80,7 @@ public class CompactConcurrentHashSetJUnitTest {
     s1 = new CompactConcurrentHashSet2<Integer>();
     s2 = new HashSet<Integer>();
     for (int i=0; i<10000; i++) {
-      int nexti = ran.nextInt(RANGE);
+      int nexti = random.nextInt(RANGE);
       s1.add(nexti);
       s2.add(nexti);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/ReentrantSemaphoreJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/ReentrantSemaphoreJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/ReentrantSemaphoreJUnitTest.java
index d24d067..f655754 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/ReentrantSemaphoreJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/ReentrantSemaphoreJUnitTest.java
@@ -16,22 +16,21 @@
  */
 package com.gemstone.gemfire.internal.util.concurrent;
 
+import static org.junit.Assert.*;
+
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class ReentrantSemaphoreJUnitTest extends TestCase {
-  
+public class ReentrantSemaphoreJUnitTest {
+
+  @Test
   public void test() throws Throwable {
     final ReentrantSemaphore sem = new ReentrantSemaphore(2);
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/SemaphoreReadWriteLockJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/SemaphoreReadWriteLockJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/SemaphoreReadWriteLockJUnitTest.java
index bc598e6..6bd7999 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/SemaphoreReadWriteLockJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/concurrent/SemaphoreReadWriteLockJUnitTest.java
@@ -16,24 +16,23 @@
  */
 package com.gemstone.gemfire.internal.util.concurrent;
 
+import static org.junit.Assert.*;
+
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Lock;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- * 
- */
 @Category(UnitTest.class)
-public class SemaphoreReadWriteLockJUnitTest extends TestCase {
+public class SemaphoreReadWriteLockJUnitTest {
 
-  public void testReaderWaitsForWriter() throws InterruptedException {
+  @Test
+  public void testReaderWaitsForWriter() throws Exception {
     SemaphoreReadWriteLock rwl = new SemaphoreReadWriteLock();
     final Lock rl = rwl.readLock();
     final Lock wl = rwl.writeLock();
@@ -56,7 +55,8 @@ public class SemaphoreReadWriteLockJUnitTest extends TestCase {
     assertTrue(latch.await(10, TimeUnit.SECONDS));
   }
 
-  public void testWriterWaitsForReader() throws InterruptedException {
+  @Test
+  public void testWriterWaitsForReader() throws Exception {
     SemaphoreReadWriteLock rwl = new SemaphoreReadWriteLock();
     final Lock rl = rwl.readLock();
     final Lock wl = rwl.writeLock();
@@ -79,7 +79,8 @@ public class SemaphoreReadWriteLockJUnitTest extends TestCase {
     assertTrue(latch.await(10, TimeUnit.SECONDS));
   }
 
-  public void testReadersNotBlockedByReaders() throws InterruptedException {
+  @Test
+  public void testReadersNotBlockedByReaders() throws Exception {
     SemaphoreReadWriteLock rwl = new SemaphoreReadWriteLock();
     final Lock rl = rwl.readLock();
     final Lock wl = rwl.writeLock();
@@ -99,7 +100,8 @@ public class SemaphoreReadWriteLockJUnitTest extends TestCase {
     assertTrue(latch.await(10, TimeUnit.SECONDS));
   }
 
-  public void testWritersBlockedByWriters() throws InterruptedException {
+  @Test
+  public void testWritersBlockedByWriters() throws Exception {
     SemaphoreReadWriteLock rwl = new SemaphoreReadWriteLock();
     final Lock rl = rwl.readLock();
     final Lock wl = rwl.writeLock();
@@ -122,7 +124,8 @@ public class SemaphoreReadWriteLockJUnitTest extends TestCase {
     assertTrue(latch.await(10, TimeUnit.SECONDS));
   }
 
-  public void testTrylock() throws InterruptedException {
+  @Test
+  public void testTrylock() throws Exception {
     SemaphoreReadWriteLock rwl = new SemaphoreReadWriteLock();
     final Lock rl = rwl.readLock();
     final Lock wl = rwl.writeLock();
@@ -146,7 +149,8 @@ public class SemaphoreReadWriteLockJUnitTest extends TestCase {
     assertFalse(failed.get());
   }
 
-  public void testLockAndReleasebyDifferentThreads() throws InterruptedException {
+  @Test
+  public void testLockAndReleasebyDifferentThreads() throws Exception {
     SemaphoreReadWriteLock rwl = new SemaphoreReadWriteLock();
     final Lock rl = rwl.readLock();
     final Lock wl = rwl.writeLock();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
index be73adc..c96c1c8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
@@ -448,7 +448,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends ClientServerTes
     assertTrue(firedBridge[JOINED]);
     assertEquals(clientMember, memberBridge[JOINED]);
     //as of 6.1 the id can change when a bridge is created or a connection pool is created
-    //assertEquals(clientMemberId, memberIdBridge[JOINED]);
+    //assertIndexDetailsEquals(clientMemberId, memberIdBridge[JOINED]);
     assertTrue(isClientBridge[JOINED]);
     assertFalse(firedBridge[LEFT]);
     assertNull(memberBridge[LEFT]);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/GatewayReceiverStatsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/GatewayReceiverStatsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/GatewayReceiverStatsJUnitTest.java
index f007800..e188df4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/GatewayReceiverStatsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/GatewayReceiverStatsJUnitTest.java
@@ -88,8 +88,8 @@ public class GatewayReceiverStatsJUnitTest extends MBeanStatsTestCase {
     
     bridge.stopMonitor();
     
-    // TODO:FAIL: assertEquals(0, getCurrentClients());
-    // TODO:FAIL: assertEquals(0, getConnectionThreads());
+    // TODO:FAIL: assertIndexDetailsEquals(0, getCurrentClients());
+    // TODO:FAIL: assertIndexDetailsEquals(0, getConnectionThreads());
   }
   
   @Test
@@ -112,8 +112,8 @@ public class GatewayReceiverStatsJUnitTest extends MBeanStatsTestCase {
     assertEquals(1, getOutoforderBatchesReceived());
     bridge.stopMonitor();
     
-    // TODO:FAIL: assertEquals(0, getOutoforderBatchesReceived());
-    // TODO:FAIL: assertEquals(0, getDuplicateBatchesReceived());
+    // TODO:FAIL: assertIndexDetailsEquals(0, getOutoforderBatchesReceived());
+    // TODO:FAIL: assertIndexDetailsEquals(0, getDuplicateBatchesReceived());
   }
 
   private float getCreateRequestsRate() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/RegionStatsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/RegionStatsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/RegionStatsJUnitTest.java
index 6650fb2..f9b5489 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/RegionStatsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/RegionStatsJUnitTest.java
@@ -78,7 +78,7 @@ public class RegionStatsJUnitTest extends MBeanStatsTestCase{
     assertEquals(1, getNumBucketsWithoutRedundancy());
     assertEquals(2, getActualRedundancy());
     //assertTrue(getAvgBucketSize() > 0);
-    //assertEquals(3, getConfiguredRedundancy());
+    //assertIndexDetailsEquals(3, getConfiguredRedundancy());
     assertEquals(1, getDataStoreEntryCount());
     assertEquals(10, getPrimaryBucketCount());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/StatsRateJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/StatsRateJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/StatsRateJUnitTest.java
index bdccba5..9869a8e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/StatsRateJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/bean/stats/StatsRateJUnitTest.java
@@ -14,9 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.management.bean.stats;
 
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.Statistics;
@@ -25,14 +28,8 @@ import com.gemstone.gemfire.management.internal.beans.stats.StatType;
 import com.gemstone.gemfire.management.internal.beans.stats.StatsRate;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- * 
- * 
- */
 @Category(UnitTest.class)
-public class StatsRateJUnitTest extends TestCase  {
+public class StatsRateJUnitTest  {
 
   private Long SINGLE_STATS_LONG_COUNTER = null;
 
@@ -48,10 +45,7 @@ public class StatsRateJUnitTest extends TestCase  {
   
   private TestMBeanStatsMonitor statsMonitor = new TestMBeanStatsMonitor("TestStatsMonitor"); 
 
-  public StatsRateJUnitTest(String name) {
-    super(name);
-  }
-
+  @Before
   public void setUp() throws Exception {
     SINGLE_STATS_LONG_COUNTER = 0L;
     SINGLE_STATS_INT_COUNTER = 0;
@@ -61,10 +55,10 @@ public class StatsRateJUnitTest extends TestCase  {
     MULTI_STATS_INT_COUNTER_2 = 0;
   }
 
+  @Test
   public void testSingleStatLongRate() throws Exception {
     StatsRate singleStatsRate = new StatsRate("SINGLE_STATS_LONG_COUNTER", StatType.LONG_TYPE, statsMonitor);
 
-
     SINGLE_STATS_LONG_COUNTER = 5000L;
     float actualRate = singleStatsRate.getRate();
 
@@ -74,13 +68,13 @@ public class StatsRateJUnitTest extends TestCase  {
 
     float expectedRate = 5000;
 
-    assertEquals(expectedRate, actualRate);
+    assertEquals(expectedRate, actualRate, 0);
   }
 
+  @Test
   public void testSingleStatIntRate() throws Exception {
     StatsRate singleStatsRate = new StatsRate("SINGLE_STATS_INT_COUNTER", StatType.INT_TYPE, statsMonitor);
 
-    
     SINGLE_STATS_INT_COUNTER = 5000;
     float actualRate = singleStatsRate.getRate();
 
@@ -91,9 +85,10 @@ public class StatsRateJUnitTest extends TestCase  {
 
     float expectedRate = 5000;
 
-    assertEquals(expectedRate, actualRate);
+    assertEquals(expectedRate, actualRate, 0);
   }
 
+  @Test
   public void testMultiStatLongRate() throws Exception {
     String[] counters = new String[] { "MULTI_STATS_LONG_COUNTER_1", "MULTI_STATS_LONG_COUNTER_2" };
     StatsRate multiStatsRate = new StatsRate(counters, StatType.LONG_TYPE, statsMonitor);
@@ -109,15 +104,14 @@ public class StatsRateJUnitTest extends TestCase  {
 
     float expectedRate = 9000;
 
-    assertEquals(expectedRate, actualRate);
-
+    assertEquals(expectedRate, actualRate, 0);
   }
 
+  @Test
   public void testMultiStatIntRate() throws Exception {
     String[] counters = new String[] { "MULTI_STATS_INT_COUNTER_1", "MULTI_STATS_INT_COUNTER_2" };
     StatsRate multiStatsRate = new StatsRate(counters, StatType.INT_TYPE, statsMonitor);
 
-
     MULTI_STATS_INT_COUNTER_1 = 5000;
     MULTI_STATS_INT_COUNTER_2 = 4000;
     float actualRate = multiStatsRate.getRate();
@@ -125,27 +119,21 @@ public class StatsRateJUnitTest extends TestCase  {
     MULTI_STATS_INT_COUNTER_1 = 10000;
     MULTI_STATS_INT_COUNTER_2 = 8000;
 
-
     actualRate = multiStatsRate.getRate();
 
     float expectedRate = 9000;
 
-    assertEquals(expectedRate, actualRate);
-
+    assertEquals(expectedRate, actualRate, 0);
   }
   
-  private class TestMBeanStatsMonitor extends MBeanStatsMonitor{
-    
-    
+  private class TestMBeanStatsMonitor extends MBeanStatsMonitor {
+
     public TestMBeanStatsMonitor(String name) {
       super(name);
-      // TODO Auto-generated constructor stub
     }
 
     @Override
     public void addStatisticsToMonitor(Statistics stats) {
-      // TODO Auto-generated method stub
-
     }
 
     @Override
@@ -174,17 +162,11 @@ public class StatsRateJUnitTest extends TestCase  {
 
     @Override
     public void removeStatisticsFromMonitor(Statistics stats) {
-      // TODO Auto-generated method stub
-
     }
 
     @Override
     public void stopListener() {
-      // TODO Auto-generated method stub
-
     }
   }
 
-  
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/JettyHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/JettyHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/JettyHelperJUnitTest.java
index b7580ca..43c3d6e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/JettyHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/JettyHelperJUnitTest.java
@@ -20,10 +20,6 @@ import static org.junit.Assert.*;
 
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -33,7 +29,8 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * The JettyHelperJUnitTest class is a test suite of test cases testing the
  * contract and functionality of the JettyHelper
  * class.
- * <p/>
+ * <p>
+ *
  * @see com.gemstone.gemfire.management.internal.JettyHelper
  * @see org.jmock.Mockery
  * @see org.junit.Assert
@@ -42,20 +39,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class JettyHelperJUnitTest {
 
-  private Mockery mockContext;
-
-  @Before
-  public void setUp() {
-    mockContext = new Mockery();
-    mockContext.setImposteriser(ClassImposteriser.INSTANCE);
-  }
-
-  @After
-  public void tearDown() {
-    mockContext.assertIsSatisfied();
-    mockContext = null;
-  }
-
   @Test
   public void testSetPortNoBindAddress() throws Exception {
 
@@ -63,7 +46,7 @@ public class JettyHelperJUnitTest {
 
     assertNotNull(jetty);
     assertNotNull(jetty.getConnectors()[0]);
-    assertEquals(8090, ((ServerConnector)jetty.getConnectors()[0]).getPort());
+    assertEquals(8090, ((ServerConnector) jetty.getConnectors()[0]).getPort());
   }
 
   @Test
@@ -73,7 +56,7 @@ public class JettyHelperJUnitTest {
 
     assertNotNull(jetty);
     assertNotNull(jetty.getConnectors()[0]);
-    assertEquals(10480, ((ServerConnector)jetty.getConnectors()[0]).getPort());
+    assertEquals(10480, ((ServerConnector) jetty.getConnectors()[0]).getPort());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridgeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridgeJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridgeJUnitTest.java
index 84b39d4..81d6b18 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridgeJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridgeJUnitTest.java
@@ -68,7 +68,7 @@ public class DistributedSystemBridgeJUnitTest {
   }
   
   @Test
-  public void testSucessfulBackup() throws Exception {
+  public void testSuccessfulBackup() throws Exception {
     DM dm = cache.getDistributionManager();
     
     DistributedSystemBridge bridge = new DistributedSystemBridge(null);
@@ -97,7 +97,6 @@ public class DistributedSystemBridgeJUnitTest {
       bridge.backupAllMembers("/tmp", null);
       fail("Should have failed with an exception");
     } catch(RuntimeException expected) {
-      
     }
     
     verify(dm).putOutgoing(isA(FinishBackupRequest.class));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CliUtilDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CliUtilDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CliUtilDUnitTest.java
index 1cfc94c..cea95c1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CliUtilDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CliUtilDUnitTest.java
@@ -388,7 +388,7 @@ public class CliUtilDUnitTest extends CacheTestCase {
     /* "FIXME - Abhishek" This is failing because last param is not considered in method
     set = CliUtil.getRegionAssociatedMembers(region1, cache, false);
     assertNotNull(set);
-    assertEquals(1, set.size());*/
+    assertIndexDetailsEquals(1, set.size());*/
     
     set = CliUtil.getRegionAssociatedMembers(region_group1, cache, true);
     assertNotNull(set);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandManagerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandManagerJUnitTest.java
index aac9528..d73597e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandManagerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandManagerJUnitTest.java
@@ -50,10 +50,10 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * CommandManagerTest - Includes tests to check the CommandManager functions
- * 
  */
 @Category(UnitTest.class)
 public class CommandManagerJUnitTest {
+
   private static final String COMMAND1_NAME = "command1";
   private static final String COMMAND1_NAME_ALIAS = "command1_alias";
   private static final String COMMAND2_NAME = "c2";
@@ -223,9 +223,9 @@ public class CommandManagerJUnitTest {
     assertTrue("Should not find unlisted plugin.", !commandManager.getCommands().containsKey("mock plugin command unlisted"));
   }
 
-
   // class that represents dummy commands
-  static public class Commands implements CommandMarker {
+  public static class Commands implements CommandMarker {
+
     @CliCommand(value = { COMMAND1_NAME, COMMAND1_NAME_ALIAS }, help = COMMAND1_HELP)
     @CliMetaData(shellOnly = true, relatedTopic = { "relatedTopicOfCommand1" })
     public static String command1(
@@ -273,11 +273,12 @@ public class CommandManagerJUnitTest {
     @CliAvailabilityIndicator({ COMMAND1_NAME })
     public boolean isAvailable() {
       return true; // always available on server
-
     }
-
   }
 
+  /**
+   * Used by testCommandManagerLoadPluginCommands
+   */
   static class SimpleConverter implements Converter<String> {
 
     public boolean supports(Class<?> type, String optionContext) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandSeparatorEscapeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandSeparatorEscapeJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandSeparatorEscapeJUnitTest.java
index 8057a1e..735a209 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandSeparatorEscapeJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandSeparatorEscapeJUnitTest.java
@@ -16,21 +16,18 @@
  */
 package com.gemstone.gemfire.management.internal.cli;
 
+import static com.gemstone.gemfire.management.internal.cli.shell.MultiCommandHelper.*;
+import static org.junit.Assert.*;
+
 import java.util.List;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-import static com.gemstone.gemfire.management.internal.cli.shell.MultiCommandHelper.getMultipleCommands;
-
-/**
- * 
- *
- */
 @Category(UnitTest.class)
-public class CommandSeparatorEscapeJUnitTest  extends TestCase{
+public class CommandSeparatorEscapeJUnitTest {
 
   //testcases : single command
   //testcases : multiple commands with cmdSeparator
@@ -38,8 +35,9 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
   //testcases : multiplecommand with comma-value : first value
   //testcases : multiplecommand with comma-value : last value
   //testcases : multiplecommand with comma-value : middle value
-  
-  public void testEmptyCommand(){
+
+  @Test
+  public void testEmptyCommand() {
     String input = ";";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -48,8 +46,9 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     }*/
     assertEquals(0,split.size());    
   }
-  
-  public void testSingleCommand(){
+
+  @Test
+  public void testSingleCommand() {
     String input = "stop server";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -59,8 +58,9 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     assertEquals(1,split.size());
     assertEquals("stop server", split.get(0));
   }
-  
-  public void testMultiCommand(){
+
+  @Test
+  public void testMultiCommand() {
     String input = "stop server1 --option1=value1; stop server2;stop server3 ";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -72,8 +72,9 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     assertEquals(" stop server2", split.get(1));
     assertEquals("stop server3 ", split.get(2));
   }
-  
-  public void testMultiCommandWithCmdSep(){    
+
+  @Test
+  public void testMultiCommandWithCmdSep() {
     String input = "put --region=/region1 --key='key1\\;part' --value='value1\\;part2';put --region=/region1 --key='key2\\;part' --value='value2\\;part2'";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -85,7 +86,8 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     assertEquals("put --region=/region1 --key='key2;part' --value='value2;part2'", split.get(1));
   }
 
-  public void testSingleCommandWithComma(){
+  @Test
+  public void testSingleCommandWithComma() {
     String input = "put --region=/region1 --key='key\\;part' --value='value\\;part2'";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -96,7 +98,8 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     assertEquals("put --region=/region1 --key='key;part' --value='value;part2'", split.get(0));
   }
 
-  public void testMultiCmdCommaValueFirst(){
+  @Test
+  public void testMultiCmdCommaValueFirst() {
     String input = "put --region=/region1 --key='key\\;part' --value='value\\;part2';stop server";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -107,8 +110,9 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     assertEquals("put --region=/region1 --key='key;part' --value='value;part2'", split.get(0));
     assertEquals("stop server", split.get(1));
   }
-  
-  public void testMultiCmdCommaValueLast(){
+
+  @Test
+  public void testMultiCmdCommaValueLast() {
     String input = "stop server;put --region=/region1 --key='key\\;part' --value='value\\;part2'";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);
@@ -120,7 +124,8 @@ public class CommandSeparatorEscapeJUnitTest  extends TestCase{
     assertEquals("put --region=/region1 --key='key;part' --value='value;part2'", split.get(1));    
   }
 
-  public void testMultiCmdCommaValueMiddle(){
+  @Test
+  public void testMultiCmdCommaValueMiddle() {
     String input = "stop server1;put --region=/region1 --key='key\\;part' --value='value\\;part2';stop server2;stop server3";
     //System.out.println("I >> " + input);
     List<String> split = getMultipleCommands(input);



[12/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreJUnitTest.java
index 0a4a13e..88324d1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreJUnitTest.java
@@ -16,13 +16,9 @@
  */
 package com.gemstone.gemfire.cache.query.internal.index;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.util.HashSet;
 import java.util.Iterator;
@@ -45,352 +41,350 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class MemoryIndexStoreJUnitTest {
 
-	Region region;
-	GemFireCacheImpl cache;
-	InternalIndexStatistics mockStats;
-	MemoryIndexStore store;
-	RegionEntry[] mockEntries;
-	int numMockEntries = 10;
+  Region region;
+  GemFireCacheImpl cache;
+  InternalIndexStatistics mockStats;
+  MemoryIndexStore store;
+  RegionEntry[] mockEntries;
+  int numMockEntries = 10;
   GemFireCacheImpl actualInstance;
   
-	public void subclassPreSetup() {
-		
-	}
-	
-	protected Region createRegion() {
-		return mock(LocalRegion.class);
-	}
-	
-	@Before
-	public void setup() {
-		subclassPreSetup();
-		region = createRegion();
-		cache = mock(GemFireCacheImpl.class);
-		actualInstance = GemFireCacheImpl.setInstanceForTests(cache);
-		mockStats = mock(AbstractIndex.InternalIndexStatistics.class);
-		
-		store = new MemoryIndexStore(region, mockStats);
-		store.setIndexOnValues(true);
-		mockEntries = new RegionEntry[numMockEntries];
-		IntStream.range(0, numMockEntries).forEach(i-> {
-			mockEntries[i] = createRegionEntry(i, new Object());
-		});
-	}
-	
-	@After
-	public void teardown() {
-	  GemFireCacheImpl.setInstanceForTests(actualInstance);
-	}
-	
-	@Test
-	public void testSizeOfStoreReturnsNumberOfKeysAndNotActualNumberOfValues() {
-		IntStream.range(0, 150).forEach(i -> {
-			try {
-				store.addMapping(i % 3, createRegionEntry(i, new Object()));
-			}
-			catch (Exception e) {
-				fail();
-			}
-		});
-		assertEquals(150, numObjectsInStore(store));
-	}
-	
-	@Test
-	public void testAddEnoughEntriesToCreateAConcurrentHashSet() {
-		IntStream.range(0, 150).forEach(i -> {
-			try {
-				store.addMapping(1, createRegionEntry(i, new Object()));
-			}
-			catch (Exception e) {
-				fail();
-			}
-		});
-		assertEquals(150, numObjectsInStore(store));
-	}
-	
-	@Test
-	public void testUpdateAgainstAConcurrentHashSet() throws Exception{
-		IntStream.range(0, 150).forEach(i -> {
-			try {
-				store.addMapping(1, createRegionEntry(1, new Object()));
-			}
-			catch (Exception e) {
-				fail();
-			}
-		});
-		RegionEntry entry = createRegionEntry(1, new Object());	
-		store.addMapping(1, entry);
-		store.updateMapping(2, 1, entry, entry.getValue(null));
-		assertEquals(151, numObjectsInStore(store));
-	}
-	
-	@Test
-	public void testCanAddObjectWithUndefinedKey() throws Exception {
-		store.addMapping(QueryService.UNDEFINED, mockEntries[0]);
-		assertEquals(1, numObjectsIterated(store.get(QueryService.UNDEFINED)));
-		assertEquals(0, numObjectsInStore(store));
-	}
-	
-	@Test
-	public void testCanAddManyObjectsWithUndefinedKey() throws Exception {
-		for (int i = 0; i < mockEntries.length; i++) {
-			store.addMapping(QueryService.UNDEFINED, mockEntries[i]);
-		}
-		assertEquals(mockEntries.length, numObjectsIterated(store.get(QueryService.UNDEFINED)));
-		//Undefined will not return without an explicit get for UNDEFINED);
-		assertEquals(0, numObjectsInStore(store));
-	}
-	
-	@Test
-	public void testIteratorWithStartInclusiveAndNoKeysToRemoveReturnsCorrectNumberOfResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(2, numObjectsIterated(store.iterator(numMockEntries - 2, true, null)));
-	}
-	
-	@Test
-	public void testIteratorWithStartExclusiveAndNoKeysToRemoveReturnsCorrectNumberOfResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(1, numObjectsIterated(store.iterator(numMockEntries - 2, false, null)));
-	}
-	
-	
-	@Test
-	public void testIteratorWithStartInclusiveAndKeyToRemoveReturnsCorrectNumberOfResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		Set keysToRemove = new HashSet();
-		keysToRemove.add("1");
-		assertEquals(9, numObjectsIterated(store.iterator(1, true, keysToRemove)));
-	}
-	
-	@Test
-	public void testIteratorWithStartExclusiveAndKeyToRemoveReturnsCorrectNumberOfResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		Set keysToRemove = new HashSet();
-		keysToRemove.add("1");
-		assertEquals(8, numObjectsIterated(store.iterator(1, false, keysToRemove)));
-	}
-	
-	@Test
-	public void testStartAndEndInclusiveReturnsCorrectResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(6, numObjectsIterated(store.iterator(1, true, 6, true, null)));
-	}
-	
-	@Test
-	public void testStartInclusiveAndEndExclusiveReturnsCorrectResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(5, numObjectsIterated(store.iterator(1, true, 6, false, null)));
-	}
-	
-	@Test
-	public void testStartExclusiveAndEndExclusiveReturnsCorrectResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(4, numObjectsIterated(store.iterator(1, false, 6, false, null)));
-	}
-	
-	@Test
-	public void testStartExclusiveAndEndInclusiveReturnsCorrectResults() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(5, numObjectsIterated(store.iterator(1, false, 6, true, null)));
-	}
-	
-	@Test
-	public void testStartIsNull() throws Exception {
-		addMockedEntries(numMockEntries);
-		assertEquals(6, numObjectsIterated(store.iterator(null, false, 6, false, null)));
-	}
-	
-	@Test
-	public void testDescendingIteratorReturnsExpectedOrderOfEntries() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		Iterator iteratorFirst = store.descendingIterator(null);
-		assertEquals(2, numObjectsIterated(iteratorFirst));
-		
-		Iterator iterator = store.descendingIterator(null);
-		iterator.hasNext();
-		assertEquals(mockEntry2, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
-		iterator.hasNext();
-		assertEquals(mockEntry1, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
-	}
-	
-	@Test
-	public void testDescendingIteratorWithRemovedKeysReturnsExpectedOrderOfEntries() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		RegionEntry mockEntry3 = mockEntries[2];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		store.addMapping("3", mockEntry3);
-		Set keysToRemove = new HashSet();
-		keysToRemove.add("2");
-		Iterator iteratorFirst = store.descendingIterator(keysToRemove);
-		assertEquals(2, numObjectsIterated(iteratorFirst));
-		
-		//keysToRemove has been modified by the store, we need to readd the key to remove
-		keysToRemove.add("2");
-		Iterator iterator = store.descendingIterator(keysToRemove);
-		iterator.hasNext();
-		assertEquals(mockEntry3, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
-		iterator.hasNext();
-		assertEquals(mockEntry1, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
-		assertFalse(iterator.hasNext());
-	}
-	
-	@Test
-	public void testDescendingIteratorWithMultipleRemovedKeysReturnsExpectedOrderOfEntries() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		RegionEntry mockEntry3 = mockEntries[2];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		store.addMapping("3", mockEntry3);
-		Set keysToRemove = new HashSet();
-		keysToRemove.add("2");
-		keysToRemove.add("1");
-		Iterator iteratorFirst = store.descendingIterator(keysToRemove);
-		assertEquals(1, numObjectsIterated(iteratorFirst));
-		
-		//keysToRemove has been modified by the store, we need to readd the key to remove
-		keysToRemove.add("2");
-		keysToRemove.add("1");
-		Iterator iterator = store.descendingIterator(keysToRemove);
-		iterator.hasNext();
-		assertEquals(mockEntry3, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
-		assertFalse(iterator.hasNext());
-	}
-	
-	@Test
-	public void testSizeWithKeyArgumentReturnsCorrectSize() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		assertEquals(1, store.size("1"));
-	}
-	
-	 @Test
-	  public void testGetReturnsExpectedIteratorValue() throws Exception {
-	    RegionEntry mockEntry1 = mockEntries[0];
-	    RegionEntry mockEntry2 = mockEntries[1];
-	    store.addMapping("1", mockEntry1);
-	    store.addMapping("2", mockEntry2);
-	    assertEquals(1, numObjectsIterated(store.get("1")));
-	  }
-	
-	@Test
-	public void testGetReturnsExpectedIteratorWithMultipleValues() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		RegionEntry mockEntry3 = mockEntries[2];
-		RegionEntry mockEntry4 = mockEntries[3];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("1", mockEntry2);
-		store.addMapping("1", mockEntry3);
-		store.addMapping("2", mockEntry4);
-		assertEquals(3, numObjectsIterated(store.get("1")));
-		assertEquals(4, numObjectsInStore(store));
-	}
-	
-	@Test
-	public void testGetWithIndexOnKeysReturnsExpectedIteratorValues() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.setIndexOnValues(false);
-		store.setIndexOnRegionKeys(true);
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		assertEquals(1, numObjectsIterated(store.get("1")));
-	}
-
-	@Test
-	public void testCorrectlyRemovesEntryProvidedTheWrongKey() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		store.removeMapping("1", mockEntry2);
-		assertEquals(1, numObjectsInStore(store));
-		assertTrue(objectContainedIn(store, mockEntry1));
-	}
-
-	@Test
-	public void testRemoveMappingRemovesFromBackingMap() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		store.removeMapping("1", mockEntry1);
-		assertEquals(1, numObjectsInStore(store));
-		assertTrue(objectContainedIn(store, mockEntry2));
-	}
-
-	@Test
-	public void testAddMappingAddsToBackingMap() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("2", mockEntry2);
-		assertEquals(2, numObjectsInStore(store));
-		assertTrue(objectContainedIn(store, mockEntry1));
-		assertTrue(objectContainedIn(store, mockEntry2));
-	}
-	
-	@Test
-	public void testClear() throws Exception {
-		RegionEntry mockEntry1 = mockEntries[0];
-		RegionEntry mockEntry2 = mockEntries[1];
-		store.addMapping("1", mockEntry1);
-		store.addMapping("1", mockEntry2);
-		store.clear();
-		assertEquals(0, numObjectsInStore(store));
-	}
-
-	private int numObjectsInStore(MemoryIndexStore store) {
-		Iterator iterator = store.iterator(null);
-		return numObjectsIterated(iterator);
-	}
-
-	private int numObjectsIterated(Iterator iterator) {
-		int count = 0;
-		while (iterator.hasNext()) {
-			iterator.next();
-			count++;
-		}
-		return count;
-	}
-
-	private boolean objectContainedIn(MemoryIndexStore store, Object o) {
-		Iterator iterator = store.valueToEntriesMap.values().iterator();
-		return objectContainedIn(iterator, o);
-	}
-	
-	private boolean objectContainedIn(Iterator iterator, Object o) {
-		while (iterator.hasNext()) {
-			if (iterator.next().equals(o)) {
-				return true;
-			}
-		}
-		return false;
-	}
-	
-	private void addMockedEntries(int numEntriesToAdd) {
-		IntStream.range(0, numEntriesToAdd).forEach(i -> {
-			try {
-				store.addMapping(mockEntries[i].getKey(), mockEntries[i]);
-			}
-			catch (Exception e) {
-				fail();
-			}
-		});
-	}
-	
-	private RegionEntry createRegionEntry(Object key, Object value) {
-		RegionEntry mockEntry = mock(RegionEntry.class);
-		when(mockEntry.getValue(any())).thenReturn(value);
-		when(mockEntry.getKey()).thenReturn(key);
-		return mockEntry;
-	}
+  protected void subclassPreSetup() {
+  }
+
+  protected Region createRegion() {
+    return mock(LocalRegion.class);
+  }
+
+  @Before
+  public void setup() {
+    subclassPreSetup();
+    region = createRegion();
+    cache = mock(GemFireCacheImpl.class);
+    actualInstance = GemFireCacheImpl.setInstanceForTests(cache);
+    mockStats = mock(AbstractIndex.InternalIndexStatistics.class);
+
+    store = new MemoryIndexStore(region, mockStats);
+    store.setIndexOnValues(true);
+    mockEntries = new RegionEntry[numMockEntries];
+    IntStream.range(0, numMockEntries).forEach(i-> {
+      mockEntries[i] = createRegionEntry(i, new Object());
+    });
+  }
+
+  @After
+  public void teardown() {
+    GemFireCacheImpl.setInstanceForTests(actualInstance);
+  }
+
+  @Test
+  public void testSizeOfStoreReturnsNumberOfKeysAndNotActualNumberOfValues() {
+    IntStream.range(0, 150).forEach(i -> {
+      try {
+        store.addMapping(i % 3, createRegionEntry(i, new Object()));
+      }
+      catch (Exception e) {
+        throw new AssertionError(e);
+      }
+    });
+    assertEquals(150, numObjectsInStore(store));
+  }
+
+  @Test
+  public void testAddEnoughEntriesToCreateAConcurrentHashSet() {
+    IntStream.range(0, 150).forEach(i -> {
+      try {
+        store.addMapping(1, createRegionEntry(i, new Object()));
+      }
+      catch (Exception e) {
+        throw new AssertionError(e);
+      }
+    });
+    assertEquals(150, numObjectsInStore(store));
+  }
+
+  @Test
+  public void testUpdateAgainstAConcurrentHashSet() throws Exception{
+    IntStream.range(0, 150).forEach(i -> {
+      try {
+        store.addMapping(1, createRegionEntry(1, new Object()));
+      }
+      catch (Exception e) {
+        throw new AssertionError(e);
+      }
+    });
+    RegionEntry entry = createRegionEntry(1, new Object());
+    store.addMapping(1, entry);
+    store.updateMapping(2, 1, entry, entry.getValue(null));
+    assertEquals(151, numObjectsInStore(store));
+  }
+
+  @Test
+  public void testCanAddObjectWithUndefinedKey() throws Exception {
+    store.addMapping(QueryService.UNDEFINED, mockEntries[0]);
+    assertEquals(1, numObjectsIterated(store.get(QueryService.UNDEFINED)));
+    assertEquals(0, numObjectsInStore(store));
+  }
+
+  @Test
+  public void testCanAddManyObjectsWithUndefinedKey() throws Exception {
+    for (int i = 0; i < mockEntries.length; i++) {
+      store.addMapping(QueryService.UNDEFINED, mockEntries[i]);
+    }
+    assertEquals(mockEntries.length, numObjectsIterated(store.get(QueryService.UNDEFINED)));
+    //Undefined will not return without an explicit get for UNDEFINED);
+    assertEquals(0, numObjectsInStore(store));
+  }
+
+  @Test
+  public void testIteratorWithStartInclusiveAndNoKeysToRemoveReturnsCorrectNumberOfResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(2, numObjectsIterated(store.iterator(numMockEntries - 2, true, null)));
+  }
+
+  @Test
+  public void testIteratorWithStartExclusiveAndNoKeysToRemoveReturnsCorrectNumberOfResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(1, numObjectsIterated(store.iterator(numMockEntries - 2, false, null)));
+  }
+
+  @Test
+  public void testIteratorWithStartInclusiveAndKeyToRemoveReturnsCorrectNumberOfResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    Set keysToRemove = new HashSet();
+    keysToRemove.add("1");
+    assertEquals(9, numObjectsIterated(store.iterator(1, true, keysToRemove)));
+  }
+
+  @Test
+  public void testIteratorWithStartExclusiveAndKeyToRemoveReturnsCorrectNumberOfResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    Set keysToRemove = new HashSet();
+    keysToRemove.add("1");
+    assertEquals(8, numObjectsIterated(store.iterator(1, false, keysToRemove)));
+  }
+
+  @Test
+  public void testStartAndEndInclusiveReturnsCorrectResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(6, numObjectsIterated(store.iterator(1, true, 6, true, null)));
+  }
+
+  @Test
+  public void testStartInclusiveAndEndExclusiveReturnsCorrectResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(5, numObjectsIterated(store.iterator(1, true, 6, false, null)));
+  }
+
+  @Test
+  public void testStartExclusiveAndEndExclusiveReturnsCorrectResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(4, numObjectsIterated(store.iterator(1, false, 6, false, null)));
+  }
+
+  @Test
+  public void testStartExclusiveAndEndInclusiveReturnsCorrectResults() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(5, numObjectsIterated(store.iterator(1, false, 6, true, null)));
+  }
+
+  @Test
+  public void testStartIsNull() throws Exception {
+    addMockedEntries(numMockEntries);
+    assertEquals(6, numObjectsIterated(store.iterator(null, false, 6, false, null)));
+  }
+
+  @Test
+  public void testDescendingIteratorReturnsExpectedOrderOfEntries() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    Iterator iteratorFirst = store.descendingIterator(null);
+    assertEquals(2, numObjectsIterated(iteratorFirst));
+
+    Iterator iterator = store.descendingIterator(null);
+    iterator.hasNext();
+    assertEquals(mockEntry2, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
+    iterator.hasNext();
+    assertEquals(mockEntry1, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
+  }
+
+  @Test
+  public void testDescendingIteratorWithRemovedKeysReturnsExpectedOrderOfEntries() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    RegionEntry mockEntry3 = mockEntries[2];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    store.addMapping("3", mockEntry3);
+    Set keysToRemove = new HashSet();
+    keysToRemove.add("2");
+    Iterator iteratorFirst = store.descendingIterator(keysToRemove);
+    assertEquals(2, numObjectsIterated(iteratorFirst));
+
+    //keysToRemove has been modified by the store, we need to readd the key to remove
+    keysToRemove.add("2");
+    Iterator iterator = store.descendingIterator(keysToRemove);
+    iterator.hasNext();
+    assertEquals(mockEntry3, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
+    iterator.hasNext();
+    assertEquals(mockEntry1, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
+    assertFalse(iterator.hasNext());
+  }
+
+  @Test
+  public void testDescendingIteratorWithMultipleRemovedKeysReturnsExpectedOrderOfEntries() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    RegionEntry mockEntry3 = mockEntries[2];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    store.addMapping("3", mockEntry3);
+    Set keysToRemove = new HashSet();
+    keysToRemove.add("2");
+    keysToRemove.add("1");
+    Iterator iteratorFirst = store.descendingIterator(keysToRemove);
+    assertEquals(1, numObjectsIterated(iteratorFirst));
+
+    //keysToRemove has been modified by the store, we need to readd the key to remove
+    keysToRemove.add("2");
+    keysToRemove.add("1");
+    Iterator iterator = store.descendingIterator(keysToRemove);
+    iterator.hasNext();
+    assertEquals(mockEntry3, ((MemoryIndexStore.MemoryIndexStoreEntry)iterator.next()).getRegionEntry());
+    assertFalse(iterator.hasNext());
+  }
+
+  @Test
+  public void testSizeWithKeyArgumentReturnsCorrectSize() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    assertEquals(1, store.size("1"));
+  }
+
+  @Test
+  public void testGetReturnsExpectedIteratorValue() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    assertEquals(1, numObjectsIterated(store.get("1")));
+  }
+
+  @Test
+  public void testGetReturnsExpectedIteratorWithMultipleValues() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    RegionEntry mockEntry3 = mockEntries[2];
+    RegionEntry mockEntry4 = mockEntries[3];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("1", mockEntry2);
+    store.addMapping("1", mockEntry3);
+    store.addMapping("2", mockEntry4);
+    assertEquals(3, numObjectsIterated(store.get("1")));
+    assertEquals(4, numObjectsInStore(store));
+  }
+
+  @Test
+  public void testGetWithIndexOnKeysReturnsExpectedIteratorValues() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.setIndexOnValues(false);
+    store.setIndexOnRegionKeys(true);
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    assertEquals(1, numObjectsIterated(store.get("1")));
+  }
+
+  @Test
+  public void testCorrectlyRemovesEntryProvidedTheWrongKey() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    store.removeMapping("1", mockEntry2);
+    assertEquals(1, numObjectsInStore(store));
+    assertTrue(objectContainedIn(store, mockEntry1));
+  }
+
+  @Test
+  public void testRemoveMappingRemovesFromBackingMap() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    store.removeMapping("1", mockEntry1);
+    assertEquals(1, numObjectsInStore(store));
+    assertTrue(objectContainedIn(store, mockEntry2));
+  }
+
+  @Test
+  public void testAddMappingAddsToBackingMap() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("2", mockEntry2);
+    assertEquals(2, numObjectsInStore(store));
+    assertTrue(objectContainedIn(store, mockEntry1));
+    assertTrue(objectContainedIn(store, mockEntry2));
+  }
+
+  @Test
+  public void testClear() throws Exception {
+    RegionEntry mockEntry1 = mockEntries[0];
+    RegionEntry mockEntry2 = mockEntries[1];
+    store.addMapping("1", mockEntry1);
+    store.addMapping("1", mockEntry2);
+    store.clear();
+    assertEquals(0, numObjectsInStore(store));
+  }
+
+  private int numObjectsInStore(MemoryIndexStore store) {
+    Iterator iterator = store.iterator(null);
+    return numObjectsIterated(iterator);
+  }
+
+  private int numObjectsIterated(Iterator iterator) {
+    int count = 0;
+    while (iterator.hasNext()) {
+      iterator.next();
+      count++;
+    }
+    return count;
+  }
+
+  private boolean objectContainedIn(MemoryIndexStore store, Object o) {
+    Iterator iterator = store.valueToEntriesMap.values().iterator();
+    return objectContainedIn(iterator, o);
+  }
+
+  private boolean objectContainedIn(Iterator iterator, Object o) {
+    while (iterator.hasNext()) {
+      if (iterator.next().equals(o)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private void addMockedEntries(int numEntriesToAdd) {
+    IntStream.range(0, numEntriesToAdd).forEach(i -> {
+      try {
+        store.addMapping(mockEntries[i].getKey(), mockEntries[i]);
+      }
+      catch (Exception e) {
+        throw new AssertionError(e);
+      }
+    });
+  }
+
+  private RegionEntry createRegionEntry(Object key, Object value) {
+    RegionEntry mockEntry = mock(RegionEntry.class);
+    when(mockEntry.getValue(any())).thenReturn(value);
+    when(mockEntry.getKey()).thenReturn(key);
+    return mockEntry;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreWithInplaceModificationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreWithInplaceModificationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreWithInplaceModificationJUnitTest.java
index 79f8616..82505de 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreWithInplaceModificationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MemoryIndexStoreWithInplaceModificationJUnitTest.java
@@ -16,9 +16,7 @@
  */
 package com.gemstone.gemfire.cache.query.internal.index;
 
-import static org.mockito.Mockito.mock;
-
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import org.junit.After;
 import org.junit.experimental.categories.Category;
@@ -30,25 +28,24 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class MemoryIndexStoreWithInplaceModificationJUnitTest extends MemoryIndexStoreJUnitTest {
-	
-	public void subclassPreSetup() {
-		IndexManager.INPLACE_OBJECT_MODIFICATION_FOR_TEST = true;
-	}
-	
-	protected Region createRegion() {
-		Region region = mock(LocalRegion.class);
-		RegionAttributes ra = mock(RegionAttributes.class);
-		when(region.getAttributes()).thenReturn(ra);
-		when(ra.getInitialCapacity()).thenReturn(16);
-		when(ra.getLoadFactor()).thenReturn(.75f);
-		when(ra.getConcurrencyLevel()).thenReturn(16);
-		return region;
-	}
-	
-	@After
-	public void resetInplaceModification() {
-		IndexManager.INPLACE_OBJECT_MODIFICATION_FOR_TEST = false;
-	}
-	
-	
+
+  public void subclassPreSetup() {
+    IndexManager.INPLACE_OBJECT_MODIFICATION_FOR_TEST = true;
+  }
+
+  protected Region createRegion() {
+    Region region = mock(LocalRegion.class);
+    RegionAttributes ra = mock(RegionAttributes.class);
+    when(region.getAttributes()).thenReturn(ra);
+    when(ra.getInitialCapacity()).thenReturn(16);
+    when(ra.getLoadFactor()).thenReturn(.75f);
+    when(ra.getConcurrencyLevel()).thenReturn(16);
+    return region;
+  }
+
+  @After
+  public void resetInPlaceModification() {
+    IndexManager.INPLACE_OBJECT_MODIFICATION_FOR_TEST = false;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicIndexCreationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicIndexCreationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicIndexCreationDUnitTest.java
index 3ce1952..1532d2a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicIndexCreationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicIndexCreationDUnitTest.java
@@ -432,7 +432,7 @@ public class PRBasicIndexCreationDUnitTest extends
                   + " and remote buckets indexed : "
                   + ((PartitionedIndex)parIndex).getNumRemoteBucketsIndexed());
           /*
-           * assertEquals("Max num of buckets in the partiotion regions and
+           * assertIndexDetailsEquals("Max num of buckets in the partiotion regions and
            * the " + "buckets indexed should be equal",
            * ((PartitionedRegion)region).getTotalNumberOfBuckets(),
            * (((PartionedIndex)parIndex).getNumberOfIndexedBucket()+((PartionedIndex)parIndex).getNumRemtoeBucketsIndexed()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicMultiIndexCreationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicMultiIndexCreationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicMultiIndexCreationDUnitTest.java
index d020ef6..8ce3949 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicMultiIndexCreationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRBasicMultiIndexCreationDUnitTest.java
@@ -393,7 +393,7 @@ public class PRBasicMultiIndexCreationDUnitTest extends
                   + " and remote buckets indexed : "
                   + ((PartitionedIndex)parIndex).getNumRemoteBucketsIndexed());
           /*
-           * assertEquals("Max num of buckets in the partiotion regions and
+           * assertIndexDetailsEquals("Max num of buckets in the partiotion regions and
            * the " + "buckets indexed should be equal",
            * ((PartitionedRegion)region).getTotalNumberOfBuckets(),
            * (((PartionedIndex)parIndex).getNumberOfIndexedBucket()+((PartionedIndex)parIndex).getNumRemtoeBucketsIndexed()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitHelper.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitHelper.java
index 680aa0b..45d91c9 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitHelper.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitHelper.java
@@ -1132,7 +1132,7 @@ public class PRQueryDUnitHelper extends PartitionedRegionDUnitTestCase
                 if(srr.size() > l*l) {
                   fail("The resultset size exceeds limit size. Limit size="+ l*l+", result size ="+ srr.asList().size());
                 }
-                //assertEquals("The resultset size is not same as limit size.", l*l, srr.asList().size());
+                //assertIndexDetailsEquals("The resultset size is not same as limit size.", l*l, srr.asList().size());
 
 //                getCache().getLogger().info("Finished executing PR query: " + qStr);
               }
@@ -1361,7 +1361,7 @@ public class PRQueryDUnitHelper extends PartitionedRegionDUnitTestCase
                 fail ("Index usage is not as expected, expected it to be either " + 
                     expectedUse + " or " + (expectedUse + indexUsageWithSizeEstimation) + 
                     " it is: " + indexUse);
-                //assertEquals(6 + indexUsageWithSizeEstimation, bukInd.getStatistics().getTotalUses());
+                //assertIndexDetailsEquals(6 + indexUsageWithSizeEstimation, bukInd.getStatistics().getTotalUses());
               }
               k++;
             }*/
@@ -2087,7 +2087,7 @@ public class PRQueryDUnitHelper extends PartitionedRegionDUnitTestCase
                   + ((PartitionedIndex)parIndex).getNumRemoteBucketsIndexed());
           }
           /*
-           * assertEquals("Max num of buckets in the partiotion regions and
+           * assertIndexDetailsEquals("Max num of buckets in the partiotion regions and
            * the " + "buckets indexed should be equal",
            * ((PartitionedRegion)region).getTotalNumberOfBuckets(),
            * (((PartionedIndex)parIndex).getNumberOfIndexedBucket()+((PartionedIndex)parIndex).getNumRemtoeBucketsIndexed()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitTest.java
index c305531..7a9a7ba 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryDUnitTest.java
@@ -711,7 +711,7 @@ public class PRQueryDUnitTest extends PartitionedRegionDUnitTestCase
           assertTrue(false);
         } catch (QueryException expected) {
         }
-//        assertEquals(1, results.size());
+//        assertIndexDetailsEquals(1, results.size());
 //        getLogWriter().info("Select results are: " + results);
         return Boolean.TRUE;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/server/internal/ConnectionCountProbeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/server/internal/ConnectionCountProbeJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/server/internal/ConnectionCountProbeJUnitTest.java
index 5522ae5..91e82f3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/server/internal/ConnectionCountProbeJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/server/internal/ConnectionCountProbeJUnitTest.java
@@ -16,35 +16,33 @@
  */
 package com.gemstone.gemfire.cache.server.internal;
 
-import org.junit.experimental.categories.Category;
+import static org.junit.Assert.*;
 
-import junit.framework.Assert;
-import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.server.ServerLoad;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
-public class ConnectionCountProbeJUnitTest extends TestCase {
+public class ConnectionCountProbeJUnitTest {
   
+  @Test
   public void test() {
     ConnectionCountProbe probe = new ConnectionCountProbe();
     ServerMetricsImpl metrics = new ServerMetricsImpl(800);
     ServerLoad load = probe.getLoad(metrics);
-    Assert.assertEquals(0f, load.getConnectionLoad(), .0001f);
-    Assert.assertEquals(0f, load.getSubscriptionConnectionLoad(), .0001f);
-    Assert.assertEquals(1/800f, load.getLoadPerConnection(), .0001f);
-    Assert.assertEquals(1f, load.getLoadPerSubscriptionConnection(), .0001f);
+    assertEquals(0f, load.getConnectionLoad(), .0001f);
+    assertEquals(0f, load.getSubscriptionConnectionLoad(), .0001f);
+    assertEquals(1/800f, load.getLoadPerConnection(), .0001f);
+    assertEquals(1f, load.getLoadPerSubscriptionConnection(), .0001f);
 
     for(int i = 0; i < 100; i++) {
       metrics.incConnectionCount();
     }
     
     load = probe.getLoad(metrics);
-    Assert.assertEquals(0.125, load.getConnectionLoad(), .0001f);
+    assertEquals(0.125, load.getConnectionLoad(), .0001f);
   }
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/util/PasswordUtilJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/util/PasswordUtilJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/util/PasswordUtilJUnitTest.java
index 883f733..2b631cd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/util/PasswordUtilJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/util/PasswordUtilJUnitTest.java
@@ -16,16 +16,19 @@
  */
 package com.gemstone.gemfire.cache.util;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.util.PasswordUtil;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.Assert;
-import junit.framework.TestCase;
+@Category({ UnitTest.class, SecurityTest.class })
+public class PasswordUtilJUnitTest {
 
-@Category(UnitTest.class)
-public class PasswordUtilJUnitTest extends TestCase {
+  @Test
   public void testPasswordUtil() {
     String x = "password";
     String z = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLoaderTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLoaderTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLoaderTestCase.java
index bb8bc5e..21e7a7e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLoaderTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLoaderTestCase.java
@@ -119,15 +119,15 @@ public abstract class CacheLoaderTestCase
 //        public Object load2(LoaderHelper helper)
 //          throws CacheLoaderException {
 //
-//          assertEquals(key, helper.getKey());
-//          assertEquals(name, helper.getRegion().getName());
+//          assertIndexDetailsEquals(key, helper.getKey());
+//          assertIndexDetailsEquals(name, helper.getRegion().getName());
 //
 //          try {
 //            RegionAttributes attrs =
 //              helper.getRegion().getAttributes();
 //            if (attrs.getScope().isDistributed()) {
 //              Object result = helper.netSearch(false);
-//              assertEquals(value, result);
+//              assertIndexDetailsEquals(value, result);
 //              return result;
 //            }
 //
@@ -158,8 +158,8 @@ public abstract class CacheLoaderTestCase
 ////            public Object load2(LoaderHelper helper)
 ////              throws CacheLoaderException {
 ////
-////              assertEquals(key, helper.getKey());
-////              assertEquals(name, helper.getRegion().getName());
+////              assertIndexDetailsEquals(key, helper.getKey());
+////              assertIndexDetailsEquals(name, helper.getRegion().getName());
 ////              return value;
 ////            }
 ////          };
@@ -174,9 +174,9 @@ public abstract class CacheLoaderTestCase
 //    });
 //
 //
-//    assertEquals(value, region.get(key));
+//    assertIndexDetailsEquals(value, region.get(key));
 //    assertTrue(loader.wasInvoked());
-//    assertEquals(value, region.getEntry(key).getValue());
+//    assertIndexDetailsEquals(value, region.getEntry(key).getValue());
 //  }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheMapTxnDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheMapTxnDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheMapTxnDUnitTest.java
index 0be86e2..56f0261 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheMapTxnDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheMapTxnDUnitTest.java
@@ -288,7 +288,7 @@ public class CacheMapTxnDUnitTest extends DistributedTestCase{
             
             //test containsValue
             boolean flag = region.containsValue(new String("first"));
-            //assertEquals(true, flag);
+            //assertIndexDetailsEquals(true, flag);
             
             //test remove
             region.put(""+i[2], "second");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheStatisticsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheStatisticsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheStatisticsDUnitTest.java
index ff635b0..18bb79f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheStatisticsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheStatisticsDUnitTest.java
@@ -48,7 +48,7 @@ public class CacheStatisticsDUnitTest extends CacheTestCase {
    * Asserts that two <code>long</code>s are equal concerning a
    * delta.
    */
-//   public static void assertEquals(long expected, long actual,
+//   public static void assertIndexDetailsEquals(long expected, long actual,
 //                                   long delta) {
 //     long difference = Math.abs(expected - actual);
 //     assertTrue("Expected: " + expected
@@ -472,7 +472,7 @@ public class CacheStatisticsDUnitTest extends CacheTestCase {
           // lastAccessed var contains stat from an Entry, which may be
           // up to 100 ms off from stat in Region because Entry has
           // less precision
-          //assertEquals(lastAccessed, stats.getLastAccessedTime(), 100);
+          //assertIndexDetailsEquals(lastAccessed, stats.getLastAccessedTime(), 100);
           assertEquals(0, stats.getHitCount());
           assertEquals(0, stats.getMissCount());
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
index 4835691..ad851ad 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
@@ -523,12 +523,12 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
   //  Region r = getRootRegion().getSubregion(name);
   //  
   //  HeapLRUCapacityController hlcc = (HeapLRUCapacityController) r.getAttributes().getCapacityController();
-  //  assertEquals(hlcc.getEvictionAction(), LRUAlgorithm.OVERFLOW_TO_DISK);
+  //  assertIndexDetailsEquals(hlcc.getEvictionAction(), LRUAlgorithm.OVERFLOW_TO_DISK);
   //  
   //  Properties p = hlcc.getProperties();
-  //  assertEquals(42, Integer.parseInt(p.getProperty(HeapLRUCapacityController.HEAP_PERCENTAGE)));
-  //  assertEquals(32, Long.parseLong(p.getProperty(HeapLRUCapacityController.EVICTOR_INTERVAL)));
-  //  assertEquals(LRUAlgorithm.OVERFLOW_TO_DISK, p.getProperty(HeapLRUCapacityController.EVICTION_ACTION));
+  //  assertIndexDetailsEquals(42, Integer.parseInt(p.getProperty(HeapLRUCapacityController.HEAP_PERCENTAGE)));
+  //  assertIndexDetailsEquals(32, Long.parseLong(p.getProperty(HeapLRUCapacityController.EVICTOR_INTERVAL)));
+  //  assertIndexDetailsEquals(LRUAlgorithm.OVERFLOW_TO_DISK, p.getProperty(HeapLRUCapacityController.EVICTION_ACTION));
   //}
   /**
    * Test Publisher region attribute
@@ -542,7 +542,7 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
 //    attrs.setPublisher(true);
 //    cache.createRegion("root", attrs);
 //    testXml(cache);
-//    assertEquals(true, cache.getRegion("root").getAttributes().getPublisher());
+//    assertIndexDetailsEquals(true, cache.getRegion("root").getAttributes().getPublisher());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
index 6b37acc..5cda16b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
@@ -234,7 +234,7 @@ public class CacheXml57DUnitTest extends CacheXml55DUnitTest
     assertEquals(12351, cp.getStatisticInterval());
     assertEquals("mygroup", cp.getServerGroup());
       // commented this out until queues are implemented
-    //    assertEquals(true, cp.getQueueEnabled());
+    //    assertIndexDetailsEquals(true, cp.getQueueEnabled());
     assertEquals(12345, cp.getSubscriptionRedundancy());
     assertEquals(12345, cp.getSubscriptionMessageTrackingTimeout());
     assertEquals(333, cp.getSubscriptionAckInterval());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml58DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml58DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml58DUnitTest.java
index c56b6ed..d5123e7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml58DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml58DUnitTest.java
@@ -304,7 +304,7 @@ public class CacheXml58DUnitTest extends CacheXml57DUnitTest
     assertEquals(regionAttrs.getEntryTimeToLive().getTimeout(), expiration.getTimeout());    
 //  TODO mthomas 01/20/09 Move test back to using LRUHeap when config issues have settled
 //    if (getGemFireVersion().equals(CacheXml.GEMFIRE_6_0)) {
-//      assertEquals(ea.getAlgorithm(),EvictionAlgorithm.LRU_HEAP);
+//      assertIndexDetailsEquals(ea.getAlgorithm(),EvictionAlgorithm.LRU_HEAP);
 //    } else {
     assertEquals(ea.getAlgorithm(),EvictionAlgorithm.LRU_MEMORY);
 //    }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml65DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml65DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml65DUnitTest.java
index c50d6d6..56376e0 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml65DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml65DUnitTest.java
@@ -275,7 +275,7 @@ public class CacheXml65DUnitTest extends CacheXml61DUnitTest {
     assertEquals(12351, cp.getStatisticInterval());
     assertEquals("mygroup", cp.getServerGroup());
     // commented this out until queues are implemented
-    // assertEquals(true, cp.getQueueEnabled());
+    // assertIndexDetailsEquals(true, cp.getQueueEnabled());
     assertEquals(12345, cp.getSubscriptionRedundancy());
     assertEquals(12345, cp.getSubscriptionMessageTrackingTimeout());
     assertEquals(333, cp.getSubscriptionAckInterval());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
index 46e91b0..3055228 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
@@ -1611,7 +1611,7 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
       totalClientCounts += clientCounts[i];
     }
     // this assertion fails because the count is 4
-    //assertEquals(1, totalClientCounts);
+    //assertIndexDetailsEquals(1, totalClientCounts);
   }
   protected static int testGetNotifiedClients_port;
   private static int getTestGetNotifiedClients_port() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskRegionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskRegionDUnitTest.java
index d7e1255..aa327f8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskRegionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskRegionDUnitTest.java
@@ -1481,7 +1481,7 @@ public class DiskRegionDUnitTest extends CacheTestCase {
 //         for (int i = 0; i < NB1_NUM_ENTRIES; i++) {
 //           rgn.put(new Integer(i), values[i]);
 //         }
-//         assertEquals(NB1_NUM_ENTRIES, rgn.keys().size());
+//         assertIndexDetailsEquals(NB1_NUM_ENTRIES, rgn.keys().size());
 //         //close and create to ensure that all data will go to htree 
 //         //TODO: Mitul : remove this later to fine tune test to also take oplogs recovery into account
 //         rgn.close();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistAckMapMethodsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistAckMapMethodsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistAckMapMethodsDUnitTest.java
index 20051f7..8555a07 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistAckMapMethodsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistAckMapMethodsDUnitTest.java
@@ -583,7 +583,7 @@ public class DistAckMapMethodsDUnitTest extends DistributedTestCase{
             
             //test args for containsKey method
             boolean val2 = region.containsValue(new String("second"));
-            //assertEquals(true, val2);
+            //assertIndexDetailsEquals(true, val2);
             
             //test args for remove method
             try{

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
index 6050e3d..0acd2aa 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
@@ -172,10 +172,10 @@ public class DistributedAckPersistentRegionCCEDUnitTest extends DistributedAckRe
 //
 //    VersionTag tag0 = getVersionTag(vm0, key);
 //    VersionTag tag1 = getVersionTag(vm1, key);
-//    assertEquals(3, tag0.getRegionVersion());
-//    assertEquals(3, tag0.getEntryVersion());
-//    assertEquals(3, tag1.getRegionVersion());
-//    assertEquals(3, tag1.getEntryVersion());
+//    assertIndexDetailsEquals(3, tag0.getRegionVersion());
+//    assertIndexDetailsEquals(3, tag0.getEntryVersion());
+//    assertIndexDetailsEquals(3, tag1.getRegionVersion());
+//    assertIndexDetailsEquals(3, tag1.getEntryVersion());
 //
 //    // shutdown and recover
 //    vm0.invoke(disconnect);
@@ -185,10 +185,10 @@ public class DistributedAckPersistentRegionCCEDUnitTest extends DistributedAckRe
 //
 //    tag0 = getVersionTag(vm0, key);
 //    tag1 = getVersionTag(vm1, key);
-//    assertEquals(3, tag0.getRegionVersion());
-//    assertEquals(3, tag0.getEntryVersion());
-//    assertEquals(3, tag1.getRegionVersion());
-//    assertEquals(3, tag1.getEntryVersion());
+//    assertIndexDetailsEquals(3, tag0.getRegionVersion());
+//    assertIndexDetailsEquals(3, tag0.getEntryVersion());
+//    assertIndexDetailsEquals(3, tag1.getRegionVersion());
+//    assertIndexDetailsEquals(3, tag1.getEntryVersion());
 //
 //    vm0.invoke(new SerializableRunnable("put with version 1, value vm1") {
 //      public void run() {
@@ -202,20 +202,20 @@ public class DistributedAckPersistentRegionCCEDUnitTest extends DistributedAckRe
 //          DistributedRegion.LOCALCLEAR_TESTHOOK = false;
 //        }
 //        CCRegion.put(key, "vm0");
-//        Assert.assertEquals("vm0", CCRegion.get(key));
+//        Assert.assertIndexDetailsEquals("vm0", CCRegion.get(key));
 //      }
 //    });
 //    vm1.invoke(new SerializableRunnable("verify that value has not been updated") {
 //      public void run() {
-//        Assert.assertEquals("dummy", CCRegion.get(key));
+//        Assert.assertIndexDetailsEquals("dummy", CCRegion.get(key));
 //      }
 //    });
 //
 //    tag0 = getVersionTag(vm0, key);
 //    tag1 = getVersionTag(vm1, key);
-//    assertEquals(4, tag0.getRegionVersion());
-//    assertEquals(1, tag0.getEntryVersion());
-//    assertEquals(3, tag1.getRegionVersion());
-//    assertEquals(3, tag1.getEntryVersion());
+//    assertIndexDetailsEquals(4, tag0.getRegionVersion());
+//    assertIndexDetailsEquals(1, tag0.getEntryVersion());
+//    assertIndexDetailsEquals(3, tag1.getRegionVersion());
+//    assertIndexDetailsEquals(3, tag1.getEntryVersion());
 //  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
index 04662ca..9ad102d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
@@ -1729,8 +1729,8 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //            public Object load2(LoaderHelper helper)
 //              throws CacheLoaderException {
 //
-//              assertEquals(key, helper.getKey());
-//              assertEquals(name, helper.getRegion().getName());
+//              assertIndexDetailsEquals(key, helper.getKey());
+//              assertIndexDetailsEquals(name, helper.getRegion().getName());
 //              return value;
 //            }
 //          };
@@ -2844,15 +2844,15 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //           // longer optimize updates based on size.
 //           Region.Entry entry1 = region.getEntry(key1);
 //           assertNotNull(entry1);
-//           assertEquals(value1, entry1.getValue());
+//           assertIndexDetailsEquals(value1, entry1.getValue());
 
 //           Region.Entry entry2 = region.getEntry(key2);
 //           assertNotNull(entry2);
-//           assertEquals(value2, entry2.getValue());
+//           assertIndexDetailsEquals(value2, entry2.getValue());
 
 //           Region.Entry entry3 = region.getEntry(key3);
 //           assertNotNull(entry3);
-//           assertEquals(value3, entry3.getValue());
+//           assertIndexDetailsEquals(value3, entry3.getValue());
 //         }
 //       });
 //   }
@@ -4395,7 +4395,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //               if (getRegionAttributes().getScope().isDistributedAck()) {
 //                 // do a nonblocking netSearch
 //                 region.localInvalidate(key);
-//                 assertEquals(value, region.get(key));
+//                 assertIndexDetailsEquals(value, region.get(key));
 //               }
               break;
             case 3: // INVALIDATE
@@ -4720,7 +4720,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //               if (getRegionAttributes().getScope().isDistributedAck()) {
 //                 // do a nonblocking netSearch
 //                 region.localInvalidate(key);
-//                 assertEquals(value, region.get(key));
+//                 assertIndexDetailsEquals(value, region.get(key));
 //               }
               break;
             case 3: // INVALIDATE
@@ -5037,7 +5037,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //               if (getRegionAttributes().getScope().isDistributedAck()) {
 //                 // do a nonblocking netSearch
 //                 region.localInvalidate(key);
-//                 assertEquals(value, region.get(key));
+//                 assertIndexDetailsEquals(value, region.get(key));
 //               }
               break;
             case 3: // INVALIDATE
@@ -5333,7 +5333,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //               if (getRegionAttributes().getScope().isDistributedAck()) {
 //                 // do a nonblocking netSearch
 //                 region.localInvalidate(key);
-//                 assertEquals(value, region.get(key));
+//                 assertIndexDetailsEquals(value, region.get(key));
 //               }
               break;
             case 3: // INVALIDATE
@@ -6716,7 +6716,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 
           ev = (EntryEvent)it.next();
           assertSame(rgn1, ev.getRegion());
-          //assertEquals(tl.expectedTxId, ev.getTransactionId());
+          //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
           assertEquals("key", ev.getKey());
           assertEquals("value1", ev.getNewValue());
           assertEquals(null, ev.getOldValue());
@@ -6732,7 +6732,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 
           ev = (EntryEvent)it.next();
           assertSame(rgn3, ev.getRegion());
-          //assertEquals(tl.expectedTxId, ev.getTransactionId());
+          //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
           assertEquals("key", ev.getKey());
           assertEquals("value3", ev.getNewValue());
           assertEquals(null, ev.getOldValue());
@@ -6791,7 +6791,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 
           ev = (EntryEvent)it.next();
           assertSame(rgn2, ev.getRegion());
-          //assertEquals(tl.expectedTxId, ev.getTransactionId());
+          //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
           assertEquals("key", ev.getKey());
           assertEquals("value2", ev.getNewValue());
           assertEquals(null, ev.getOldValue());
@@ -6807,7 +6807,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 
           ev = (EntryEvent)it.next();
           assertSame(rgn3, ev.getRegion());
-          //assertEquals(tl.expectedTxId, ev.getTransactionId());
+          //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
           assertEquals("key", ev.getKey());
           assertEquals("value3", ev.getNewValue());
           assertEquals(null, ev.getOldValue());
@@ -6847,7 +6847,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
             }
             assertEquals(1, events.size());
             EntryEvent ev = (EntryEvent)events.iterator().next();
-            //assertEquals(tl.expectedTxId, ev.getTransactionId());
+            //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
             assertTrue(ev.getRegion() == rgn);
             assertEquals("key", ev.getKey());
             assertEquals("value1", ev.getNewValue());
@@ -6887,7 +6887,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
             }
             assertEquals(1, events.size());
             EntryEvent ev = (EntryEvent)events.iterator().next();
-            //assertEquals(tl.expectedTxId, ev.getTransactionId());
+            //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
             assertTrue(ev.getRegion() == rgn);
             assertEquals("key", ev.getKey());
             assertEquals("value2", ev.getNewValue());
@@ -6927,7 +6927,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
             }
             assertEquals(1, events.size());
             EntryEvent ev = (EntryEvent)events.iterator().next();
-            //assertEquals(tl.expectedTxId, ev.getTransactionId());
+            //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
             assertTrue(ev.getRegion() == rgn);
             assertEquals("key", ev.getKey());
             assertEquals("value3", ev.getNewValue());
@@ -7379,7 +7379,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
           Collection events = tl.lastEvent.getCreateEvents();
           assertEquals(1, events.size());
           EntryEvent ev = (EntryEvent)events.iterator().next();
-          //assertEquals(tl.expectedTxId, ev.getTransactionId());
+          //assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
           assertTrue(ev.getRegion() == rgn);
           assertEquals("key", ev.getKey());
           assertEquals("value", ev.getNewValue());
@@ -7530,7 +7530,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
               }
               assertEquals(1, events.size());
               EntryEvent ev = (EntryEvent)events.iterator().next();
-              // assertEquals(tl.expectedTxId, ev.getTransactionId());
+              // assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
               assertTrue(ev.getRegion() == rgn1);
               assertEquals("key", ev.getKey());
               assertEquals("value2", ev.getNewValue());
@@ -7608,7 +7608,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
               }
               assertEquals(1, events.size());
               EntryEvent ev = (EntryEvent)events.iterator().next();
-              // assertEquals(tl.expectedTxId, ev.getTransactionId());
+              // assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
               assertTrue(ev.getRegion() == rgn1);
               assertEquals("key", ev.getKey());
               assertNull(ev.getNewValue());
@@ -7753,7 +7753,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
               Collection events = tl.lastEvent.getDestroyEvents();
               assertEquals(1, events.size());
               EntryEvent ev = (EntryEvent)events.iterator().next();
-              // assertEquals(tl.expectedTxId, ev.getTransactionId());
+              // assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
               assertTrue(ev.getRegion() == rgn1);
               assertNull(ev.getKey());
               assertNull(ev.getNewValue());
@@ -7817,7 +7817,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
               }
               assertEquals(1, events.size());
               EntryEvent ev = (EntryEvent)events.iterator().next();
-              // assertEquals(tl.expectedTxId, ev.getTransactionId());
+              // assertIndexDetailsEquals(tl.expectedTxId, ev.getTransactionId());
               assertTrue(ev.getRegion() == rgn1);
               assertEquals("key", ev.getKey());
               assertEquals("value1", ev.getNewValue());
@@ -8404,13 +8404,13 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
       // that they originate that maybe should have been elided.  For this reason we can't
       // guarantee their consistency and don't check for it here.
 //      if (r1Contents.containsKey(key)) {
-//        assertEquals("region contents are not consistent", r1Contents.get(key), r2Contents.get(key));
+//        assertIndexDetailsEquals("region contents are not consistent", r1Contents.get(key), r2Contents.get(key));
 //      }
       assertEquals("region contents are not consistent for " + key, r2Contents.get(key), r3Contents.get(key));
       for (int subi=1; subi<3; subi++) {
         String subkey = key + "-" + subi;
         if (r2Contents.containsKey(subkey)) {
-//          assertEquals("region contents are not consistent for " + subkey, r1Contents.get(subkey), r2Contents.get(subkey));
+//          assertIndexDetailsEquals("region contents are not consistent for " + subkey, r1Contents.get(subkey), r2Contents.get(subkey));
           assertEquals("region contents are not consistent for " + subkey, r2Contents.get(subkey), r3Contents.get(subkey));
         } else {
           // can't assert this because a clear() op will cause non-replicated to start rejecting
@@ -8628,8 +8628,8 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
 //            fail("sleep was interrupted");
 //          }
 //          count = CCRegion.getTombstoneCount();
-//          assertEquals("expected zero tombstones", 0, count);
-//          assertEquals(numEntries, CCRegion.size());
+//          assertIndexDetailsEquals("expected zero tombstones", 0, count);
+//          assertIndexDetailsEquals(numEntries, CCRegion.size());
         }
       });
 
@@ -8921,7 +8921,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
       assertEquals("region contents are not consistent", r1Contents.get(key), r2Contents.get(key));
       assertEquals("region contents are not consistent", r2Contents.get(key), r3Contents.get(key));
 //      if (r0Contents.containsKey(key)) {
-//        assertEquals("region contents are not consistent", r1Contents.get(key), r0Contents.get(key));
+//        assertIndexDetailsEquals("region contents are not consistent", r1Contents.get(key), r0Contents.get(key));
 //      }
       for (int subi=1; subi<3; subi++) {
         String subkey = key + "-" + subi;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/PutAllMultiVmDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PutAllMultiVmDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PutAllMultiVmDUnitTest.java
index 2baeb52..cee7263 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PutAllMultiVmDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PutAllMultiVmDUnitTest.java
@@ -174,7 +174,7 @@ public class PutAllMultiVmDUnitTest extends DistributedTestCase{
         
         vm0.invoke(new CacheSerializableRunnable("testSimplePutAll2"){
             public void run2() throws CacheException {
-                //assertEquals(0, region.size());
+                //assertIndexDetailsEquals(0, region.size());
                 createMirroredRegion();
                 cacheTxnMgr = cache.getCacheTransactionManager();
                 int cntr = 0;
@@ -276,7 +276,7 @@ public class PutAllMultiVmDUnitTest extends DistributedTestCase{
         
         vm0.invoke(new CacheSerializableRunnable("testPutAllExceptions2"){
             public void run2() throws CacheException {
-                //assertEquals(0, region.size());
+                //assertIndexDetailsEquals(0, region.size());
                 createMirroredRegion();
                 
                 for(int i=1; i<6; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/QueueMsgDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/QueueMsgDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/QueueMsgDUnitTest.java
index ae0c3f2..76c1b03 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/QueueMsgDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/QueueMsgDUnitTest.java
@@ -135,7 +135,7 @@ public class QueueMsgDUnitTest extends ReliabilityTestCase {
         public void run2() throws CacheException {
           Region r = getRootRegion();
           assertEquals(null, r.getEntry("createKey"));
-          //assertEquals("putValue", r.getEntry("createKey").getValue());
+          //assertIndexDetailsEquals("putValue", r.getEntry("createKey").getValue());
           {
             int evIdx = 0;
             TestCacheListener cl = (TestCacheListener)r.getAttributes().getCacheListener();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionMembershipListenerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionMembershipListenerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionMembershipListenerDUnitTest.java
index 1a8d126..2f6b7a5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionMembershipListenerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionMembershipListenerDUnitTest.java
@@ -194,7 +194,7 @@ public class RegionMembershipListenerDUnitTest extends CacheTestCase {
     // test new methods added for #43098
     if (expectedId != null) {
       Cache cache = (Cache)this.r.getRegionService();
-      //assertEquals(l, new ArrayList(cache.getMembers()));
+      //assertIndexDetailsEquals(l, new ArrayList(cache.getMembers()));
       assertEquals(l, new ArrayList(cache.getMembers(this.r)));
       assertEquals(l, new ArrayList(cache.getMembers(this.sr)));
     }
@@ -219,7 +219,7 @@ public class RegionMembershipListenerDUnitTest extends CacheTestCase {
       assertEquals(this.r, e.getRegion());
       // the test now uses a hook to get the member's DistributionAdvisor profile in the callback argument
       assertTrue(e.getCallbackArgument() instanceof Profile);
-//      assertEquals(null, e.getCallbackArgument());
+//      assertIndexDetailsEquals(null, e.getCallbackArgument());
     }
     assertTrue(this.mySRListener.lastOpWasCreate());
     {
@@ -231,7 +231,7 @@ public class RegionMembershipListenerDUnitTest extends CacheTestCase {
       assertEquals(this.sr, e.getRegion());
       // the test now uses a hook to get the member's DistributionAdvisor profile in the callback argument
       assertTrue(e.getCallbackArgument() instanceof Profile);
-//      assertEquals(null, e.getCallbackArgument());
+//      assertIndexDetailsEquals(null, e.getCallbackArgument());
     }
   }
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
index e2e270c..45cb8a5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
@@ -421,7 +421,7 @@ public abstract class RegionTestCase extends CacheTestCase {
     Region subregion = region.createSubregion(name, attrs);
     assertTrue(attrs != subregion.getAttributes());
     /* @todo compare each individual attribute for equality?
-    assertEquals(attrs, subregion.getAttributes());
+    assertIndexDetailsEquals(attrs, subregion.getAttributes());
      */
     
     Set subregions = region.subregions(false);
@@ -1607,7 +1607,7 @@ public abstract class RegionTestCase extends CacheTestCase {
       // assert that if this is a disk region, the disk dirs are empty
       // to make sure we start with a clean slate
       getCache().getLogger().info("list="+Arrays.toString(diskDir.list()));
-//       assertEquals("list="+Arrays.toString(diskDir.list()),
+//       assertIndexDetailsEquals("list="+Arrays.toString(diskDir.list()),
 //                    0, diskDir.list().length);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache30/SlowRecDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/SlowRecDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/SlowRecDUnitTest.java
index 3b6034a..2441256 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/SlowRecDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/SlowRecDUnitTest.java
@@ -194,7 +194,7 @@ public class SlowRecDUnitTest extends CacheTestCase {
               };
               Wait.waitForCriterion(ev, 50 * 1000, 200, true);
 //              assertNotNull(re);
-//              assertEquals(null, value);
+//              assertIndexDetailsEquals(null, value);
             }
           } else {
             {
@@ -1451,7 +1451,7 @@ public class SlowRecDUnitTest extends CacheTestCase {
               continue;
             }
             assertEquals(new Integer(expectedArgs[i]), arg);
-            //assertEquals(CALLBACK_UPDATE_INTEGER, type);
+            //assertIndexDetailsEquals(CALLBACK_UPDATE_INTEGER, type);
             i++;
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
index f5867d4..05d2368 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
@@ -42,7 +42,7 @@ import org.junit.experimental.categories.Category;
 @Category(UnitTest.class)
 public class AbstractLauncherJUnitTest {
 
-  protected AbstractLauncher<?> createAbstractLauncher(final String memberName, final String memberId) {
+  private AbstractLauncher<?> createAbstractLauncher(final String memberName, final String memberId) {
     return new FakeServiceLauncher(memberName, memberId);
   }
 
@@ -249,7 +249,7 @@ public class AbstractLauncherJUnitTest {
       TimeUnit.DAYS.toMillis(2) + TimeUnit.HOURS.toMillis(1) + TimeUnit.MINUTES.toMillis(30) + TimeUnit.SECONDS.toMillis(1)));
   }
 
-  protected static final class FakeServiceLauncher extends AbstractLauncher<String> {
+  private static final class FakeServiceLauncher extends AbstractLauncher<String> {
 
     private final String memberId;
     private final String memberName;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherServiceStatusJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherServiceStatusJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherServiceStatusJUnitTest.java
index ca24a4e..c613d4c 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherServiceStatusJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherServiceStatusJUnitTest.java
@@ -22,23 +22,21 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
-import java.net.UnknownHostException;
 import java.util.Arrays;
 import java.util.List;
 
-import com.gemstone.gemfire.distributed.AbstractLauncherServiceStatusJUnitTest.TestLauncher.TestState;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.internal.GemFireVersion;
-import com.gemstone.gemfire.internal.process.PidUnavailableException;
 import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonArray;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonObject;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * Tests marshaling of ServiceStatus to and from JSON.
  * 
@@ -47,34 +45,58 @@ import org.junit.experimental.categories.Category;
 @Category(UnitTest.class)
 public class AbstractLauncherServiceStatusJUnitTest {
 
-  private static final String SERVICE_NAME = "Test";
-  private static final InetAddress HOST = getLocalHost();
-  private static final int PORT = 12345;
-  private static final String NAME = AbstractLauncherServiceStatusJUnitTest.class.getSimpleName();
-  private static final int PID = identifyPid();
-  private static final long UPTIME = 123456789;
-  private static final String WORKING_DIRECTORY = identifyWorkingDirectory();
-  private static final List<String> JVM_ARGUMENTS = ManagementFactory.getRuntimeMXBean().getInputArguments();
-  private static final String CLASSPATH = ManagementFactory.getRuntimeMXBean().getClassPath();
-  private static final String GEMFIRE_VERSION = GemFireVersion.getGemFireVersion();
-  private static final String JAVA_VERSION = System.getProperty("java.version");
+  private static String serviceName;
+  private static InetAddress host;
+  private static int port;
+  private static String name;
+  private static int pid;
+  private static long uptime;
+  private static String workingDirectory;
+  private static List<String> jvmArguments;
+  private static String classpath;
+  private static String gemfireVersion;
+  private static String javaVersion;
 
   private TestLauncher launcher;
 
   @Before
-  public void setUp() {
-    this.launcher = new TestLauncher(HOST, PORT, NAME);
+  public void setUp() throws Exception {
+    serviceName = "Test";
+    port = 12345;
+    host = InetAddress.getLocalHost();
+    pid = ProcessUtils.identifyPid();
+    uptime = 123456789;
+    name = AbstractLauncherServiceStatusJUnitTest.class.getSimpleName();
+    workingDirectory = new File(System.getProperty("user.dir")).getAbsolutePath();
+    jvmArguments = ManagementFactory.getRuntimeMXBean().getInputArguments();
+    classpath = ManagementFactory.getRuntimeMXBean().getClassPath();
+    gemfireVersion = GemFireVersion.getGemFireVersion();
+    javaVersion = System.getProperty("java.version");
+
+    this.launcher = new TestLauncher(host, port, name);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    serviceName = null;
+    host = null;
+    name = null;
+    workingDirectory = null;
+    jvmArguments = null;
+    classpath = null;
+    gemfireVersion = null;
+    javaVersion = null;
   }
 
   @Test
   public void testMarshallingTestStatusToAndFromJson() {
-    final TestState status = this.launcher.status();
+    final TestLauncher.TestState status = this.launcher.status();
     final String json = status.toJson();
     validateJson(status, json);
-    validateStatus(status, TestState.fromJson(json));
+    validateStatus(status, TestLauncher.TestState.fromJson(json));
   }
 
-  private void validateStatus(final TestState expected, final TestState actual) {
+  private void validateStatus(final TestLauncher.TestState expected, final TestLauncher.TestState actual) {
     assertEquals(expected.getClasspath(), actual.getClasspath());
     assertEquals(expected.getGemFireVersion(), actual.getGemFireVersion());
     assertEquals(expected.getJavaVersion(), actual.getJavaVersion());
@@ -89,39 +111,12 @@ public class AbstractLauncherServiceStatusJUnitTest {
     assertEquals(expected.getMemberName(), actual.getMemberName());
   }
 
-  private void validateJson(final TestState expected, final String json) {
-    final TestState actual = TestState.fromJson(json);
+  private void validateJson(final TestLauncher.TestState expected, final String json) {
+    final TestLauncher.TestState actual = TestLauncher.TestState.fromJson(json);
     validateStatus(expected, actual);
   }
 
-  private static int identifyPid() {
-    try {
-      return ProcessUtils.identifyPid();
-    }
-    catch (PidUnavailableException e) {
-      return 0;
-    }
-  }
-
-  private static String identifyWorkingDirectory() {
-    try {
-      return new File(System.getProperty("user.dir")).getCanonicalPath();
-    }
-    catch (IOException e) {
-      return new File(System.getProperty("user.dir")).getAbsolutePath();
-    }
-  }
-
-  private static InetAddress getLocalHost() {
-    try {
-      return InetAddress.getLocalHost();
-    }
-    catch (UnknownHostException e) {
-      return null;
-    }
-  }
-  
-  static class TestLauncher extends AbstractLauncher<String> {
+  private static class TestLauncher extends AbstractLauncher<String> {
 
     private final InetAddress bindAddress;
     private final int port;
@@ -142,17 +137,17 @@ public class AbstractLauncherServiceStatusJUnitTest {
         null,
         System.currentTimeMillis(),
         getId(),
-        PID,
-        UPTIME,
-        WORKING_DIRECTORY,
-        JVM_ARGUMENTS,
-        CLASSPATH,
-        GEMFIRE_VERSION,
-        JAVA_VERSION,
+              pid,
+              uptime,
+              workingDirectory,
+              jvmArguments,
+              classpath,
+              gemfireVersion,
+              javaVersion,
         getLogFileName(),
         getBindAddressAsString(),
         getPortAsString(),
-        NAME);
+              name);
     }
 
     @Override
@@ -185,7 +180,7 @@ public class AbstractLauncherServiceStatusJUnitTest {
 
     @Override
     public String getServiceName() {
-      return SERVICE_NAME;
+      return serviceName;
     }
 
     InetAddress getBindAddress() {
@@ -204,7 +199,7 @@ public class AbstractLauncherServiceStatusJUnitTest {
       return String.valueOf(getPort());
     }
 
-    public static class TestState extends ServiceState<String> {
+    private static class TestState extends ServiceState<String> {
 
       protected static TestState fromJson(final String json) {
         try {
@@ -256,7 +251,7 @@ public class AbstractLauncherServiceStatusJUnitTest {
 
       @Override
       protected String getServiceName() {
-        return SERVICE_NAME;
+        return serviceName;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
index a8ddd6a..39204ec 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
@@ -21,20 +21,19 @@ import static org.junit.Assert.*;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 
-import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
-import com.gemstone.gemfire.distributed.LocatorLauncher.Command;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
 import joptsimple.OptionException;
-
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Command;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * The LocatorLauncherJUnitTest class is a test suite of test cases for testing the contract and functionality of
  * launching a GemFire Locator.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalJUnitTest.java
index bfed4d0..0bc4750 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalJUnitTest.java
@@ -297,7 +297,7 @@ public class LocatorLauncherLocalJUnitTest extends AbstractLocatorLauncherJUnitT
       final int pid = readPid(this.pidFile);
       assertTrue(pid > 0);
       assertTrue(ProcessUtils.isProcessAlive(pid));
-      assertEquals(getPid(), pid);
+      assertIndexDetailsEquals(getPid(), pid);
       
       // validate log file was created
       final String logFileName = getUniqueName()+".log";
@@ -311,7 +311,7 @@ public class LocatorLauncherLocalJUnitTest extends AbstractLocatorLauncherJUnitT
     }
 
     try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
+      assertIndexDetailsEquals(Status.STOPPED, this.launcher.stop().getStatus());
       waitForFileToDelete(this.pidFile);
     } catch (Throwable e) {
       logger.error(e);
@@ -442,7 +442,7 @@ public class LocatorLauncherLocalJUnitTest extends AbstractLocatorLauncherJUnitT
       expected = e;
       assertNotNull(expected.getMessage());
       assertTrue(expected.getMessage(), expected.getMessage().contains("A PID file already exists and a Locator may be running in"));
-      assertEquals(RuntimeException.class, expected.getClass());
+      assertIndexDetailsEquals(RuntimeException.class, expected.getClass());
     } catch (Throwable e) {
       logger.error(e);
       if (failure == null) {


[13/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledInJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledInJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledInJUnitTest.java
index 4ee2299..b20b365 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledInJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledInJUnitTest.java
@@ -16,15 +16,13 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 import java.util.ArrayList;
 import java.util.Collection;
 
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -56,7 +54,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
   
   @Test
@@ -67,7 +65,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
   
   @Test
@@ -78,7 +76,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
   
   @Test
@@ -89,7 +87,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
   
   @Test
@@ -125,7 +123,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -136,7 +134,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -147,7 +145,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -158,7 +156,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -169,7 +167,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -180,7 +178,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -191,7 +189,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -202,7 +200,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -213,7 +211,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -224,7 +222,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -235,7 +233,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -246,7 +244,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -257,7 +255,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -268,7 +266,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -279,7 +277,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -290,7 +288,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -301,7 +299,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -312,7 +310,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -323,7 +321,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -334,7 +332,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -360,7 +358,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -373,7 +371,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -385,7 +383,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   // String form
@@ -399,7 +397,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -412,7 +410,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertFalse((Boolean) result);
+    assertFalse((Boolean) result);
   }
 
   @Test
@@ -425,7 +423,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -437,7 +435,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertTrue((Boolean) result);
+    assertTrue((Boolean) result);
   }
 
   @Test
@@ -446,7 +444,7 @@ public class CompiledInJUnitTest {
 
     CompiledIn compiledIn = new CompiledIn(elm, colln);
     Object result = compiledIn.evaluate(context);
-    Assert.assertNotNull(result);
+    assertNotNull(result);
   }
   
   private PdxInstanceEnumInfo createPdxInstanceEnumInfo(Enum<?> e, int enumId) {
@@ -455,6 +453,4 @@ public class CompiledInJUnitTest {
   }
   
   private enum EnumForTest {ONE, TWO, THREE};
-  
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/NWayMergeResultsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/NWayMergeResultsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/NWayMergeResultsJUnitTest.java
index a4043f0..a39cd8e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/NWayMergeResultsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/NWayMergeResultsJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
+import static org.junit.Assert.*;
+
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,10 +30,7 @@ import java.util.NoSuchElementException;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import junit.framework.Test;
-import junit.framework.TestCase;
-import junit.framework.TestSuite;
-
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.query.Struct;
@@ -40,22 +39,10 @@ import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl;
 import com.gemstone.gemfire.cache.query.types.ObjectType;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * 
- *
- */
 @Category(UnitTest.class)
-public class NWayMergeResultsJUnitTest extends TestCase {
-
-  public NWayMergeResultsJUnitTest(String testName) {
-    super(testName);
-  }
-
-  public static Test suite() {
-    TestSuite suite = new TestSuite(NWayMergeResultsJUnitTest.class);
-    return suite;
-  }
+public class NWayMergeResultsJUnitTest {
 
+  @Test
   public void testNonDistinct() throws Exception {
     final int numSortedLists = 40;
     Collection<List<Integer>> listOfSortedLists = new ArrayList<List<Integer>>();
@@ -100,6 +87,7 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     assertEquals(combinedArray.length, mergedResults.size());
   }
 
+  @Test
   public void testDistinct() throws Exception {
     final int numSortedLists = 40;
     Collection<List<Integer>> listOfSortedLists = new ArrayList<List<Integer>>();
@@ -141,8 +129,8 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     assertEquals(sortedSet.size(), mergedResults.size());
   }
 
+  @Test
   public void testLimitNoDistinct() throws Exception {
-
     final int numSortedLists = 40;
     final int limit = 53;
     Collection<List<Integer>> listOfSortedLists = new ArrayList<List<Integer>>();
@@ -193,6 +181,7 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     assertEquals(limit, mergedResults.size());
   }
 
+  @Test
   public void testLimitDistinct() throws Exception {
     final int numSortedLists = 40;
     final int limit = 53;
@@ -241,6 +230,7 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     assertEquals(limit, mergedResults.size());
   }
 
+  @Test
   public void testNonDistinctStruct() throws Exception {
     final int numSortedLists = 40;
     StructTypeImpl structType = new StructTypeImpl(new String[] { "a", "b" },
@@ -304,6 +294,7 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testDistinctStruct() throws Exception {
     final int numSortedLists = 40;
     StructTypeImpl structType = new StructTypeImpl(new String[] { "a", "b" },
@@ -363,8 +354,8 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testOccurenceNonDistinct() throws Exception {
-
     final int numSortedLists = 40;
     Collection<List<Integer>> listOfSortedLists = new ArrayList<List<Integer>>();
     for (int i = 0; i < numSortedLists; ++i) {
@@ -411,8 +402,8 @@ public class NWayMergeResultsJUnitTest extends TestCase {
     assertEquals(num75, mergedResults.occurrences(Integer.valueOf(75)));
   }
 
+  @Test
   public void testOccurenceDistinct() throws Exception {
-
     final int numSortedLists = 40;
     Collection<List<Integer>> listOfSortedLists = new ArrayList<List<Integer>>();
     for (int i = 0; i < numSortedLists; ++i) {
@@ -484,6 +475,7 @@ public class NWayMergeResultsJUnitTest extends TestCase {
         orderByAttribs, context, structType);
   }
 
+  @Test
   public void testCombination() throws Exception {
     List<String> results1 = new ArrayList<String>();
     results1.add("IBM");
@@ -542,9 +534,6 @@ public class NWayMergeResultsJUnitTest extends TestCase {
         sortedLists1, true, -1);
 
     assertEquals(12, netMergedResults.size());
-
   }
-  
-  
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
index 22bdf39..8226363 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryFromClauseCanonicalizationJUnitTest.java
@@ -201,7 +201,7 @@ public class QueryFromClauseCanonicalizationJUnitTest
 ////    context.setBindArguments(new Object[]{"bindkey"});
 //    sbuff = new StringBuffer();
 //    cop.generateCanonicalizedExpression(sbuff, context);
-//    assertEquals(sbuff.toString(),"iter1.positions.get('bindkey')");
+//    assertIndexDetailsEquals(sbuff.toString(),"iter1.positions.get('bindkey')");
 //    
     
 //    cp = new CompiledPath(new CompiledID("pf"), "getPositions()");
@@ -211,7 +211,7 @@ public class QueryFromClauseCanonicalizationJUnitTest
 //    cop = new CompiledOperation(cp, "get", args);
 //    sbuff = new StringBuffer();
 //    cop.generateCanonicalizedExpression(sbuff, context);
-//    assertEquals(sbuff.toString(),"iter1.positions().get('bindkey')");
+//    assertIndexDetailsEquals(sbuff.toString(),"iter1.positions().get('bindkey')");
 //    
 //    
 //    cp = new CompiledPath(new CompiledID("pf"), "getPositions");
@@ -221,7 +221,7 @@ public class QueryFromClauseCanonicalizationJUnitTest
 //    cop = new CompiledOperation(cp, "get", args);
 //    sbuff = new StringBuffer();
 //    cop.generateCanonicalizedExpression(sbuff, context);
-//    assertEquals(sbuff.toString(),"iter1.positions.get('bindkey')");
+//    assertIndexDetailsEquals(sbuff.toString(),"iter1.positions.get('bindkey')");
     
     cp = new CompiledPath(new CompiledID("pf"), "getPositions");
     CompiledPath cp1 = new CompiledPath(new CompiledID("pf"),"pkid");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryObjectSerializationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryObjectSerializationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryObjectSerializationJUnitTest.java
index 8da72df..b593da5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryObjectSerializationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/QueryObjectSerializationJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -26,8 +28,9 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collection;
 
-import junit.framework.TestCase;
-
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.DataSerializer;
@@ -42,36 +45,21 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * @since 3.0
  */
 @Category(UnitTest.class)
-public class QueryObjectSerializationJUnitTest extends TestCase implements Serializable {
+public class QueryObjectSerializationJUnitTest implements Serializable {
   
   /** A <code>ByteArrayOutputStream</code> that data is serialized to */
   private transient ByteArrayOutputStream baos;
 
-  public QueryObjectSerializationJUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Helper Class
-  public static class SimpleObjectType implements ObjectType {
-   public SimpleObjectType() {}
-   public boolean isCollectionType() { return false; }
-   public boolean isMapType() { return false; }
-   public boolean isStructType() { return false; }
-   public String getSimpleClassName() { return "java.lang.Object"; }
-   public Class resolveClass() { return Object.class; }
-   public void toData(DataOutput out) {}
-   public void fromData(DataInput in) {}
-   public boolean equals(Object o) { return o instanceof SimpleObjectType; }
-  }
-
   /**
    * Creates a new <code>ByteArrayOutputStream</code> for this test to
    * work with.
    */
+  @Before
   public void setUp() {
     this.baos = new ByteArrayOutputStream();
   }
 
+  @After
   public void tearDown() {
     this.baos = null;
   }
@@ -79,14 +67,14 @@ public class QueryObjectSerializationJUnitTest extends TestCase implements Seria
   /**
    * Returns a <code>DataOutput</code> to write to
    */
-  protected DataOutputStream getDataOutput() {
+  private DataOutputStream getDataOutput() {
     return new DataOutputStream(this.baos);
   }
 
   /**
    * Returns a <code>DataInput</code> to read from
    */
-  protected DataInputStream getDataInput() {
+  private DataInputStream getDataInput() {
     ByteArrayInputStream bais = new ByteArrayInputStream(this.baos.toByteArray());
     return new DataInputStream(bais);
   }
@@ -96,7 +84,6 @@ public class QueryObjectSerializationJUnitTest extends TestCase implements Seria
    * asserts that the two objects satisfy o1.equals(o2)
    */
   private void checkRoundTrip(Object o1) throws IOException, ClassNotFoundException {
-
     DataOutputStream out = getDataOutput();
     DataSerializer.writeObject(o1, out);
     out.flush();
@@ -108,6 +95,7 @@ public class QueryObjectSerializationJUnitTest extends TestCase implements Seria
   /**
    * Tests the serialization of many, but not all of the possible ResultSets
    */
+  @Test
   public void testSerializationOfQueryResults() throws IOException, ClassNotFoundException {
     Collection data = new java.util.ArrayList();
     data.add(null);
@@ -147,4 +135,16 @@ public class QueryObjectSerializationJUnitTest extends TestCase implements Seria
     //SortedStructSet sssWithoutData = new SortedStructSet();
     //checkRoundTrip(sssWithoutData); 
   }
+
+  private static class SimpleObjectType implements ObjectType {
+    public SimpleObjectType() {}
+    public boolean isCollectionType() { return false; }
+    public boolean isMapType() { return false; }
+    public boolean isStructType() { return false; }
+    public String getSimpleClassName() { return "java.lang.Object"; }
+    public Class resolveClass() { return Object.class; }
+    public void toData(DataOutput out) {}
+    public void fromData(DataInput in) {}
+    public boolean equals(Object o) { return o instanceof SimpleObjectType; }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagJUnitTest.java
index 181bb82..865651c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagJUnitTest.java
@@ -16,30 +16,30 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import java.util.*;
-import java.io.*;
+import static org.junit.Assert.*;
 
-import org.junit.experimental.categories.Category;
+import java.io.DataInputStream;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
 
-import junit.framework.*;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.internal.types.ObjectTypeImpl;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Test ResultsBag, including null elements
  */
 @Category(UnitTest.class)
-public class ResultsBagJUnitTest extends TestCase {
+public class ResultsBagJUnitTest {
 
-  public ResultsBagJUnitTest(String testName) {
-    super(testName);
-  }
-  
+  @Test
   public void testDuplicates() {
     ResultsBag bag = new ResultsBag();
     bag.add("one");
@@ -56,9 +56,9 @@ public class ResultsBagJUnitTest extends TestCase {
     assertEquals(0, bag.occurrences("one"));
     assertTrue(!bag.remove("one"));
     assertEquals(0, bag.occurrences("one"));
-    
   }
-  
+
+  @Test
   public void testIteration() {
     ResultsBag bag = new ResultsBag();
     bag.add(new Integer(1));
@@ -84,9 +84,9 @@ public class ResultsBagJUnitTest extends TestCase {
     assertEquals(1, numOnes);
     assertEquals(2, numTwos);
   }
-  
-  public void testSerializingSetViewWithNulls()
-  throws ClassNotFoundException, IOException {
+
+  @Test
+  public void testSerializingSetViewWithNulls() throws Exception {
     ResultsBag bag = new ResultsBag();
     bag.add(new Integer(4));
     bag.add(new Integer(2));
@@ -117,7 +117,8 @@ public class ResultsBagJUnitTest extends TestCase {
     assertTrue(setCopy.contains(new Integer(4)));
     assertTrue(setCopy.contains(null));
   }
-    
+
+  @Test
   public void testNulls() {
     ResultsBag bag = new ResultsBag();
     assertTrue(bag.isEmpty());
@@ -172,7 +173,8 @@ public class ResultsBagJUnitTest extends TestCase {
     assertTrue(!bag.remove(null));
     assertEquals(0, bag.occurrences(null));
   }
-  
+
+  @Test
   public void testIterationNullRemoval() {
     ResultsBag bag = new ResultsBag();
     bag.add(null);
@@ -192,7 +194,8 @@ public class ResultsBagJUnitTest extends TestCase {
     assertEquals(3, bag.size());
     assertEquals(0, bag.occurrences(null));
   }
-  
+
+  @Test
   public void testIterationRemoval() {
     ResultsBag bag = new ResultsBag();
     
@@ -205,8 +208,6 @@ public class ResultsBagJUnitTest extends TestCase {
     
     assertEquals(6, bag.size());
     
-//    Integer one = new Integer(1);
-//    Integer two = new Integer(2);
     Iterator itr = bag.iterator();
     for (int i = 0 ; i < 3; i++) {
       itr.next();
@@ -221,7 +222,8 @@ public class ResultsBagJUnitTest extends TestCase {
     assertTrue(bag.isEmpty());
     assertEquals(0, bag.size());
   }
-  
+
+  @Test
   public void testNoSuchElementException() {
     ResultsBag bag = new ResultsBag();
     
@@ -242,7 +244,7 @@ public class ResultsBagJUnitTest extends TestCase {
       itr.next();
       fail("should have thrown a NoSuchElementException");
     }
-    catch (NoSuchElementException e) {
+    catch (NoSuchElementException expected) {
       // pass
     }
     
@@ -257,7 +259,7 @@ public class ResultsBagJUnitTest extends TestCase {
       itr.next();
       fail("should have thrown a NoSuchElementException");
     }
-    catch (NoSuchElementException e) {
+    catch (NoSuchElementException expected) {
       // pass
     }
     
@@ -284,7 +286,7 @@ public class ResultsBagJUnitTest extends TestCase {
       itr.next();
       fail("should have thrown a NoSuchElementException");
     }
-    catch (NoSuchElementException e) {
+    catch (NoSuchElementException expected) {
       // pass
     }
     
@@ -299,7 +301,7 @@ public class ResultsBagJUnitTest extends TestCase {
       itr.next();
       fail("should have thrown a NoSuchElementException");
     }
-    catch (NoSuchElementException e) {
+    catch (NoSuchElementException expected) {
       // pass
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagLimitBehaviourJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagLimitBehaviourJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagLimitBehaviourJUnitTest.java
index aacb00e..e92797d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagLimitBehaviourJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsBagLimitBehaviourJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -26,8 +28,7 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-import junit.framework.TestCase;
-
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.query.Struct;
@@ -36,19 +37,15 @@ import com.gemstone.gemfire.cache.query.types.ObjectType;
 import com.gemstone.gemfire.cache.query.types.StructType;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-// TODO:Asif: Test for null behaviour in various functions
-
 /**
  * Test ResultsBag Limit behaviour
- * 
+ *
+ * TODO: Test for null behaviour in various functions
  */
 @Category(UnitTest.class)
-public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
-
-  public ResultsBagLimitBehaviourJUnitTest(String testName) {
-    super(testName);
-  }
+public class ResultsBagLimitBehaviourJUnitTest {
 
+  @Test
   public void testAsListAndAsSetMethod() {
     ResultsBag bag = getBagObject(String.class);
     bag.add(wrap("one", bag.getCollectionType().getElementType()));
@@ -63,6 +60,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertEquals(2, set.size());
   }
 
+  @Test
   public void testOccurence() {
     ResultsBag bag = getBagObject(String.class);
     bag.add(wrap(null, bag.getCollectionType().getElementType()));
@@ -87,6 +85,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertEquals(6, total);
   }
 
+  @Test
   public void testIteratorType() {
     ResultsBag bag = getBagObject(String.class);
     bag.add(wrap("one", bag.getCollectionType().getElementType()));
@@ -114,6 +113,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testContains() {
     ResultsBag bag = getBagObject(Integer.class);
     bag.add(wrap(new Integer(1), bag.getCollectionType().getElementType()));
@@ -144,6 +144,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertTrue(temp.contains(wrap(null, bag.getCollectionType().getElementType())) == bag.contains(wrap(null, bag.getCollectionType().getElementType())));
   }
 
+  @Test
   public void testAddExceptionIfLimitApplied() {
     ResultsBag bag = getBagObject(String.class);
     bag.add(wrap("one", bag.getCollectionType().getElementType()));
@@ -161,6 +162,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
 
   // Internal method AddAndGetOccurence used for iter evaluating
   // only up till the limit
+  @Test
   public void testAddAndGetOccurence() {
     ResultsBag bag = getBagObject(String.class);
     bag = getBagObject(String.class);
@@ -176,6 +178,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertEquals(3, bag.addAndGetOccurence(elementType instanceof StructType ? ((Struct)wrap(null, elementType)).getFieldValues() : wrap(null, elementType)));
   }
 
+  @Test
   public void testSizeWithLimitApplied() {
     ResultsBag bag = getBagObject(String.class);
     bag.add(wrap("two", bag.getCollectionType().getElementType()));
@@ -194,6 +197,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertEquals(2, bag.size());
   }
 
+  @Test
   public void testRemove() {
     // Test when actual size in resultset is less than the limit
     ResultsBag bag = getBagObject(String.class);
@@ -280,6 +284,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
 
   }
 
+  @Test
   public void testAddAllExceptionIfLimitApplied() {
     ResultsBag bag = getBagObject(Object.class);
     bag.applyLimit(6);
@@ -292,6 +297,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testToDataFromData() throws Exception {
     // Test with limit specified & limit less than internal size
     ResultsBag toBag = getBagObject(String.class);
@@ -329,6 +335,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertFalse(toBag.asList().retainAll(fromBag.asList()));
   }
 
+  @Test
   public void testLimitResultsBagIterator_1() {
     ResultsBag bag = getBagObject(Integer.class);
     bag.add(wrap(null, bag.getCollectionType().getElementType()));
@@ -380,6 +387,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testLimitResultsBagIterator_2() {
     ResultsBag bag = getBagObject(Object.class);
     for (int i = 0; i < 20; ++i) {
@@ -420,6 +428,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertEquals(i, 3);
   }
 
+  @Test
   public void testValidExceptionThrown() {
     ResultsBag bag = getBagObject(Integer.class);
     bag.add(wrap(new Integer(1), bag.getCollectionType().getElementType()));
@@ -466,6 +475,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testRemoveAll() {
     ResultsBag bag = getBagObject(Integer.class);
     // Add Integer & null Objects
@@ -502,6 +512,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testRetainAll() {
     ResultsBag bag = getBagObject(Integer.class);
     // Add Integer & null Objects
@@ -540,6 +551,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testContainAll() {
     ResultsBag bag = getBagObject(Integer.class);
     // Add Integer & null Objects
@@ -557,7 +569,7 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     // asList.add(wrap(new
     // Integer(13),bag.getCollectionType().getElementType()));
     assertEquals(4, bag.size());
-    // assertEquals(5,asList.size());
+    // assertIndexDetailsEquals(5,asList.size());
     // Remove all the elements from the list which match the
     // first element pf the list
     int occurence = bag.occurrences(asList.get(0));
@@ -572,12 +584,12 @@ public class ResultsBagLimitBehaviourJUnitTest extends TestCase {
     assertFalse(bag.containsAll(asList));
   }
 
-  public ResultsBag getBagObject(Class clazz) {
+  private ResultsBag getBagObject(Class clazz) {
     ObjectType type = new ObjectTypeImpl(clazz);
     return new ResultsBag(type, null);
   }
 
-  public Object wrap(Object obj, ObjectType elementType) {
+  private Object wrap(Object obj, ObjectType elementType) {
     return obj;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsCollectionWrapperLimitJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsCollectionWrapperLimitJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsCollectionWrapperLimitJUnitTest.java
index b24da54..04c88b6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsCollectionWrapperLimitJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/ResultsCollectionWrapperLimitJUnitTest.java
@@ -19,8 +19,7 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import com.gemstone.gemfire.cache.query.internal.types.ObjectTypeImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -31,20 +30,16 @@ import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
+import com.gemstone.gemfire.cache.query.internal.types.ObjectTypeImpl;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * 
- */
 @Category(UnitTest.class)
-public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
-
-  public ResultsCollectionWrapperLimitJUnitTest(String testName) {
-    super(testName);
-  }
+public class ResultsCollectionWrapperLimitJUnitTest {
 
+  @Test
   public void testConstructorBehaviour() {
     // Create a Collection of unordered data elements
     HashSet unordered = new HashSet();
@@ -97,6 +92,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testContains() {
     // Create a Collection of unordered data elements
     HashSet unordered = new HashSet();
@@ -120,6 +116,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertTrue(wrapper.contains(new Integer(1)));
   }
 
+  @Test
   public void testContainsAll() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -135,6 +132,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertTrue(wrapper.containsAll(newList));
   }
 
+  @Test
   public void testEmpty() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -145,6 +143,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertEquals(0, wrapper.size());
   }
 
+  @Test
   public void testRemove() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -163,6 +162,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertTrue(wrapper.containsAll(newList));
   }
 
+  @Test
   public void testRemoveAll() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -196,6 +196,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testRetainAll() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -223,6 +224,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testToArray() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -239,6 +241,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testToArrayParameterized() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -256,6 +259,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testAsList() {
     List ordered = new ArrayList();
     for (int i = 1; i < 11; ++i) {
@@ -286,6 +290,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertTrue(newList.containsAll(toList));
   }
 
+  @Test
   public void testAsSet() {
     Collection ordered = new TreeSet();
     for (int i = 1; i < 11; ++i) {
@@ -315,6 +320,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     assertTrue(newSet.containsAll(toSet));
   }
 
+  @Test
   public void testOccurences() {
     Collection ordered = new TreeSet();
     for (int i = 1; i < 11; ++i) {
@@ -338,6 +344,7 @@ public class ResultsCollectionWrapperLimitJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testLimitIterator() {
     Collection ordered = new TreeSet();
     for (int i = 1; i < 11; ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/SelectResultsComparatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/SelectResultsComparatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/SelectResultsComparatorJUnitTest.java
index d8c2913..0ff6a15 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/SelectResultsComparatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/SelectResultsComparatorJUnitTest.java
@@ -19,34 +19,26 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
+import static org.junit.Assert.*;
+
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Random;
-import java.util.Collections;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class SelectResultsComparatorJUnitTest extends TestCase implements OQLLexerTokenTypes {
-  
-  public SelectResultsComparatorJUnitTest(String testName) {
-    super(testName);
-  }
-  
+public class SelectResultsComparatorJUnitTest implements OQLLexerTokenTypes {
+
+  @Test
   public void testComparatorForSortedSet() throws Exception {
-//    CompiledValue[] operands = new CompiledValue[3];
-//    
-//    CompiledJunction cj = new CompiledJunction(operands, LITERAL_and);
     int sameSizeVar = 0;
     boolean sameSizeVarSetFlag = false;
     SortedSet testSet = 
@@ -85,6 +77,4 @@ public class SelectResultsComparatorJUnitTest extends TestCase implements OQLLex
         fail("This is not expected behaviour");
     }
   }
-  
-    
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructBagLimitBehaviourJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructBagLimitBehaviourJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructBagLimitBehaviourJUnitTest.java
index ea4f5f7..f32f11b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructBagLimitBehaviourJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructBagLimitBehaviourJUnitTest.java
@@ -19,27 +19,26 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import com.gemstone.gemfire.cache.query.Struct;
-import com.gemstone.gemfire.cache.query.types.ObjectType;
-import com.gemstone.gemfire.cache.query.types.StructType;
-import com.gemstone.gemfire.cache.query.internal.types.*;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.junit.Assert.*;
 
 import java.util.Iterator;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.cache.query.Struct;
+import com.gemstone.gemfire.cache.query.internal.types.ObjectTypeImpl;
+import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl;
+import com.gemstone.gemfire.cache.query.types.ObjectType;
+import com.gemstone.gemfire.cache.query.types.StructType;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
- * Test StructsBag Limit behaviour
- * 
+ * Test StructBag Limit behaviour
  */
 @Category(UnitTest.class)
 public class StructBagLimitBehaviourJUnitTest extends ResultsBagLimitBehaviourJUnitTest {
 
-  public StructBagLimitBehaviourJUnitTest(String testName) {
-    super(testName);
-  }
-
   public ResultsBag getBagObject(Class clazz) {
     ObjectType[] types = new ObjectType[] { new ObjectTypeImpl(clazz),
         new ObjectTypeImpl(clazz) };
@@ -58,6 +57,7 @@ public class StructBagLimitBehaviourJUnitTest extends ResultsBagLimitBehaviourJU
     }
   }
 
+  @Test
   public void testRemoveAllStructBagSpecificMthod() {
     StructBag bag1 = (StructBag)getBagObject(Integer.class);
     // Add Integer & null Objects
@@ -91,6 +91,7 @@ public class StructBagLimitBehaviourJUnitTest extends ResultsBagLimitBehaviourJU
     assertFalse(itr.hasNext());
   }
 
+  @Test
   public void testRetainAllStructBagSpecific() {
     StructBag bag1 = (StructBag)getBagObject(Integer.class);
     // Add Integer & null Objects

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructSetJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructSetJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructSetJUnitTest.java
index fd66df8..a1c9703 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructSetJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/StructSetJUnitTest.java
@@ -22,10 +22,11 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
-import java.util.Iterator;
+import static org.junit.Assert.*;
 
-import junit.framework.TestCase;
+import java.util.Iterator;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.query.SelectResults;
@@ -35,16 +36,10 @@ import com.gemstone.gemfire.cache.query.internal.types.TypeUtils;
 import com.gemstone.gemfire.cache.query.types.ObjectType;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
-public class StructSetJUnitTest extends TestCase {
-  
-  public StructSetJUnitTest(String testName) {
-    super(testName);
-  }
+public class StructSetJUnitTest {
   
+  @Test
   public void testIntersectionAndRetainAll() {
     String names[] = {"p","pos"};
     ObjectType types[] = {TypeUtils.OBJECT_TYPE, TypeUtils.OBJECT_TYPE};

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/aggregate/AggregatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/aggregate/AggregatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/aggregate/AggregatorJUnitTest.java
index 9645b4e..9eeba95 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/aggregate/AggregatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/aggregate/AggregatorJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.query.internal.aggregate;
 
+import static org.junit.Assert.*;
+
 import java.util.HashSet;
 import java.util.Set;
 
@@ -24,14 +26,8 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- * 
- *
- */
 @Category(UnitTest.class)
-public class AggregatorJUnitTest extends TestCase{
+public class AggregatorJUnitTest {
 
   @Test
   public void testCount() throws Exception {
@@ -125,9 +121,8 @@ public class AggregatorJUnitTest extends TestCase{
     avg.accumulate(null);
     avg.accumulate(null);
     float expected = (1 + 2+ 3 + 4 + 5 + 6 + 7 +7)/8.0f ;    
-    assertEquals(expected, ((Number)avg.terminate()).floatValue());
+    assertEquals(expected, ((Number)avg.terminate()).floatValue(), 0);
     
-   
     AvgBucketNode abn = new AvgBucketNode();
     abn.accumulate(new Integer(1));
     abn.accumulate(new Integer(2));
@@ -143,14 +138,13 @@ public class AggregatorJUnitTest extends TestCase{
     assertEquals(8, ((Integer)arr[0]).intValue());
     assertEquals(35, ((Number)arr[1]).intValue());
     
-    
     AvgPRQueryNode apqn = new AvgPRQueryNode();
     Object[] val1 = new Object[]{new Integer(7), new Double(43)};
     Object[] val2 = new Object[]{new Integer(5), new Double(273.86)};
     apqn.accumulate(val1);
     apqn.accumulate(val2);
     expected = (43+273.86f)/12.0f ;
-    assertEquals(expected, ((Number)apqn.terminate()).floatValue());    
+    assertEquals(expected, ((Number)apqn.terminate()).floatValue(), 0);
   }
   
   @Test
@@ -170,9 +164,8 @@ public class AggregatorJUnitTest extends TestCase{
     avg.accumulate(null);
     avg.accumulate(null);
     float expected = (1 + 2+ 3 + 4 + 5 + 6 + 7)/7.0f ;    
-    assertEquals(expected, ((Number)avg.terminate()).floatValue());
-    
-   
+    assertEquals(expected, ((Number)avg.terminate()).floatValue(), 0);
+
     AvgDistinctPRQueryNode adpqn = new AvgDistinctPRQueryNode();
     
     Set<Integer> set1 = new HashSet<Integer>();
@@ -191,7 +184,7 @@ public class AggregatorJUnitTest extends TestCase{
     adpqn.accumulate(set2);
    
     expected = (3+4+5+6+7+8)/6.0f ;
-    assertEquals(expected, ((Number)adpqn.terminate()).floatValue());    
+    assertEquals(expected, ((Number)adpqn.terminate()).floatValue(), 0);
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/AsynchIndexMaintenanceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/AsynchIndexMaintenanceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/AsynchIndexMaintenanceJUnitTest.java
index 18bc54c..cb32555 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/AsynchIndexMaintenanceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/AsynchIndexMaintenanceJUnitTest.java
@@ -103,7 +103,7 @@ public class AsynchIndexMaintenanceJUnitTest {
       region.put(""+(i+1), new Portfolio(i+1));
       idSet.add((i+1) + "");
     }    
-    //assertEquals(0, getIndexSize(ri));
+    //assertIndexDetailsEquals(0, getIndexSize(ri));
     region.put("50", new Portfolio(50));
     WaitCriterion ev = new WaitCriterion() {
       public boolean done() {
@@ -129,7 +129,7 @@ public class AsynchIndexMaintenanceJUnitTest {
       idSet.add((i+1) + "");
     }    
 
-    //assertEquals(0, getIndexSize(ri));
+    //assertIndexDetailsEquals(0, getIndexSize(ri));
 
     WaitCriterion evSize = new WaitCriterion() {
       public boolean done() {
@@ -160,7 +160,7 @@ public class AsynchIndexMaintenanceJUnitTest {
       region.put(""+(i+1), new Portfolio(i+1));
       idSet.add((i+1) + "");
     }    
-    //assertEquals(0, getIndexSize(ri));
+    //assertIndexDetailsEquals(0, getIndexSize(ri));
     Wait.waitForCriterion(evSize, 17 * 1000, 200, true);
   }
   
@@ -200,7 +200,7 @@ public class AsynchIndexMaintenanceJUnitTest {
       idSet.add((i+1) + "");
     }    
     Thread.sleep(10000);
-    //assertEquals(0, this.getIndexSize(ri));    
+    //assertIndexDetailsEquals(0, this.getIndexSize(ri));
         
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
index de5cebd..7f96fde 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
@@ -14,15 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package com.gemstone.gemfire.cache.query.internal.index;
 
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -32,7 +29,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.IntStream;
 
-import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.invocation.InvocationOnMock;
@@ -49,7 +45,7 @@ public class HashIndexSetJUnitTest {
   Set<Portfolio> portfolioSet;
   HashIndexSet his;
   
-  public void setupHashIndexSet(int numEntries) {
+  private void setupHashIndexSet(int numEntries) {
     his = createHashIndexSet();
     portfoliosMap = createPortfolioObjects(numEntries, 0);
     portfolioSet = new HashSet<Portfolio>(portfoliosMap.values());
@@ -81,9 +77,8 @@ public class HashIndexSetJUnitTest {
    * IDs are startID -> startID + numEntries
    * @param numToCreate how many portfolios to create
    * @param startID the ID value to start incrementing from
-   * @return
    */
-  public Map<Integer, Portfolio> createPortfolioObjects(int numToCreate, int startID) {
+  private Map<Integer, Portfolio> createPortfolioObjects(int numToCreate, int startID) {
     Map<Integer, Portfolio> portfoliosMap = new HashMap<>();
     IntStream.range(0, numToCreate).forEach(e -> {
         Portfolio p = new Portfolio(e + startID);
@@ -98,9 +93,9 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.iterator().forEachRemaining((e ->portfolioSet.remove(e)));
-    Assert.assertTrue(portfolioSet.isEmpty());  
+    assertTrue(portfolioSet.isEmpty());  
   }
   
   @Test
@@ -108,26 +103,28 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.add(null, new Portfolio(numEntries + 1));
-    Assert.assertEquals(numEntries + 1, his.size());
+    assertEquals(numEntries + 1, his.size());
   }
-  
-  //we have to be sure that we dont cause a compaction or growth or else
-  //removed tokens will be removed and a new backing array created
+
+  /**
+   * we have to be sure that we dont cause a compaction or growth or else
+   * removed tokens will be removed and a new backing array created
+   */
   @Test
   public void testHashIndexSetAddUseRemoveTokenSlot() throws Exception {
     int numEntries = 20;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.removeAll(portfolioSet);
-    Assert.assertEquals(numEntries, his._removedTokens);
-    Assert.assertEquals(0, his.size());
+    assertEquals(numEntries, his._removedTokens);
+    assertEquals(0, his.size());
     addPortfoliosToHashIndexSet(portfoliosMap, his);
     
-    Assert.assertEquals(0, his._removedTokens);
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(0, his._removedTokens);
+    assertEquals(numEntries, his.size());
   }
   
   @Test
@@ -135,19 +132,19 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;    
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.removeAll(portfolioSet);
-    Assert.assertEquals(numEntries, his._removedTokens);
+    assertEquals(numEntries, his._removedTokens);
     
-    Assert.assertEquals(0, his.size());
+    assertEquals(0, his.size());
     
     //Very very bad but we fake out the number of removed tokens
     his._removedTokens = his._maxSize;
     addPortfoliosToHashIndexSet(portfoliosMap, his);
     
     //compaction should have occured, removed tokens should now be gone
-    Assert.assertEquals(0, his._removedTokens);
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(0, his._removedTokens);
+    assertEquals(numEntries, his.size());
   }
   
   @Test
@@ -155,11 +152,11 @@ public class HashIndexSetJUnitTest {
     int numEntries = 80;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.rehash(1000);
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.iterator().forEachRemaining((e ->portfolioSet.remove(e)));
-    Assert.assertTrue(portfolioSet.isEmpty());  
+    assertTrue(portfolioSet.isEmpty());  
   }
   
   @Test
@@ -167,11 +164,11 @@ public class HashIndexSetJUnitTest {
     int numEntries = 20;
     setupHashIndexSet(numEntries);
 
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.rehash(64);
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.iterator().forEachRemaining((e ->portfolioSet.remove(e)));
-    Assert.assertTrue(portfolioSet.isEmpty());  
+    assertTrue(portfolioSet.isEmpty());  
   }
   
   @Test
@@ -179,9 +176,9 @@ public class HashIndexSetJUnitTest {
     int numEntries = 20;
     setupHashIndexSet(numEntries);
 
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.get(1).forEachRemaining((e ->portfolioSet.remove(e)));
-    Assert.assertEquals(numEntries - 1, portfolioSet.size());  
+    assertEquals(numEntries - 1, portfolioSet.size());  
   }
   
   @Test
@@ -195,15 +192,15 @@ public class HashIndexSetJUnitTest {
     addPortfoliosToHashIndexSet(collectionOfPorts1, his);
     addPortfoliosToHashIndexSet(collectionOfPorts2, his);
     
-    Assert.assertEquals(numEntries * 2, his.size());
+    assertEquals(numEntries * 2, his.size());
     Iterator iterator = his.get(keyToLookup);
     int numIterated = 0;
     while (iterator.hasNext()) {
       numIterated ++;
       //verify that the returned values match what we lookedup
-      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+      assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
     }
-    Assert.assertEquals(2, numIterated);  
+    assertEquals(2, numIterated);  
   }
   
   @Test
@@ -219,15 +216,15 @@ public class HashIndexSetJUnitTest {
     addPortfoliosToHashIndexSet(collectionOfPorts2, his);
     addPortfoliosToHashIndexSet(collectionOfPorts3, his);
     
-    Assert.assertEquals(numEntries * 3, his.size());
+    assertEquals(numEntries * 3, his.size());
     Iterator iterator = his.get(keyToLookup);
     int numIterated = 0;
     while (iterator.hasNext()) {
       numIterated ++;
       //verify that the returned values match what we lookedup
-      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+      assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
     }
-    Assert.assertEquals(3, numIterated);  
+    assertEquals(3, numIterated);  
     
     //let's remove the second collision
     his.remove(keyToLookup, collectionOfPorts2.get(keyToLookup));
@@ -237,9 +234,9 @@ public class HashIndexSetJUnitTest {
     while (iterator.hasNext()) {
       numIterated ++;
       //verify that the returned values match what we lookedup
-      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+      assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
     }
-    Assert.assertEquals(2, numIterated);  
+    assertEquals(2, numIterated);  
     
     //Add it back in and make sure we can iterate all 3 again
     his.add(keyToLookup, collectionOfPorts2.get(keyToLookup));
@@ -248,9 +245,9 @@ public class HashIndexSetJUnitTest {
     while (iterator.hasNext()) {
       numIterated ++;
       //verify that the returned values match what we lookedup
-      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+      assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
     }
-    Assert.assertEquals(3, numIterated);  
+    assertEquals(3, numIterated);  
 
   }
   
@@ -264,7 +261,7 @@ public class HashIndexSetJUnitTest {
     addPortfoliosToHashIndexSet(collectionOfPorts1, his);
     addPortfoliosToHashIndexSet(collectionOfPorts2, his);
     
-    Assert.assertEquals(numEntries * 2, his.size());
+    assertEquals(numEntries * 2, his.size());
     List<Integer> keysNotToMatch = new LinkedList<>();
     keysNotToMatch.add(3);
     keysNotToMatch.add(4);
@@ -273,10 +270,10 @@ public class HashIndexSetJUnitTest {
     while (iterator.hasNext()) {
       numIterated ++;
       int idFound = ((Portfolio)iterator.next()).indexKey;
-      Assert.assertTrue(idFound != 3 && idFound != 4);
+      assertTrue(idFound != 3 && idFound != 4);
     }
     //Make sure we iterated all the entries minus the entries that we decided not to match
-    Assert.assertEquals(numEntries * 2 - 4, numIterated);  
+    assertEquals(numEntries * 2 - 4, numIterated);  
   }
   
   @Test
@@ -288,16 +285,18 @@ public class HashIndexSetJUnitTest {
       try {
         int index = his.add(k, portfoliosMap.get(k));
         int foundIndex = his.index(portfoliosMap.get(k));
-        Assert.assertEquals(index, foundIndex);
+        assertEquals(index, foundIndex);
       }
       catch (TypeMismatchException ex) {
         throw new Error(ex);
       }
     });
   }
-  
-  //Add multiple portfolios with the same id
-  //they should collide, we should then be able to look up each one correctly
+
+  /**
+   * Add multiple portfolios with the same id
+   * they should collide, we should then be able to look up each one correctly
+   */
   @Test
   public void testIndexOfObjectWithCollision() throws Exception {
     int numEntries = 10;
@@ -309,7 +308,7 @@ public class HashIndexSetJUnitTest {
       try {
         int index = his.add(k, portfoliosMap1.get(k));
         int foundIndex = his.index(portfoliosMap1.get(k));
-        Assert.assertEquals(index, foundIndex);
+        assertEquals(index, foundIndex);
       }
       catch (TypeMismatchException ex) {
         throw new Error(ex);
@@ -319,65 +318,65 @@ public class HashIndexSetJUnitTest {
       try {
         int index = his.add(k, portfoliosMap2.get(k));
         int foundIndex = his.index(portfoliosMap2.get(k));
-        Assert.assertEquals(index, foundIndex);
+        assertEquals(index, foundIndex);
       }
       catch (TypeMismatchException ex) {
         throw new Error(ex);
       }
     });
   }
-  
-  
+
   @Test
   public void testIndexWhenObjectNotInSet() {
     int numEntries = 10;
     his = createHashIndexSet();
     portfoliosMap = createPortfolioObjects(numEntries, 0);
-    Assert.assertEquals(-1, his.index(portfoliosMap.get(1)));
+    assertEquals(-1, his.index(portfoliosMap.get(1)));
   }
   
   @Test
   public void testIndexWhenObjectNotInSetWhenPopulated() {
     int numEntries = 10;
     this.setupHashIndexSet(numEntries);
-    Assert.assertEquals(-1, his.index(new Portfolio(numEntries+1)));
+    assertEquals(-1, his.index(new Portfolio(numEntries+1)));
   }
-  
-  
+
   @Test
   public void testRemove() throws Exception {
     int numEntries = 20;
     setupHashIndexSet(numEntries);
 
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     portfoliosMap.forEach((k,v) -> his.remove(k, v));
-    Assert.assertEquals(0, his.size());
+    assertEquals(0, his.size());
   }
-  
-  //Test remove where we look for an instance that is not at the specified index slot
+
+  /**
+   * Test remove where we look for an instance that is not at the specified index slot
+   */
   @Test
   public void testRemoveIgnoreSlot() throws Exception {
     int numEntries = 20;
     setupHashIndexSet(numEntries);
 
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     portfoliosMap.forEach((k,v) -> his.remove(k, v, his.index(v)));
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
   }
   
   @Test
   public void testRemoveAtWithNull() throws Exception {
     his = createHashIndexSet();
-    Assert.assertTrue(his.isEmpty());
-    Assert.assertFalse(his.removeAt(0));
+    assertTrue(his.isEmpty());
+    assertFalse(his.removeAt(0));
   }
   
   @Test
   public void testRemoveAtWithRemoveToken() throws Exception {
     his = createHashIndexSet();
     int index = his.add(1, new Portfolio(1));
-    Assert.assertTrue(his.removeAt(index));
-    Assert.assertFalse(his.removeAt(index));
+    assertTrue(his.removeAt(index));
+    assertFalse(his.removeAt(index));
   }
   
   @Test
@@ -385,21 +384,23 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.removeAll(portfolioSet);
-    Assert.assertTrue(his.isEmpty());  
+    assertTrue(his.isEmpty());  
   }
-  
-  //Remove all should still remove all portfolios provided, even if there are more provided then contained
+
+  /**
+   * Remove all should still remove all portfolios provided, even if there are more provided then contained
+   */
   @Test
   public void testHashIndexRemoveAllWithAdditionalPortfolios() throws Exception {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     portfolioSet.add(new Portfolio(numEntries + 1));
     his.removeAll(portfolioSet);
-    Assert.assertTrue(his.isEmpty());  
+    assertTrue(his.isEmpty());  
   }
   
   @Test
@@ -407,8 +408,8 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
-    Assert.assertTrue(his.containsAll(portfolioSet));
+    assertEquals(numEntries, his.size());
+    assertTrue(his.containsAll(portfolioSet));
   }
   
   @Test
@@ -417,11 +418,11 @@ public class HashIndexSetJUnitTest {
     setupHashIndexSet(numEntries);
     Set subset = new HashSet();
     portfolioSet.forEach(e -> {if (e.indexKey % 2 == 0) {subset.add(e);}});
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.retainAll(subset);
     his.iterator().forEachRemaining((e ->subset.remove(e)));
-    Assert.assertTrue(subset.isEmpty()); 
-    Assert.assertEquals(numEntries/2, his.size());
+    assertTrue(subset.isEmpty()); 
+    assertEquals(numEntries/2, his.size());
   }
   
   @Test
@@ -429,9 +430,9 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     portfolioSet.add(new Portfolio(numEntries + 1));
-    Assert.assertFalse(his.containsAll(portfolioSet));
+    assertFalse(his.containsAll(portfolioSet));
   }
   
   @Test
@@ -439,16 +440,16 @@ public class HashIndexSetJUnitTest {
     int numEntries = 100;
     setupHashIndexSet(numEntries);
     
-    Assert.assertEquals(numEntries, his.size());
+    assertEquals(numEntries, his.size());
     his.clear();
-    Assert.assertTrue(his.isEmpty());
-    Assert.assertTrue(his._removedTokens == 0);
+    assertTrue(his.isEmpty());
+    assertTrue(his._removedTokens == 0);
   }
   
   @Test
   public void testAreNullObjectsEqual() throws Exception {
     his = createHashIndexSet();
-    Assert.assertTrue(his.areObjectsEqual(null, null));
+    assertTrue(his.areObjectsEqual(null, null));
   }
   
   @Test
@@ -460,9 +461,9 @@ public class HashIndexSetJUnitTest {
     addPortfoliosToHashIndexSet(portfolioMap, indexSet1);
     addPortfoliosToHashIndexSet(portfolioMap, indexSet2);
  
-    Assert.assertTrue(indexSet1.equals(indexSet2));
-    Assert.assertTrue(indexSet2.equals(indexSet1));
-    Assert.assertEquals(indexSet1.hashCode(), indexSet2.hashCode());
+    assertTrue(indexSet1.equals(indexSet2));
+    assertTrue(indexSet2.equals(indexSet1));
+    assertEquals(indexSet1.hashCode(), indexSet2.hashCode());
   }
   
   @Test
@@ -474,16 +475,16 @@ public class HashIndexSetJUnitTest {
     addPortfoliosToHashIndexSet(portfolioMap, indexSet1);
 
     indexSet2.add(1, portfolioMap.get(1));
-    Assert.assertFalse(indexSet2.equals(indexSet1));
-    Assert.assertFalse(indexSet1.equals(indexSet2));
-    Assert.assertNotEquals(indexSet1.hashCode(), indexSet2.hashCode()); 
+    assertFalse(indexSet2.equals(indexSet1));
+    assertFalse(indexSet1.equals(indexSet2));
+    assertNotEquals(indexSet1.hashCode(), indexSet2.hashCode());
   }
   
   @Test
   public void testIndexSetNotEqualsOtherObjectType() {
     HashIndexSet indexSet = createHashIndexSet();
-    Assert.assertFalse(indexSet.equals("Other type"));
-    Assert.assertFalse(indexSet.equals(new Object()));
+    assertFalse(indexSet.equals("Other type"));
+    assertFalse(indexSet.equals(new Object()));
   }
  
   private static class EvaluateKeyAnswer implements Answer {
@@ -499,6 +500,5 @@ public class HashIndexSetJUnitTest {
     }
     
   }
-  
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexElemArrayJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexElemArrayJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexElemArrayJUnitTest.java
index 91dbf7b..1ac115a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexElemArrayJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexElemArrayJUnitTest.java
@@ -16,11 +16,7 @@
  */
 package com.gemstone.gemfire.cache.query.internal.index;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -30,7 +26,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.stream.IntStream;
 
-import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -40,16 +36,15 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class IndexElemArrayJUnitTest {
   
-  private IndexElemArray list = new IndexElemArray(7);
+  private IndexElemArray list;
 
-  @After
-  public void tearDown() {
-    //System.clearProperty("index_elemarray_size");
+  @Before
+  public void setUp() throws Exception {
+    list = new IndexElemArray(7);
   }
-  
+
   @Test
   public void testFunctionality() throws Exception {
-    list.clear();
     boundaryCondition();
     add();
     clearAndAdd();
@@ -63,14 +58,13 @@ public class IndexElemArrayJUnitTest {
     clearAndAdd();
   }
 
-  @Test
   /**
    * This tests concurrent modification of IndexElemArray and to make 
    * sure elementData and size are updated atomically. Ticket# GEODE-106.   
    */
+  @Test
   public void testFunctionalityUsingMultiThread() throws Exception {
-    list.clear();
-    Collection<Callable> callables = new ConcurrentLinkedQueue<>();    
+    Collection<Callable> callables = new ConcurrentLinkedQueue<>();
     IntStream.range(0, 1000).parallel().forEach(i -> {
       callables.add(() -> {
         if (i%3 == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexMaintenanceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexMaintenanceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexMaintenanceJUnitTest.java
index 5616078..e087304 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexMaintenanceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexMaintenanceJUnitTest.java
@@ -1050,18 +1050,18 @@ public class IndexMaintenanceJUnitTest
 //    --ID;
 //    Index i1 = qs.createIndex("Index1", IndexType.FUNCTIONAL, "objs.maap[*]",
 //        "/testRgn objs");
-//    assertEquals(i1.getCanonicalizedIndexedExpression(), "index_iter1.maap[*]");
+//    assertIndexDetailsEquals(i1.getCanonicalizedIndexedExpression(), "index_iter1.maap[*]");
 //    assertTrue(i1 instanceof CompactMapRangeIndex);
 //    CompactMapRangeIndex mri = (CompactMapRangeIndex)i1;
 //    // Test index maintenance
 //    // addition of new Portfolio object
 //    Map<Object, CompactRangeIndex> indxMap = mri.getRangeIndexHolderForTesting();
-//    assertEquals(indxMap.size(), ID);
+//    assertIndexDetailsEquals(indxMap.size(), ID);
 //    for (int j = 1; j <= ID; ++j) {
 //      assertTrue(indxMap.containsKey("key" + j));
 //      CompactRangeIndex rng = indxMap.get("key" + j);
 //      Iterator itr = rng.valueToEntriesMap.values().iterator();
-//      assertEquals(rng.valueToEntriesMap.size(), 1);
+//      assertIndexDetailsEquals(rng.valueToEntriesMap.size(), 1);
 //      assertTrue(rng.valueToEntriesMap.containsKey("val" + j));
 //      Set<Integer> expectedElements = new HashSet<Integer>();
 //      for (int k = j; k <= ID; ++k) {
@@ -1075,7 +1075,7 @@ public class IndexMaintenanceJUnitTest
 //      else if (mapValue instanceof ConcurrentHashSet) {            
 //       size = ((ConcurrentHashSet)mapValue).size();
 //      }
-//      assertEquals(expectedElements.size(), size);
+//      assertIndexDetailsEquals(expectedElements.size(), size);
 //        for (Integer elem : expectedElements) {
 //          RegionEntry re = testRgn.basicGetEntry(elem);
 //          if (mapValue instanceof RegionEntry) {
@@ -1093,18 +1093,18 @@ public class IndexMaintenanceJUnitTest
 //    im.rerunIndexCreationQuery();
 //    ID =5;
 //    i1 =im.getIndex("Index1");
-//    assertEquals(i1.getCanonicalizedIndexedExpression(), "index_iter1.maap[*]");
+//    assertIndexDetailsEquals(i1.getCanonicalizedIndexedExpression(), "index_iter1.maap[*]");
 //    assertTrue(i1 instanceof CompactMapRangeIndex);
 //    mri = (CompactMapRangeIndex)i1;
 //    // Test index maintenance
 //    // addition of new Portfolio object
 //    indxMap = mri.getRangeIndexHolderForTesting();
-//    assertEquals(indxMap.size(), ID);
+//    assertIndexDetailsEquals(indxMap.size(), ID);
 //    for (int j = 1; j <= ID; ++j) {
 //      assertTrue(indxMap.containsKey("key" + j));
 //      CompactRangeIndex rng = indxMap.get("key" + j);
 //      Iterator itr = rng.valueToEntriesMap.values().iterator();
-//      assertEquals(rng.valueToEntriesMap.size(), 1);
+//      assertIndexDetailsEquals(rng.valueToEntriesMap.size(), 1);
 //      assertTrue(rng.valueToEntriesMap.containsKey("val" + j));
 //      Set<Integer> expectedElements = new HashSet<Integer>();
 //      for (int k = j; k <= ID; ++k) {
@@ -1118,7 +1118,7 @@ public class IndexMaintenanceJUnitTest
 //      else if (mapValue instanceof ConcurrentHashSet) {            
 //       size = ((ConcurrentHashSet)mapValue).size();
 //      }
-//      assertEquals(expectedElements.size(), size);
+//      assertIndexDetailsEquals(expectedElements.size(), size);
 //        for (Integer elem : expectedElements) {
 //          RegionEntry re = testRgn.basicGetEntry(elem);
 //          if (mapValue instanceof RegionEntry) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexStatisticsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexStatisticsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexStatisticsJUnitTest.java
index 58d966e..8275af7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexStatisticsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/IndexStatisticsJUnitTest.java
@@ -728,7 +728,7 @@ public class IndexStatisticsJUnitTest {
     region.put(1, obj1);
     assertEquals(1, stats.getNumberOfValues());
     assertEquals(1, stats.getNumberOfKeys());
-    // assertEquals(1, stats.getNumberOfValues(20f));
+    // assertIndexDetailsEquals(1, stats.getNumberOfValues(20f));
     assertEquals(1, stats.getNumUpdates());
 
     // add a second object with the same index key
@@ -737,21 +737,21 @@ public class IndexStatisticsJUnitTest {
     region.put(2, obj2);
     assertEquals(2, stats.getNumberOfValues());
     assertEquals(1, stats.getNumberOfKeys());
-    // assertEquals(2, stats.getNumberOfValues(20f));
+    // assertIndexDetailsEquals(2, stats.getNumberOfValues(20f));
     assertEquals(2, stats.getNumUpdates());
 
     // remove the second object and check that keys are 1
     region.remove(2);
     assertEquals(1, stats.getNumberOfValues());
     assertEquals(1, stats.getNumberOfKeys());
-    // assertEquals(1, stats.getNumberOfValues(20f));
+    // assertIndexDetailsEquals(1, stats.getNumberOfValues(20f));
     assertEquals(3, stats.getNumUpdates());
 
     // remove the first object and check that keys are 0
     region.remove(1);
     assertEquals(0, stats.getNumberOfValues());
     assertEquals(0, stats.getNumberOfKeys());
-    // assertEquals(0, stats.getNumberOfValues(20f));
+    // assertIndexDetailsEquals(0, stats.getNumberOfValues(20f));
     assertEquals(4, stats.getNumUpdates());
 
     // add object with a different key and check results
@@ -759,7 +759,7 @@ public class IndexStatisticsJUnitTest {
     region.put(3, obj2);
     assertEquals(1, stats.getNumberOfValues());
     assertEquals(1, stats.getNumberOfKeys());
-    // assertEquals(0, stats.getNumberOfValues(20f));
+    // assertIndexDetailsEquals(0, stats.getNumberOfValues(20f));
     assertEquals(5, stats.getNumUpdates());
 
     // add object with original key and check that num keys are 2
@@ -767,7 +767,7 @@ public class IndexStatisticsJUnitTest {
     region.put(1, obj1);
     assertEquals(2, stats.getNumberOfValues());
     assertEquals(2, stats.getNumberOfKeys());
-    // assertEquals(1, stats.getNumberOfValues(20f));
+    // assertIndexDetailsEquals(1, stats.getNumberOfValues(20f));
     assertEquals(6, stats.getNumUpdates());
   }
   


[11/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
index 63c7c74..d6c2e40 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
@@ -121,7 +121,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     Thread waiting = new Thread(new Runnable() {
       public void run() {
         try {
-          assertEquals(0, process.waitFor());
+          assertIndexDetailsEquals(0, process.waitFor());
         }
         catch (InterruptedException ignore) {
           logger.error("Interrupted while waiting for process!", ignore);
@@ -142,8 +142,8 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
 
     LocatorLauncher locatorLauncher = new Builder().setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath()).build();
 
-    assertEquals(Status.ONLINE, locatorLauncher.status().getStatus());
-    assertEquals(Status.STOPPED, locatorLauncher.stop().getStatus());
+    assertIndexDetailsEquals(Status.ONLINE, locatorLauncher.status().getStatus());
+    assertIndexDetailsEquals(Status.STOPPED, locatorLauncher.stop().getStatus());
   }
   */
 
@@ -584,7 +584,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
       // check the status
       final LocatorState locatorState = dirLauncher.status();
       assertNotNull(locatorState);
-      assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
+      assertIndexDetailsEquals(Status.NOT_RESPONDING, locatorState.getStatus());
       
       final String logFileName = getUniqueName()+".log";
       assertFalse("Log file should not exist: " + logFileName, new File(this.temporaryFolder.getRoot(), logFileName).exists());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
index 248c39f..90d8ece 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorStateJUnitTest.java
@@ -40,6 +40,22 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class LocatorStateJUnitTest {
 
+  private String classpath = "test_classpath";
+  private String gemFireVersion = "test_gemfireversion";
+  private String host = "test_host";
+  private String javaVersion = "test_javaversion";
+  private String jvmArguments = "test_jvmarguments";
+  private String serviceLocation = "test_location";
+  private String logFile = "test_logfile";
+  private String memberName = "test_membername";
+  private Integer pid = 6396;
+  private String port = "test_port";
+  private String statusDescription = Status.NOT_RESPONDING.getDescription();
+  private String statusMessage = "test_statusmessage";
+  private Long timestampTime = 1450728233024L;
+  private Long uptime = 1629L;
+  private String workingDirectory = "test_workingdirectory";
+
   @Test
   public void fromJsonWithEmptyStringThrowsIllegalArgumentException() throws Exception {
     // given: empty string
@@ -122,66 +138,64 @@ public class LocatorStateJUnitTest {
     return LocatorState.fromJson(value);
   }
 
-  private String classpath = "test_classpath";
-  private String gemFireVersion = "test_gemfireversion";
-  private String host = "test_host";
-  private String javaVersion = "test_javaversion";
-  private String jvmArguments = "test_jvmarguments";
-  private String serviceLocation = "test_location";
-  private String logFile = "test_logfile";
-  private String memberName = "test_membername";
-  private Integer pid = 6396;
-  private String port = "test_port";
-  private String statusDescription = Status.NOT_RESPONDING.getDescription();
-  private String statusMessage = "test_statusmessage";
-  private Long timestampTime = 1450728233024L;
-  private Long uptime = 1629L;
-  private String workingDirectory = "test_workingdirectory";
-  
   private String getClasspath() {
     return this.classpath;
   }
+
   private String getGemFireVersion() {
     return this.gemFireVersion;
   }
+
   private String getHost() {
     return this.host;
   }
+
   private String getJavaVersion() {
     return this.javaVersion;
   }
+
   private List<String> getJvmArguments() {
     List<String> list = new ArrayList<String>();
     list.add(this.jvmArguments);
     return list;
   }
+
   private String getServiceLocation() {
     return this.serviceLocation;
   }
+
   private String getLogFile() {
     return this.logFile;
   }
+
   private String getMemberName() {
     return this.memberName;
   }
+
   private Integer getPid() {
     return this.pid;
   }
+
   private String getPort() {
     return this.port;
   }
+
   private String getStatusDescription() {
     return this.statusDescription;
   }
+
   private String getStatusMessage() {
     return this.statusMessage;
   }
+
   private Long getTimestampTime() {
     return this.timestampTime;
   }
+
   private Long getUptime() {
     return this.uptime;
   }
+
   private String getWorkingDirectory() {
     return this.workingDirectory;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
index 395a9e6..4462ca3 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
@@ -18,40 +18,35 @@ package com.gemstone.gemfire.distributed;
 
 import static org.junit.Assert.*;
 
-import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collections;
-import java.util.Properties;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.distributed.ServerLauncher.Builder;
-import com.gemstone.gemfire.distributed.ServerLauncher.Command;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.support.DistributedSystemAdapter;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
 import edu.umd.cs.mtc.MultithreadedTestCase;
 import edu.umd.cs.mtc.TestFramework;
-
 import org.jmock.Expectations;
 import org.jmock.Mockery;
 import org.jmock.lib.concurrent.Synchroniser;
 import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.ServerLauncher.Builder;
+import com.gemstone.gemfire.distributed.ServerLauncher.Command;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.support.DistributedSystemAdapter;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * The ServerLauncherJUnitTest class is a test suite of unit tests testing the contract, functionality and invariants
  * of the ServerLauncher class.
@@ -744,6 +739,7 @@ public class ServerLauncherJUnitTest {
     assertFalse(serverLauncher.isDisableDefaultServer());
     assertFalse(serverLauncher.isDefaultServerEnabled(mockCache));
   }
+
   @Test
   public void testIsDefaultServerEnabledWhenNoCacheServersExistAndDefaultServerDisabled() {
     final Cache mockCache = mockContext.mock(Cache.class, "Cache");
@@ -826,12 +822,6 @@ public class ServerLauncherJUnitTest {
     serverLauncher.startCacheServer(mockCache);
   }
   
-  public static void main(final String... args) {
-    System.err.printf("Thread (%1$s) is daemon (%2$s)%n", Thread.currentThread().getName(),
-      Thread.currentThread().isDaemon());
-    new Builder(args).setCommand(Command.START).build().run();
-  }
-
   private final class ServerWaitMultiThreadedTestCase extends MultithreadedTestCase {
 
     private final AtomicBoolean connectionStateHolder = new AtomicBoolean(true);
@@ -845,7 +835,8 @@ public class ServerLauncherJUnitTest {
       final Cache mockCache = mockContext.mock(Cache.class, "Cache");
 
       final DistributedSystem mockDistributedSystem = new DistributedSystemAdapter() {
-        @Override public boolean isConnected() {
+        @Override
+        public boolean isConnected() {
           return connectionStateHolder.get();
         }
       };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
index fd7d806..dd028d2 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
@@ -432,7 +432,7 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
       final int pid = readPid(this.pidFile);
       assertTrue(pid > 0);
       assertTrue(ProcessUtils.isProcessAlive(pid));
-      assertEquals(getPid(), pid);
+      assertIndexDetailsEquals(getPid(), pid);
       
       // validate log file was created
       final String logFileName = getUniqueName()+".log";
@@ -446,7 +446,7 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
     }
 
     try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
+      assertIndexDetailsEquals(Status.STOPPED, this.launcher.stop().getStatus());
       waitForFileToDelete(this.pidFile);
     } catch (Throwable e) {
       logger.error(e);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
index 056e6ce..700cb18 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
@@ -137,7 +137,7 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
 //      @Override
 //      public void run() {
 //        try {
-//          assertEquals(0, process.waitFor());
+//          assertIndexDetailsEquals(0, process.waitFor());
 //        } catch (InterruptedException e) {
 //          logger.error("Interrupted while waiting for process", e);
 //        }
@@ -958,7 +958,7 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
       // check the status
       final ServerState serverState = dirLauncher.status();
       assertNotNull(serverState);
-      assertEquals(Status.NOT_RESPONDING, serverState.getStatus());
+      assertIndexDetailsEquals(Status.NOT_RESPONDING, serverState.getStatus());
       
       final String logFileName = getUniqueName()+".log";
       assertFalse("Log file should not exist: " + logFileName, new File(this.temporaryFolder.getRoot(), logFileName).exists());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/AtomicLongWithTerminalStateJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/AtomicLongWithTerminalStateJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/AtomicLongWithTerminalStateJUnitTest.java
index aca6874..eec4730 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/AtomicLongWithTerminalStateJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/AtomicLongWithTerminalStateJUnitTest.java
@@ -16,18 +16,17 @@
  */
 package com.gemstone.gemfire.distributed.internal;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class AtomicLongWithTerminalStateJUnitTest extends TestCase {
-  
+public class AtomicLongWithTerminalStateJUnitTest {
+
+  @Test
   public void test() {
     AtomicLongWithTerminalState al = new AtomicLongWithTerminalState();
     assertEquals(23, al.compareAddAndGet(-1, 23));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionConfigJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionConfigJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionConfigJUnitTest.java
index d2b5643..cc6310f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionConfigJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionConfigJUnitTest.java
@@ -16,49 +16,60 @@
  */
 package com.gemstone.gemfire.distributed.internal;
 
-import com.gemstone.gemfire.InternalGemFireException;
-import com.gemstone.gemfire.UnmodifiableException;
-import com.gemstone.gemfire.internal.ConfigSource;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.File;
-import java.lang.reflect.Method;
-import java.util.*;
-
-import static org.junit.Assert.*;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import com.gemstone.gemfire.InternalGemFireException;
+import com.gemstone.gemfire.UnmodifiableException;
+import com.gemstone.gemfire.internal.ConfigSource;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * Created by jiliao on 2/2/16.
- */
 @Category(UnitTest.class)
-
 public class DistributionConfigJUnitTest {
-  static Map<String, ConfigAttribute> attributes;
-  static Map<String, Method> setters;
-  static Map<String, Method> getters;
-  static Map<String, Method> isModifiables;
-  static Map<String, Method> checkers;
-  static String[] attNames;
-  DistributionConfigImpl config;
-
-  @BeforeClass
-  public static void beforeClass() {
+
+  private Map<Class<?>, Class<?>> classMap;
+
+  private Map<String, ConfigAttribute> attributes;
+  private Map<String, Method> setters;
+  private Map<String, Method> getters;
+  private Map<String, Method> checkers;
+  private String[] attNames;
+
+  private DistributionConfigImpl config;
+
+  @Before
+  public void before() {
+    classMap = new HashMap<Class<?>, Class<?>>();
+    classMap.put(boolean.class, Boolean.class);
+    classMap.put(byte.class, Byte.class);
+    classMap.put(short.class, Short.class);
+    classMap.put(char.class, Character.class);
+    classMap.put(int.class, Integer.class);
+    classMap.put(long.class, Long.class);
+    classMap.put(float.class, Float.class);
+    classMap.put(double.class, Double.class);
+
     attributes = DistributionConfig.attributes;
     setters = DistributionConfig.setters;
     getters = DistributionConfig.getters;
     attNames = DistributionConfig.dcValidAttributeNames;
     checkers = AbstractDistributionConfig.checkers;
-  }
 
-  @Before
-  public void before() {
     config = new DistributionConfigImpl(new Properties());
   }
 
@@ -96,6 +107,7 @@ public class DistributionConfigJUnitTest {
     System.out.println("filelList: " + fileList);
     System.out.println();
     System.out.println("otherList: " + otherList);
+
     assertEquals(boolList.size(), 30);
     assertEquals(intList.size(), 33);
     assertEquals(stringList.size(), 69);
@@ -227,6 +239,7 @@ public class DistributionConfigJUnitTest {
         modifiables.add(attName);
       }
     }
+
     assertEquals(modifiables.size(), 10);
     assertEquals(modifiables.get(0), "archive-disk-space-limit");
     assertEquals(modifiables.get(1), "archive-file-size-limit");
@@ -238,7 +251,6 @@ public class DistributionConfigJUnitTest {
     assertEquals(modifiables.get(7), "statistic-archive-file");
     assertEquals(modifiables.get(8), "statistic-sample-rate");
     assertEquals(modifiables.get(9), "statistic-sampling-enabled");
-
   }
 
   @Test(expected = IllegalArgumentException.class)
@@ -297,17 +309,4 @@ public class DistributionConfigJUnitTest {
     assertTrue(config.isAttributeModifiable(DistributionConfig.HTTP_SERVICE_PORT_NAME));
     assertTrue(config.isAttributeModifiable("jmx-manager-http-port"));
   }
-
-  public final static Map<Class<?>, Class<?>> classMap = new HashMap<Class<?>, Class<?>>();
-
-  static {
-    classMap.put(boolean.class, Boolean.class);
-    classMap.put(byte.class, Byte.class);
-    classMap.put(short.class, Short.class);
-    classMap.put(char.class, Character.class);
-    classMap.put(int.class, Integer.class);
-    classMap.put(long.class, Long.class);
-    classMap.put(float.class, Float.class);
-    classMap.put(double.class, Double.class);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
index 2024bf2..8832459 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
@@ -132,7 +132,7 @@ public class InternalDistributedSystemJUnitTest
     assertEquals(DistributionConfig.DEFAULT_LOG_FILE, config.getLogFile());
 
     //default log level gets overrided by the gemfire.properties created for unit tests.
-//    assertEquals(DistributionConfig.DEFAULT_LOG_LEVEL, config.getLogLevel());
+//    assertIndexDetailsEquals(DistributionConfig.DEFAULT_LOG_LEVEL, config.getLogLevel());
 
     assertEquals(DistributionConfig.DEFAULT_STATISTIC_SAMPLING_ENABLED,
                  config.getStatisticSamplingEnabled());
@@ -145,7 +145,7 @@ public class InternalDistributedSystemJUnitTest
 
     // ack-wait-threadshold is overridden on VM's command line using a
     // system property.  This is not a valid test.  Hrm.
-//     assertEquals(DistributionConfig.DEFAULT_ACK_WAIT_THRESHOLD, config.getAckWaitThreshold());
+//     assertIndexDetailsEquals(DistributionConfig.DEFAULT_ACK_WAIT_THRESHOLD, config.getAckWaitThreshold());
 
     assertEquals(DistributionConfig.DEFAULT_ACK_SEVERE_ALERT_THRESHOLD, config.getAckSevereAlertThreshold());
     
@@ -683,7 +683,7 @@ public class InternalDistributedSystemJUnitTest
     Locator locator = (Locator) locators.iterator().next();
     Assert.assertTrue(locator.isPeerLocator());
 //    Assert.assertFalse(locator.isServerLocator()); server location is forced on while licensing is disabled in GemFire
-//    Assert.assertEquals("127.0.0.1", locator.getBindAddress().getHostAddress());  removed this check for ipv6 testing
+//    Assert.assertIndexDetailsEquals("127.0.0.1", locator.getBindAddress().getHostAddress());  removed this check for ipv6 testing
     Assert.assertEquals(unusedPort, locator.getPort());
     deleteStateFile(unusedPort);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotJUnitTest.java
index 34ac767..fb391f9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/LocatorLoadSnapshotJUnitTest.java
@@ -22,23 +22,18 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.server.ServerLoad;
-import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Tests the functionality of the LocatorLoadSnapshot, which
  * is the data structure that is used in the locator to compare
  * the load between multiple servers.
- * 
  */
 @Category(UnitTest.class)
 public class LocatorLoadSnapshotJUnitTest {
@@ -61,7 +56,6 @@ public class LocatorLoadSnapshotJUnitTest {
    * and then we test that after several requests, the
    * load balancer starts sending connections to the second
    * server.
-   * 
    */
   @Test
   public void testTwoServers() {
@@ -174,7 +168,7 @@ public class LocatorLoadSnapshotJUnitTest {
    * servers with interecting groups correctly.
    */
   @Test
-  public void testInterectingGroups() {
+  public void testIntersectingGroups() {
     LocatorLoadSnapshot sn = new LocatorLoadSnapshot();
     ServerLocation l1 = new ServerLocation("localhost", 1);
     ServerLocation l2 = new ServerLocation("localhost", 2);
@@ -258,88 +252,6 @@ public class LocatorLoadSnapshotJUnitTest {
     assertEquals(Arrays.asList(new ServerLocation[] {} ), sn.getServersForQueue(null, excludeAll, 3));
   }
   
-  /**
-   * A basic test of concurrent functionality. Starts a number of
-   * threads making requests and expects the load to be balanced between
-   * three servers.
-   * @throws InterruptedException
-   */
-  @Category(FlakyTest.class) // GEODE-613: lots of threads, async action, IntegrationTest-not-UnitTest, thread joins, time sensitive
-  @Test
-  public void testConcurrentBalancing() throws InterruptedException {
-    int NUM_THREADS = 50;
-    final int NUM_REQUESTS = 10000;
-    int ALLOWED_THRESHOLD = 50; //We should never be off by more than
-    //the number of concurrent threads.
-    
-    final LocatorLoadSnapshot sn = new LocatorLoadSnapshot();
-    final ServerLocation l1 = new ServerLocation("localhost", 1);
-    final ServerLocation l2 = new ServerLocation("localhost", 2);
-    final ServerLocation l3 = new ServerLocation("localhost", 3);
-    
-    int initialLoad1 = (int) (Math.random() * (NUM_REQUESTS / 2));
-    int initialLoad2 = (int) (Math.random() * (NUM_REQUESTS / 2));
-    int initialLoad3 = (int) (Math.random() * (NUM_REQUESTS / 2));
-    
-    sn.addServer(l1, new String[0], new ServerLoad(initialLoad1, 1, 0, 1));
-    sn.addServer(l2, new String[0], new ServerLoad(initialLoad2, 1, 0, 1));
-    sn.addServer(l3, new String[0], new ServerLoad(initialLoad3, 1, 0, 1));
-    
-    final Map loadCounts = new HashMap();
-    loadCounts.put(l1, new AtomicInteger(initialLoad1));
-    loadCounts.put(l2, new AtomicInteger(initialLoad2));
-    loadCounts.put(l3, new AtomicInteger(initialLoad3));
-    
-    Thread[] threads = new Thread[NUM_THREADS];
-//    final Object lock = new Object();
-    for(int i =0; i < NUM_THREADS; i++) {
-      threads[i] = new Thread("Thread-" + i) {
-        public void run() {
-          for(int ii = 0; ii < NUM_REQUESTS; ii++) {
-            ServerLocation location;
-//            synchronized(lock) {
-              location = sn.getServerForConnection(null, Collections.EMPTY_SET);
-//            }
-            AtomicInteger count = (AtomicInteger) loadCounts.get(location);
-            count.incrementAndGet();
-          }
-        }
-      };
-    }
-    
-    for(int i =0; i < NUM_THREADS; i++) {
-      threads[i].start();
-    }
-    
-    for(int i =0; i < NUM_THREADS; i++) {
-      Thread t = threads[i];
-      long ms = 30 * 1000;
-      t.join(30 * 1000);
-      if (t.isAlive()) {
-        for(int j =0; j < NUM_THREADS; j++) {
-          threads[j].interrupt();
-        }
-        fail("Thread did not terminate after " + ms + " ms: " + t);
-      }
-    }
-    
-    double expectedPerServer = ( initialLoad1 + initialLoad2 + initialLoad3 + 
-              NUM_REQUESTS * NUM_THREADS) / (double) loadCounts.size();
-//    for(Iterator itr = loadCounts.entrySet().iterator(); itr.hasNext(); ) {
-//      Map.Entry entry = (Entry) itr.next();
-//      ServerLocation location = (ServerLocation) entry.getKey();
-//      AI count= (AI) entry.getValue();
-//    }
-    
-    for(Iterator itr = loadCounts.entrySet().iterator(); itr.hasNext(); ) {
-      Map.Entry entry = (Entry) itr.next();
-      ServerLocation location = (ServerLocation) entry.getKey();
-      AtomicInteger count= (AtomicInteger) entry.getValue();
-      int difference = (int) Math.abs(count.get() - expectedPerServer);
-      assertTrue("Count " + count + " for server " + location + " is not within " + ALLOWED_THRESHOLD + " of " + expectedPerServer, difference < ALLOWED_THRESHOLD);
-    }
-  }
-  
   @Test
   public void testAreBalanced() {
     final LocatorLoadSnapshot sn = new LocatorLoadSnapshot();
@@ -367,19 +279,5 @@ public class LocatorLoadSnapshotJUnitTest {
     assertTrue(sn.hasBalancedConnections("a"));
     assertFalse(sn.hasBalancedConnections("b"));
   }
-  
-  public void _test2() { // delete this method?
-    final LocatorLoadSnapshot sn = new LocatorLoadSnapshot();
-    sn.addServer(new ServerLocation("hs20h.gemstone.com",28543), new String[0], new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20l.gemstone.com",22385), new String[0], new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20n.gemstone.com",23482), new String[0], new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20m.gemstone.com",23429), new String[0], new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20e.gemstone.com",20154), new String[0],new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20j.gemstone.com",24273), new String[0],new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20g.gemstone.com",27125), new String[0],new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20i.gemstone.com",25201), new String[0],new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20k.gemstone.com",23711), new String[0],new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-    sn.addServer(new ServerLocation("hs20f.gemstone.com",21025), new String[0],new ServerLoad(0.0f, 0.00125f, 0.0f, 1.0f));
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogJUnitTest.java
index ffcf09c..7e23f7b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogJUnitTest.java
@@ -16,71 +16,92 @@
  */
 package com.gemstone.gemfire.distributed.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
+@Category(UnitTest.class) // Fails on Windows -- see GEODE-373
+public class ProductUseLogJUnitTest {
 
-@Category(UnitTest.class)
-public class ProductUseLogJUnitTest extends TestCase {
+  private long oldMax;
+  private File logFile;
+  private ProductUseLog log;
 
-  public void testBasics() throws Exception {
-    File logFile = new File("ProductUseLogTest_testBasics.log");
-    if (logFile.exists()) {
-      logFile.delete();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    oldMax = ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE;
+    logFile = temporaryFolder.newFile(getClass().getSimpleName() + "_" + testName.getMethodName() + ".log");
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE = oldMax;
+    if (log != null) {
+      log.close();
     }
-    ProductUseLog log = new ProductUseLog(logFile);
+  }
+
+  @Test
+  public void testBasics() throws Exception {
+    assertTrue(logFile.delete());
+
+    log = new ProductUseLog(logFile);
+
     assertTrue(logFile.exists());
+
     log.log("test message");
     log.close();
     log.log("shouldn't be logged");
     log.reopen();
     log.log("test message");
     log.close();
+
     BufferedReader reader = new BufferedReader(new  FileReader(logFile));
-    try {
-      String line = reader.readLine();
-      assertTrue(line.length() == 0);
-      line = reader.readLine();
-      assertTrue("expected first line to contain 'test message'", line.contains("test message"));
-
-      line = reader.readLine();
-      assertTrue(line.length() == 0);
-      line = reader.readLine();
-      assertTrue("expected second line to contain 'test message'", line.contains("test message"));
-
-      line = reader.readLine();
-      assertTrue("expected only two non-empty lines in the file", line == null);
-    } finally {
-      reader.close();
-    }
+
+    String line = reader.readLine();
+    assertTrue(line.length() == 0);
+    line = reader.readLine();
+    assertTrue("expected first line to contain 'test message'", line.contains("test message"));
+
+    line = reader.readLine();
+    assertTrue(line.length() == 0);
+    line = reader.readLine();
+    assertTrue("expected second line to contain 'test message'", line.contains("test message"));
+
+    line = reader.readLine();
+    assertTrue("expected only two non-empty lines in the file", line == null);
   }
 
+  @Test
   public void testSizeLimit() throws Exception {
-    long oldMax = ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE;
     ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE = 2000L;
-    try {
-      File logFile = new File("ProductUseLogTest_testSizeLimit.log");
-      assertTrue(logFile.createNewFile());
-      ProductUseLog log = new ProductUseLog(logFile);
-      try {
-        String logEntry = "log entry";
-        for (long i=0; i<ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE; i++) {
-          log.log(logEntry);
-          assertTrue("expected " + logFile.getPath() + " to remain under "+ 
-              ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE + " bytes in length",
-              logFile.length() < ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE);
-        }
-      } finally {
-        log.close();
-      }
-    } finally {
-      ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE = oldMax;
+
+    ProductUseLog log = new ProductUseLog(logFile);
+
+    String logEntry = "log entry";
+    for (long i=0; i<ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE; i++) {
+      log.log(logEntry);
+      assertTrue("expected " + logFile.getPath() + " to remain under "+
+          ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE + " bytes in length",
+          logFile.length() < ProductUseLog.MAX_PRODUCT_USE_FILE_SIZE);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
index 7690c87..bb83df9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
@@ -20,33 +20,30 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusResponse;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
-import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * The ServerLocatorJUnitTest class is a test suite of test cases testing the contract and functionality of the
  * ServerLocator class.
  * </p>
+ * TODO: write more unit tests for this class...
+ * </p>
  * @see com.gemstone.gemfire.distributed.internal.ServerLocator
  * @see org.junit.Assert
  * @see org.junit.Test
  * @since 7.0
  */
-// TODO Dan, write more unit tests for this class...
 @Category(UnitTest.class)
 public class ServerLocatorJUnitTest {
 
-  protected ServerLocator createServerLocator() throws IOException {
-    return new TestServerLocator();
-  }
-
   @Test
   public void testProcessRequestProcessesLocatorStatusRequest() throws IOException {
     final ServerLocator serverLocator = createServerLocator();
@@ -56,7 +53,11 @@ public class ServerLocatorJUnitTest {
     assertTrue(response instanceof LocatorStatusResponse);
   }
 
-  protected static class TestServerLocator extends ServerLocator {
+  private ServerLocator createServerLocator() throws IOException {
+    return new TestServerLocator();
+  }
+
+  private static class TestServerLocator extends ServerLocator {
     TestServerLocator() throws IOException {
       super();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.java
index f308855..8af42b6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.java
@@ -31,7 +31,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link SharedConfiguration}.
- * 
  *
  * @since 8.1
  */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/StartupMessageDataJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/StartupMessageDataJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/StartupMessageDataJUnitTest.java
index aa0a88e..ee3f99e 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/StartupMessageDataJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/StartupMessageDataJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.distributed.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.DataInput;
 import java.io.DataOutputStream;
 import java.util.ArrayList;
@@ -24,6 +26,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.StringTokenizer;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.DataSerializer;
@@ -33,21 +36,15 @@ import com.gemstone.gemfire.internal.ByteArrayData;
 import com.gemstone.gemfire.internal.admin.remote.DistributionLocatorId;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 /**
  * Tests {@link StartupMessageData}.
  * 
  * @since 7.0
  */
 @Category(UnitTest.class)
-public class StartupMessageDataJUnitTest extends TestCase {
-
-  public StartupMessageDataJUnitTest(String name) {
-    super(name);
-  }
-  
+public class StartupMessageDataJUnitTest {
 
+  @Test
   public void testWriteHostedLocatorsWithEmpty() throws Exception {
     Collection<String> hostedLocators = new ArrayList<String>();
     StartupMessageData data = new StartupMessageData();
@@ -55,6 +52,7 @@ public class StartupMessageDataJUnitTest extends TestCase {
     assertTrue(data.getOptionalFields().isEmpty());
   }
 
+  @Test
   public void testWriteHostedLocatorsWithNull() throws Exception {
     Collection<String> hostedLocators = null;
     StartupMessageData data = new StartupMessageData();
@@ -62,6 +60,7 @@ public class StartupMessageDataJUnitTest extends TestCase {
     assertTrue(data.getOptionalFields().isEmpty());
   }
 
+  @Test
   public void testWriteHostedLocatorsWithOne() throws Exception {
     String locatorString = createOneLocatorString();
     
@@ -74,6 +73,7 @@ public class StartupMessageDataJUnitTest extends TestCase {
     assertEquals(locatorString, data.getOptionalFields().get(StartupMessageData.HOSTED_LOCATORS));
   }
 
+  @Test
   public void testWriteHostedLocatorsWithThree() throws Exception {
     String[] locatorStrings = createManyLocatorStrings(3);
     List<String> hostedLocators = new ArrayList<String>();
@@ -94,7 +94,8 @@ public class StartupMessageDataJUnitTest extends TestCase {
       assertEquals(locatorStrings[i], st.nextToken());
     }
   }
-  
+
+  @Test
   public void testReadHostedLocatorsWithThree() throws Exception {
     // set up the data
     String[] locatorStrings = createManyLocatorStrings(3);
@@ -116,7 +117,8 @@ public class StartupMessageDataJUnitTest extends TestCase {
       i++;
     }
   }
-  
+
+  @Test
   public void testToDataWithEmptyHostedLocators() throws Exception {
     Collection<String> hostedLocators = new ArrayList<String>();
     StartupMessageData data = new StartupMessageData();
@@ -134,6 +136,7 @@ public class StartupMessageDataJUnitTest extends TestCase {
     assertNull(props);
   }
 
+  @Test
   public void testToDataWithNullHostedLocators() throws Exception {
     Collection<String> hostedLocators = null;
     StartupMessageData data = new StartupMessageData();
@@ -150,7 +153,8 @@ public class StartupMessageDataJUnitTest extends TestCase {
     Properties props = (Properties) DataSerializer.readObject(in);
     assertNull(props);
   }
-  
+
+  @Test
   public void testToDataWithOneHostedLocator() throws Exception {
     String locatorString = createOneLocatorString();
     
@@ -176,6 +180,7 @@ public class StartupMessageDataJUnitTest extends TestCase {
     assertEquals(locatorString, hostedLocatorsString);
   }
 
+  @Test
   public void testToDataWithThreeHostedLocators() throws Exception {
     String[] locatorStrings = createManyLocatorStrings(3);
     List<String> hostedLocators = new ArrayList<String>();
@@ -215,6 +220,7 @@ public class StartupMessageDataJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testNullHostedLocator() throws Exception {
     String locatorString = null;
     DataInput in = getDataInputWithOneHostedLocator(locatorString);
@@ -223,7 +229,8 @@ public class StartupMessageDataJUnitTest extends TestCase {
     Collection<String> readHostedLocators = dataToRead.readHostedLocators();
     assertNull(readHostedLocators);
   }
-  
+
+  @Test
   public void testEmptyHostedLocator() throws Exception {
     String locatorString = "";
     DataInput in = getDataInputWithOneHostedLocator(locatorString);
@@ -232,7 +239,8 @@ public class StartupMessageDataJUnitTest extends TestCase {
     Collection<String> readHostedLocators = dataToRead.readHostedLocators();
     assertNull(readHostedLocators);
   }
-  
+
+  @Test
   public void testOneHostedLocator() throws Exception {
     String locatorString = createOneLocatorString();
     DataInput in = getDataInputWithOneHostedLocator(locatorString);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorJUnitTest.java
deleted file mode 100644
index d27aa2c..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorJUnitTest.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * 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.distributed.internal.deadlock;
-
-import static org.junit.Assert.*;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.junit.After;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- *
- */
-@Category(UnitTest.class)
-public class DeadlockDetectorJUnitTest {
-  
-  private final Set<Thread> stuckThreads = Collections.synchronizedSet(new HashSet<Thread>());
-  
-  @After
-  public void tearDown() {
-    for(Thread thread: stuckThreads) {
-      thread.interrupt();
-      try {
-        thread.join(20 * 1000);
-      } catch (InterruptedException e) {
-        throw new RuntimeException("interrupted", e);
-      }
-      if(thread.isAlive()) {
-        fail("Couldn't kill" + thread);
-      }
-    }
-    stuckThreads.clear();
-  }
-  
-  @Test
-  public void testNoDeadlocks() {
-    DeadlockDetector detector = new DeadlockDetector();
-    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
-    assertEquals(null, detector.findDeadlock());
-  }
-  
-  //this is commented out, because we can't
-  //clean up the threads deadlocked on monitors.
-  @Ignore
-  @Test
-  public void testMonitorDeadlock() throws InterruptedException {
-    final Object lock1 = new Object();
-    final Object lock2 = new Object();
-    Thread thread1 =  new Thread() {
-      public void run() {
-        stuckThreads.add(Thread.currentThread());
-        synchronized(lock1) {
-          Thread thread2 = new Thread() {
-            public void run() {
-              stuckThreads.add(Thread.currentThread());
-              synchronized(lock2) {
-               synchronized(lock1) {
-                 System.out.println("we won't get here");
-               }
-              }
-            }
-          };
-          thread2.start();
-          try {
-            Thread.sleep(1000);
-            synchronized(lock2) {
-              System.out.println("We won't get here");
-            }
-          } catch (InterruptedException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
-          }
-        }
-      }
-    };
-    
-    thread1.start();
-    Thread.sleep(2000);
-    DeadlockDetector detector = new DeadlockDetector();
-    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
-    LinkedList<Dependency> deadlocks = detector.findDeadlock();
-    System.out.println("deadlocks=" +  DeadlockDetector.prettyFormat(deadlocks));
-    assertEquals(4, detector.findDeadlock().size());
-  }
-
-  /**
-   * Make sure that we can detect a deadlock between two threads
-   * that are trying to acquire a two different syncs in the different orders.
-   * @throws InterruptedException
-   */
-  @Test
-  public void testSyncDeadlock() throws InterruptedException {
-
-    final Lock lock1 = new ReentrantLock();
-    final Lock lock2 = new ReentrantLock();
-    Thread thread1 =  new Thread() {
-      public void run() {
-        stuckThreads.add(Thread.currentThread());
-        lock1.lock();
-        Thread thread2 = new Thread() {
-          public void run() {
-            stuckThreads.add(Thread.currentThread());
-            lock2.lock();
-            try {
-              lock1.tryLock(10, TimeUnit.SECONDS);
-            } catch (InterruptedException e) {
-              //ignore
-            }
-            lock2.unlock();
-          }
-        };
-        thread2.start();
-        try {
-          Thread.sleep(1000);
-          lock2.tryLock(10, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-          //ignore
-        }
-        lock1.unlock();
-      }
-    };
-    
-    thread1.start();
-    Thread.sleep(2000);
-    DeadlockDetector detector = new DeadlockDetector();
-    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
-    LinkedList<Dependency> deadlocks = detector.findDeadlock();
-    System.out.println("deadlocks=" +  DeadlockDetector.prettyFormat(deadlocks));
-    assertEquals(4, detector.findDeadlock().size());
-  }
-  
-  //Semaphore are also not supported by the JDK
-  @Ignore
-  @Test
-  public void testSemaphoreDeadlock() throws InterruptedException {
-
-    final Semaphore lock1 = new Semaphore(1);
-    final Semaphore lock2 = new Semaphore(1);
-    Thread thread1 =  new Thread() {
-      public void run() {
-        stuckThreads.add(Thread.currentThread());
-        try {
-          lock1.acquire();
-        } catch (InterruptedException e1) {
-          e1.printStackTrace();
-        }
-        Thread thread2 = new Thread() {
-          public void run() {
-            stuckThreads.add(Thread.currentThread());
-            try {
-              lock2.acquire();
-              lock1.tryAcquire(10, TimeUnit.SECONDS);
-            } catch (InterruptedException e) {
-              //ignore
-            }
-            lock2.release();
-          }
-        };
-        thread2.start();
-        try {
-          Thread.sleep(1000);
-          lock2.tryAcquire(10, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-          //ignore
-        }
-        lock1.release();
-      }
-    };
-    
-    thread1.start();
-    Thread.sleep(2000);
-    DeadlockDetector detector = new DeadlockDetector();
-    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
-    LinkedList<Dependency> deadlocks = detector.findDeadlock();
-    System.out.println("deadlocks=" +  DeadlockDetector.prettyFormat(deadlocks));
-    assertEquals(4, detector.findDeadlock().size());
-  }
-  
-  /**
-   * This type of deadlock is currently not detected
-   * @throws InterruptedException
-   */
-  @Ignore
-  @Test
-  public void testReadLockDeadlock() throws InterruptedException {
-
-    final ReadWriteLock lock1 = new ReentrantReadWriteLock();
-    final ReadWriteLock lock2 = new ReentrantReadWriteLock();
-    Thread thread1 =  new Thread() {
-      public void run() {
-        stuckThreads.add(Thread.currentThread());
-        lock1.readLock().lock();
-        Thread thread2 = new Thread() {
-          public void run() {
-            stuckThreads.add(Thread.currentThread());
-            lock2.readLock().lock();
-            try {
-              lock1.writeLock().tryLock(10, TimeUnit.SECONDS);
-            } catch (InterruptedException e) {
-              e.printStackTrace();
-            }
-            lock2.readLock().unlock();
-          }
-        };
-        thread2.start();
-        try {
-          Thread.sleep(1000);
-          lock2.writeLock().tryLock(10, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-          // TODO Auto-generated catch block
-          e.printStackTrace();
-        }
-        lock1.readLock().unlock();
-      }
-    };
-    
-    thread1.start();
-    Thread.sleep(2000);
-    DeadlockDetector detector = new DeadlockDetector();
-    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
-    LinkedList<Dependency> deadlocks = detector.findDeadlock();
-    System.out.println("deadlocks=" +  deadlocks);
-    assertEquals(4, detector.findDeadlock().size());
-  }
-  
-  /**
-   * Test that the deadlock detector will find deadlocks
-   * that are reported by the {@link DependencyMonitorManager}
-   */
-  @Test
-  public void testProgramaticDependencies() {
-    final CountDownLatch cdl = new CountDownLatch(1);
-    MockDependencyMonitor mock = new MockDependencyMonitor();
-    DependencyMonitorManager.addMonitor(mock);
-    
-    Thread t1 = startAThread(cdl);
-    Thread t2 = startAThread(cdl);
-    
-    String resource1 = "one";
-    String resource2 = "two";
-    
-    mock.addDependency(t1, resource1);
-    mock.addDependency(resource1, t2);
-    mock.addDependency(t2, resource2);
-    mock.addDependency(resource2, t1);
-    
-
-    DeadlockDetector detector = new DeadlockDetector();
-    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
-    LinkedList<Dependency> deadlocks = detector.findDeadlock();
-    System.out.println("deadlocks=" + deadlocks);
-    assertEquals(4, deadlocks.size());
-    cdl.countDown();
-    DependencyMonitorManager.removeMonitor(mock);
-  }
-
-  private Thread startAThread(final CountDownLatch cdl) {
-    Thread thread = new Thread() {
-      public void run() {
-        try {
-          cdl.await();
-        } catch (InterruptedException e) {
-        }
-      }
-    };
-    thread.start();
-    
-    return thread;
-  }
-  
-  /**
-   * A fake dependency monitor.
-   *
-   */
-  private static class MockDependencyMonitor implements DependencyMonitor {
-    
-    Set<Dependency<Thread, Serializable>> blockedThreads = new HashSet<Dependency<Thread, Serializable>>();
-    Set<Dependency<Serializable, Thread>> held = new HashSet<Dependency<Serializable, Thread>>();
-
-    public Set<Dependency<Thread, Serializable>> getBlockedThreads(
-        Thread[] allThreads) {
-      return blockedThreads;
-    }
-
-    public void addDependency(String resource, Thread thread) {
-      held.add(new Dependency<Serializable, Thread>(resource, thread));
-      
-    }
-
-    public void addDependency(Thread thread, String resource) {
-      blockedThreads.add(new Dependency<Thread, Serializable>(thread, resource));
-    }
-
-    public Set<Dependency<Serializable, Thread>> getHeldResources(
-        Thread[] allThreads) {
-      return held;
-    }
-    
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorTest.java
new file mode 100644
index 0000000..9490f47
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DeadlockDetectorTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.distributed.internal.deadlock;
+
+import static org.junit.Assert.*;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class DeadlockDetectorTest {
+  
+  private volatile Set<Thread> stuckThreads;
+
+  @Before
+  public void setUp() throws Exception {
+    stuckThreads = Collections.synchronizedSet(new HashSet<Thread>());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    for (Thread thread: stuckThreads) {
+      thread.interrupt();
+      thread.join(20 * 1000);
+      if (thread.isAlive()) {
+        fail("Couldn't kill" + thread);
+      }
+    }
+
+    stuckThreads.clear();
+  }
+  
+  @Test
+  public void testNoDeadlocks() {
+    DeadlockDetector detector = new DeadlockDetector();
+    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
+    assertEquals(null, detector.findDeadlock());
+  }
+
+  /**
+   * Test that the deadlock detector will find deadlocks
+   * that are reported by the {@link DependencyMonitorManager}
+   */
+  @Test
+  public void testProgrammaticDependencies() throws Exception {
+    final CountDownLatch latch = new CountDownLatch(1);
+    MockDependencyMonitor mockDependencyMonitor = new MockDependencyMonitor();
+    DependencyMonitorManager.addMonitor(mockDependencyMonitor);
+    
+    Thread thread1 = startAThread(latch);
+    Thread thread2 = startAThread(latch);
+    
+    String resource1 = "one";
+    String resource2 = "two";
+    
+    mockDependencyMonitor.addDependency(thread1, resource1);
+    mockDependencyMonitor.addDependency(resource1, thread2);
+    mockDependencyMonitor.addDependency(thread2, resource2);
+    mockDependencyMonitor.addDependency(resource2, thread1);
+
+    DeadlockDetector detector = new DeadlockDetector();
+    detector.addDependencies(DeadlockDetector.collectAllDependencies("here"));
+    LinkedList<Dependency> deadlocks = detector.findDeadlock();
+
+    System.out.println("deadlocks=" + deadlocks);
+
+    assertEquals(4, deadlocks.size());
+
+    latch.countDown();
+    DependencyMonitorManager.removeMonitor(mockDependencyMonitor);
+  }
+
+  private Thread startAThread(final CountDownLatch latch) {
+    Thread thread = new Thread() {
+      @Override
+      public void run() {
+        try {
+          latch.await();
+        } catch (InterruptedException ignore) {
+        }
+      }
+    };
+
+    thread.start();
+    
+    return thread;
+  }
+  
+  /**
+   * A fake dependency monitor.
+   */
+  private static class MockDependencyMonitor implements DependencyMonitor {
+    
+    Set<Dependency<Thread, Serializable>> blockedThreads = new HashSet<>();
+    Set<Dependency<Serializable, Thread>> held = new HashSet<>();
+
+    @Override
+    public Set<Dependency<Thread, Serializable>> getBlockedThreads(Thread[] allThreads) {
+      return blockedThreads;
+    }
+
+    public void addDependency(String resource, Thread thread) {
+      held.add(new Dependency<>(resource, thread));
+    }
+
+    public void addDependency(Thread thread, String resource) {
+      blockedThreads.add(new Dependency<>(thread, resource));
+    }
+
+    @Override
+    public Set<Dependency<Serializable, Thread>> getHeldResources(Thread[] allThreads) {
+      return held;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DependencyGraphJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DependencyGraphJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DependencyGraphJUnitTest.java
index b93ea09..93f47f0 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DependencyGraphJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/DependencyGraphJUnitTest.java
@@ -16,23 +16,21 @@
  */
 package com.gemstone.gemfire.distributed.internal.deadlock;
 
+import static org.junit.Assert.*;
+
 import java.util.HashSet;
 import java.util.Set;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class DependencyGraphJUnitTest extends TestCase {
-  
+public class DependencyGraphJUnitTest {
+
+  @Test
   public void testFindCycle() {
-    
     DependencyGraph graph = new DependencyGraph();
     graph.addEdge(new Dependency("A", "B"));
     graph.addEdge(new Dependency("A", "F"));
@@ -47,9 +45,9 @@ public class DependencyGraphJUnitTest extends TestCase {
     expected.add(new Dependency("E", "A"));
     assertEquals(expected, new HashSet(graph.findCycle()));
   }
-  
+
+  @Test
   public void testSubGraph() {
-    
     DependencyGraph graph = new DependencyGraph();
     graph.addEdge(new Dependency("A", "B"));
     graph.addEdge(new Dependency("B", "C"));
@@ -68,7 +66,8 @@ public class DependencyGraphJUnitTest extends TestCase {
     DependencyGraph sub2 = graph.getSubGraph("E");
     assertEquals(null, sub2.findCycle());
   }
-  
+
+  @Test
   public void testTwoPaths() {
     DependencyGraph graph = new DependencyGraph();
     graph.addEdge(new Dependency("A", "B"));
@@ -78,7 +77,8 @@ public class DependencyGraphJUnitTest extends TestCase {
     
     assertEquals(null, graph.findCycle());
   }
-  
+
+  @Test
   public void testEmptySet() {
     DependencyGraph graph = new DependencyGraph();
     assertEquals(null, graph.findCycle());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/UnsafeThreadLocalJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/UnsafeThreadLocalJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/UnsafeThreadLocalJUnitTest.java
index 7c7a7fc..41e2ff7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/UnsafeThreadLocalJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/deadlock/UnsafeThreadLocalJUnitTest.java
@@ -16,45 +16,62 @@
  */
 package com.gemstone.gemfire.distributed.internal.deadlock;
 
+import static org.junit.Assert.*;
+
 import java.util.concurrent.CountDownLatch;
 
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- * 
- */
 @Category(UnitTest.class)
-public class UnsafeThreadLocalJUnitTest extends TestCase {
+public class UnsafeThreadLocalJUnitTest {
+
+  private static final long INTERVAL = 10;
+
+  private volatile boolean sleep;
+
+  @Before
+  public void setUp() throws Exception {
+    sleep = true;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    sleep = false;
+  }
 
   /**
    * Test that we can get the value of a thread local from another thread.
-   * 
-   * @throws InterruptedException
    */
+  @Test
   public void test() throws InterruptedException {
-    final UnsafeThreadLocal<String> utl = new UnsafeThreadLocal<String>();
+    final UnsafeThreadLocal<String> unsafeThreadLocal = new UnsafeThreadLocal<String>();
     final CountDownLatch localSet = new CountDownLatch(1);
 
     Thread test = new Thread() {
       public void run() {
-        utl.set("hello");
+        unsafeThreadLocal.set("hello");
         localSet.countDown();
         try {
-          Thread.sleep(100 * 1000);
+          while (sleep) {
+            Thread.sleep(INTERVAL);
+          }
         } catch (InterruptedException e) {
-          e.printStackTrace();
+          throw new AssertionError(e);
         }
       }
     };
+
     test.setDaemon(true);
     test.start();
+
     localSet.await();
-    assertEquals("hello", utl.get(test));
-    assertEquals(null, utl.get(Thread.currentThread()));
-  }
 
+    assertEquals("hello", unsafeThreadLocal.get(test));
+    assertEquals(null, unsafeThreadLocal.get(Thread.currentThread()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
index dd7b432..5caf997 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
@@ -1,54 +1,42 @@
+/*
+ * 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.distributed.internal.membership;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
-import java.io.IOException;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
-import java.util.Timer;
 
-import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/*
- * 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.
- */
 @Category(UnitTest.class)
 public class NetViewJUnitTest {
-  List<InternalDistributedMember> members;
+
+  private List<InternalDistributedMember> members;
   
   @Before
   public void initMembers() throws Exception {
@@ -205,7 +193,6 @@ public class NetViewJUnitTest {
   
   /**
    * Test that failed weight calculations are correctly performed.  See bug #47342
-   * @throws Exception
    */
   @Test
   public void testFailedWeight() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
index 606ae1a..7eef594 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
@@ -16,32 +16,32 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.net.InetAddress;
 
 import org.jgroups.util.UUID;
-import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-@Category(UnitTest.class)
+@Category({ UnitTest.class, SecurityTest.class })
 public class GMSMemberJUnitTest {
 
   @Test
   public void testEqualsNotSameType() {
     GMSMember member = new GMSMember();
-    Assert.assertFalse(member.equals("Not a GMSMember"));
+    assertFalse(member.equals("Not a GMSMember"));
   }
   
   @Test
   public void testEqualsIsSame() {
     GMSMember member = new GMSMember();
-    Assert.assertTrue(member.equals(member));
+    assertTrue(member.equals(member));
   }
   
   @Test
@@ -49,7 +49,7 @@ public class GMSMemberJUnitTest {
     GMSMember member = new GMSMember();
     UUID uuid = new UUID(0, 0);
     member.setUUID(uuid);
-    Assert.assertEquals(0, member.compareTo(member));
+    assertEquals(0, member.compareTo(member));
   }
   
   private GMSMember createGMSMember(byte[] inetAddress, int viewId, long msb, long lsb) {
@@ -66,84 +66,85 @@ public class GMSMemberJUnitTest {
   public void testCompareToInetAddressIsLongerThan() {
     GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1}, 1, 1, 1);
-    Assert.assertEquals(1, member1.compareTo(member2));
+    assertEquals(1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToInetAddressIsShorterThan() {
     GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
-    Assert.assertEquals(-1, member1.compareTo(member2));
+    assertEquals(-1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToInetAddressIsGreater() {
     GMSMember member1 = createGMSMember(new byte[] {1, 2, 1, 1, 1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
-    Assert.assertEquals(1, member1.compareTo(member2));
+    assertEquals(1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToInetAddressIsLessThan() {
     GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1, 2, 1, 1, 1}, 1, 1, 1);
-    Assert.assertEquals(-1, member1.compareTo(member2));
+    assertEquals(-1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToMyViewIdLarger() {
     GMSMember member1 = createGMSMember(new byte[] {1}, 2, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 1);
-    Assert.assertEquals(1, member1.compareTo(member2));
+    assertEquals(1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToTheirViewIdLarger() {
     GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1}, 2, 1, 1);
-    Assert.assertEquals(-1, member1.compareTo(member2));
+    assertEquals(-1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToMyMSBLarger() {
     GMSMember member1 = createGMSMember(new byte[] {1}, 1, 2, 1);
     GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 1);
-    Assert.assertEquals(1, member1.compareTo(member2));
+    assertEquals(1, member1.compareTo(member2));
   }
 
   @Test
   public void testCompareToTheirMSBLarger() {
     GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1}, 1, 2, 1);
-    Assert.assertEquals(-1, member1.compareTo(member2));
+    assertEquals(-1, member1.compareTo(member2));
   }
 
   @Test
   public void testCompareToMyLSBLarger() {
     GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 2);
     GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 1);
-    Assert.assertEquals(1, member1.compareTo(member2));
+    assertEquals(1, member1.compareTo(member2));
   }
   
   @Test
   public void testCompareToTheirLSBLarger() {
     GMSMember member1 = createGMSMember(new byte[] {1}, 1, 1, 1);
     GMSMember member2 = createGMSMember(new byte[] {1}, 1, 1, 2);
-    Assert.assertEquals(-1, member1.compareTo(member2));
+    assertEquals(-1, member1.compareTo(member2));
   }
 
-  
-  //Makes sure a NPE is not thrown
+  /**
+   * Makes sure a NPE is not thrown
+   */
   @Test
   public void testNoNPEWhenSetAttributesWithNull() {
     GMSMember member = new GMSMember();
     member.setAttributes(null);
     MemberAttributes attrs = member.getAttributes(); 
     MemberAttributes invalid = MemberAttributes.INVALID;
-    Assert.assertEquals(attrs.getVmKind(), invalid.getVmKind());
-    Assert.assertEquals(attrs.getPort(), invalid.getPort());
-    Assert.assertEquals(attrs.getVmViewId(), invalid.getVmViewId());
-    Assert.assertEquals(attrs.getName(), invalid.getName());
+    assertEquals(attrs.getVmKind(), invalid.getVmKind());
+    assertEquals(attrs.getPort(), invalid.getPort());
+    assertEquals(attrs.getVmViewId(), invalid.getVmViewId());
+    assertEquals(attrs.getName(), invalid.getName());
   }
   
   @Test
@@ -151,7 +152,7 @@ public class GMSMemberJUnitTest {
     GMSMember member = new GMSMember();
     UUID uuid = new UUID(0, 0);
     member.setUUID(uuid);
-    Assert.assertNull(member.getUUID());
+    assertNull(member.getUUID());
   }
   
   @Test
@@ -159,6 +160,6 @@ public class GMSMemberJUnitTest {
     GMSMember member = new GMSMember();
     UUID uuid = new UUID(1, 1);
     member.setUUID(uuid);
-    Assert.assertNotNull(member.getUUID());
+    assertNotNull(member.getUUID());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticatorJUnitTest.java
index 8246a43..022d1c7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticatorJUnitTest.java
@@ -16,6 +16,18 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.auth;
 
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.security.Principal;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
@@ -24,45 +36,34 @@ import com.gemstone.gemfire.security.AuthInitialize;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.Authenticator;
 import com.gemstone.gemfire.security.GemFireSecurityException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import java.security.Principal;
-import java.util.Properties;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.junit.Assert.*;
-
-@Category(UnitTest.class)
+@Category({ UnitTest.class, SecurityTest.class })
 public class GMSAuthenticatorJUnitTest {
 
-  static String prefix = "com.gemstone.gemfire.distributed.internal.membership.gms.auth.GMSAuthenticatorJUnitTest$";
+  private String prefix;
+  private Properties props;
+  private Services services;
+  private GMSAuthenticator authenticator;
+  private DistributedMember member;
 
-  Properties originalSystemProps = null;
-  Properties props = null;
-  Services services = null;
-  GMSAuthenticator authenticator = null;
-  DistributedMember member = null;
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
   @Before
   public void setUp() throws Exception {
-    originalSystemProps = System.getProperties();
+    prefix = getClass().getName() + "$";
     props = new Properties();
     authenticator = new GMSAuthenticator();
+
     services = mock(Services.class);
     InternalLogWriter securityLog = mock(InternalLogWriter.class);
-    when(services.getSecurityLogWriter()).thenReturn(securityLog);
+    when(services.getSecurityLogWriter()).thenReturn(mock(InternalLogWriter.class));
+
     authenticator.init(services);
-    member = mock(DistributedMember.class);
-  }
 
-  @After
-  public void tearDown() throws Exception {
-    System.setProperties(originalSystemProps);
+    member = mock(DistributedMember.class);
   }
 
   @Test
@@ -126,12 +127,12 @@ public class GMSAuthenticatorJUnitTest {
     verifyNegativeGetCredential(props, "expected get credential error");
   }
 
-  void verifyNegativeGetCredential(Properties props, String expectedError) throws Exception {
+  private void verifyNegativeGetCredential(Properties props, String expectedError) throws Exception {
     try {
       authenticator.getCredentials(member, props);
       fail("should catch: " + expectedError);
-    } catch (GemFireSecurityException e) {
-      assertTrue(e.getMessage().startsWith(expectedError));
+    } catch (GemFireSecurityException expected) {
+      assertTrue(expected.getMessage().startsWith(expectedError));
     }
   }
 
@@ -149,8 +150,8 @@ public class GMSAuthenticatorJUnitTest {
 
   @Test
   public void testAuthenticatorWithNoAuth() throws Exception {
-      String result = authenticator.authenticate(member, props, props, member);
-      assertNull(result);
+    String result = authenticator.authenticate(member, props, props, member);
+    assertNull(result);
   }
 
   @Test
@@ -195,27 +196,29 @@ public class GMSAuthenticatorJUnitTest {
     assertTrue(result, result.startsWith(expectedError));
   }
 
-  // ----------------------------------------
-  //           Test AuthInitialize
-  // ----------------------------------------
-
-  public static class TestAuthInit1 implements AuthInitialize {
+  private static class TestAuthInit1 implements AuthInitialize {
     public static AuthInitialize create() {
       return null;
     }
+    @Override
     public void init(LogWriter systemLogger, LogWriter securityLogger) throws AuthenticationFailedException {
     }
-    public Properties getCredentials(Properties props, DistributedMember server, boolean isPeer)
-        throws AuthenticationFailedException {
+    @Override
+    public Properties getCredentials(Properties props, DistributedMember server, boolean isPeer) throws AuthenticationFailedException {
       throw new AuthenticationFailedException("expected get credential error");
     }
+    @Override
     public void close() {
     }
   }
 
-  public static class TestAuthInit2 extends TestAuthInit1 {
-    static TestAuthInit2 instance = null;
-    static int createCount = 0;
+  private static class TestAuthInit2 extends TestAuthInit1 {
+
+    private static TestAuthInit2 instance = null;
+    private static int createCount = 0;
+
+    boolean closed = false;
+
     public static void setAuthInitialize(TestAuthInit2 auth) {
       instance = auth;
     }
@@ -223,11 +226,11 @@ public class GMSAuthenticatorJUnitTest {
       createCount ++;
       return instance;
     }
-    public Properties getCredentials(Properties props, DistributedMember server, boolean isPeer)
-        throws AuthenticationFailedException {
+    @Override
+    public Properties getCredentials(Properties props, DistributedMember server, boolean isPeer) throws AuthenticationFailedException {
       return props;
     }
-    boolean closed = false;
+    @Override
     public void close() {
       closed = true;
     }
@@ -239,59 +242,66 @@ public class GMSAuthenticatorJUnitTest {
     }
   }
 
-  public static class TestAuthInit3 extends TestAuthInit1 {
+  // used by reflection by test
+  private static class TestAuthInit3 extends TestAuthInit1 {
     public static AuthInitialize create() {
       return new TestAuthInit3();
     }
+    @Override
     public void init(LogWriter systemLogger, LogWriter securityLogger) throws AuthenticationFailedException {
       throw new AuthenticationFailedException("expected init error");
     }
   }
 
-  public static class TestAuthInit4 extends TestAuthInit1 {
+  private static class TestAuthInit4 extends TestAuthInit1 {
     public static AuthInitialize create() {
       return new TestAuthInit4();
     }
   }
 
-  // ----------------------------------------
-  //          Test Authenticator
-  // ----------------------------------------
-
-  public static class TestAuthenticator1 implements Authenticator {
+  private static class TestAuthenticator1 implements Authenticator {
     public static Authenticator create() {
       return null;
     }
+    @Override
     public void init(Properties securityProps, LogWriter systemLogger, LogWriter securityLogger) throws AuthenticationFailedException {
     }
+    @Override
     public Principal authenticate(Properties props, DistributedMember member) throws AuthenticationFailedException {
       return null;
     }
+    @Override
     public void close() {
     }
   }
 
-  public static class TestAuthenticator2 extends TestAuthenticator1 {
+  private static class TestAuthenticator2 extends TestAuthenticator1 {
     public static Authenticator create() {
       return new TestAuthenticator2();
     }
+    @Override
     public void init(Properties securityProps, LogWriter systemLogger, LogWriter securityLogger) throws AuthenticationFailedException {
       throw new AuthenticationFailedException("expected init error");
     }
   }
 
-  public static class TestAuthenticator3 extends TestAuthenticator1 {
+  private static class TestAuthenticator3 extends TestAuthenticator1 {
     public static Authenticator create() {
       return new TestAuthenticator3();
     }
+    @Override
     public Principal authenticate(Properties props, DistributedMember member) throws AuthenticationFailedException {
       throw new AuthenticationFailedException("expected authenticate error");
     }
   }
 
-  public static class TestAuthenticator4 extends TestAuthenticator1 {
-    static Authenticator instance = null;
-    static int createCount = 0;
+  private static class TestAuthenticator4 extends TestAuthenticator1 {
+
+    private static Authenticator instance = null;
+    private static int createCount = 0;
+
+    private boolean closed = false;
+
     public static void setAuthenticator(Authenticator auth) {
       instance = auth;
     }
@@ -299,10 +309,11 @@ public class GMSAuthenticatorJUnitTest {
       createCount ++;
       return instance;
     }
+    @Override
     public Principal authenticate(Properties props, DistributedMember member) throws AuthenticationFailedException {
       return null;
     }
-    boolean closed = false;
+    @Override
     public void close() {
       closed = true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index 2d042fc..00d9d05 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -16,6 +16,20 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.locator;
 
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetAddress;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.distributed.Locator;
@@ -30,28 +44,12 @@ import com.gemstone.gemfire.distributed.internal.membership.MemberFactory;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.OSProcess;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.ObjectOutputStream;
-import java.net.InetAddress;
-import java.util.Properties;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class GMSLocatorRecoveryJUnitTest {
 
   File tempStateFile = null;


[05/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/BackupInspectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/BackupInspectorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/BackupInspectorJUnitTest.java
index 342dcf5..dbb58bc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/BackupInspectorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/BackupInspectorJUnitTest.java
@@ -16,10 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache.persistence;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 import java.io.File;
 import java.io.IOException;
@@ -28,11 +25,13 @@ import java.util.Set;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
 
 import com.gemstone.gemfire.internal.FileUtil;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * TODO: fails when running integrationTest from gradle command-line on Windows 7
@@ -45,10 +44,10 @@ com.gemstone.gemfire.internal.cache.DiskRegionJUnitTest > testAssertionErrorIfMi
         at com.gemstone.gemfire.internal.cache.DiskRegionJUnitTest.testAssertionErrorIfMissingOplog(DiskRegionJUnitTest.java:2630)
  * 
  * Tests for the BackupInspector.
- * 
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class BackupInspectorJUnitTest {
+
   private static final String UNIX_INCREMENTAL_BACKUP_SCRIPT = "#!/bin/bash -e\ncd `dirname $0`\n\n#Restore a backup of gemfire persistent data to the location it was backed up\n#from.\n#This script will refuse to restore if the original data still exists.\n\n#This script was automatically generated by the gemfire backup utility.\n\n#Test for existing originals. If they exist, do not restore the backup.\ntest -e '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3/BACKUPbar.if' && echo 'Backup not restored. Refusing to overwrite /Users/rholmes/Projects/gemfire/test/cacheserver/disk3/BACKUPbar.if' && exit 1 \ntest -e '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1/BACKUPfoo.if' && echo 'Backup not restored. Refusing to overwrite /Users/rholmes/Projects/gemfire/test/cacheserver/disk1/BACKUPfoo.if' && exit 1 \n\n#Restore data\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3'\ncp -rp 'diskstores/bar/dir0'/* '/Users/rholmes/Projects/gemfire/test/cacheserver/di
 sk3'\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk4'\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1'\ncp -rp 'diskstores/foo/dir0'/* '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1'\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk2'\n\n#Incremental backup.  Restore baseline originals from a previous backup.\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/bar/dir0/BACKUPbar_1.drf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3/BACKUPbar_1.drf'\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/foo/dir1/BACKUPfoo_2.crf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk2/BACKUPfoo_2.crf'\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/foo/dir1/BACKUPfoo_2.drf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk2/BACKUPfoo_2.drf'\ncp -p '/Users
 /rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/bar/dir1/BACKUPbar_2.drf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk4/BACKUPbar_2.drf'\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/foo/dir0/BACKUPfoo_1.crf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1/BACKUPfoo_1.crf'\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/bar/dir1/BACKUPbar_2.crf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk4/BACKUPbar_2.crf'\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/bar/dir0/BACKUPbar_1.crf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3/BACKUPbar_1.crf'\ncp -p '/Users/rholmes/Projects/gemfire/test/backup/2012-05-24-09-42/rholmes_mbp_410_v1_56425/diskstores/foo/dir0/BACKUPfoo_1.drf' '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1/BACKUPfo
 o_1.drf'";
   private static final String UNIX_FULL_BACKUP_SCRIPT = "#!/bin/bash -e\ncd `dirname $0`\n\n#Restore a backup of gemfire persistent data to the location it was backed up\n#from.\n#This script will refuse to restore if the original data still exists.\n\n#This script was automatically generated by the gemfire backup utility.\n\n#Test for existing originals. If they exist, do not restore the backup.\ntest -e '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3/BACKUPbar.if' && echo 'Backup not restored. Refusing to overwrite /Users/rholmes/Projects/gemfire/test/cacheserver/disk3/BACKUPbar.if' && exit 1 \ntest -e '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1/BACKUPfoo.if' && echo 'Backup not restored. Refusing to overwrite /Users/rholmes/Projects/gemfire/test/cacheserver/disk1/BACKUPfoo.if' && exit 1 \n\n#Restore data\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3'\ncp -rp 'diskstores/bar/dir0'/* '/Users/rholmes/Projects/gemfire/test/cacheserver/disk3'\nm
 kdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk4'\ncp -rp 'diskstores/bar/dir1'/* '/Users/rholmes/Projects/gemfire/test/cacheserver/disk4'\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1'\ncp -rp 'diskstores/foo/dir0'/* '/Users/rholmes/Projects/gemfire/test/cacheserver/disk1'\nmkdir -p '/Users/rholmes/Projects/gemfire/test/cacheserver/disk2'\ncp -rp 'diskstores/foo/dir1'/* '/Users/rholmes/Projects/gemfire/test/cacheserver/disk2'";
   
@@ -85,12 +84,15 @@ public class BackupInspectorJUnitTest {
    */
   private File fullBackupDir = null;
 
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
   /**
    * Set up data for all tests.
    */
   @Before
   public void setUp() throws Exception {
-    File tempDir = new File(System.getProperty("java.io.tmpdir"));
+    File tempDir = temporaryFolder.newFolder();
     
     /*
      * Create an incremental backup on the file system.
@@ -132,27 +134,10 @@ public class BackupInspectorJUnitTest {
   }
 
   /**
-   * Delete leftover data for all tests.
-   */
-  @After
-  public void tearDown() throws Exception {
-    
-    if((null != this.incrementalBackupDir) && (this.incrementalBackupDir.exists())) {
-      FileUtil.delete(this.incrementalBackupDir);
-    }
-    
-    if((null != this.fullBackupDir) && (this.fullBackupDir.exists())) {
-      FileUtil.delete(fullBackupDir);
-    }
-  }
-
-  /**
    * Tests that an IOException is thrown for a non-existent restore script.
-   * @throws Exception
    */
   @Test
-  public void testNonExistentScriptFile() 
-  throws Exception {
+  public void testNonExistentScriptFile() throws Exception {
     boolean ioexceptionThrown = false;
     
     try {
@@ -169,8 +154,7 @@ public class BackupInspectorJUnitTest {
    * Tests copy lines for windows.
    * @param inspector a BackupInspector.
    */
-  private void testIncrementalBackupScriptForWindows(BackupInspector inspector) 
-  throws Exception {
+  private void testIncrementalBackupScriptForWindows(BackupInspector inspector) throws Exception {
     assertEquals(WINDOWS_COPY_FROM_1,inspector.getCopyFromForOplogFile(OPLOG_FILENAME_1));
     assertEquals(WINDOWS_COPY_TO_1,inspector.getCopyToForOplogFile(OPLOG_FILENAME_1));    
     assertEquals(WINDOWS_COPY_FROM_2,inspector.getCopyFromForOplogFile(OPLOG_FILENAME_2));
@@ -182,10 +166,8 @@ public class BackupInspectorJUnitTest {
   /**
    * Tests copy lines for unix.
    * @param inspector a BackupInspector. 
-   * @throws Exception
    */
-  private void testIncrementalBackupScriptForUnix(BackupInspector inspector) 
-  throws Exception {    
+  private void testIncrementalBackupScriptForUnix(BackupInspector inspector) throws Exception {
     assertEquals(UNIX_COPY_FROM_1,inspector.getCopyFromForOplogFile(OPLOG_FILENAME_1));
     assertEquals(UNIX_COPY_TO_1,inspector.getCopyToForOplogFile(OPLOG_FILENAME_1));    
     assertEquals(UNIX_COPY_FROM_2,inspector.getCopyFromForOplogFile(OPLOG_FILENAME_2));
@@ -196,11 +178,9 @@ public class BackupInspectorJUnitTest {
   
   /**
    * Tests that the parser succeeds for an incremental backup restore script.
-   * @throws Exception
    */
   @Test
-  public void testIncrementalBackupScript() 
-  throws Exception {
+  public void testIncrementalBackupScript() throws Exception {
     BackupInspector inspector = BackupInspector.createInspector(incrementalBackupDir);
     
     assertTrue(inspector.isIncremental());
@@ -222,11 +202,9 @@ public class BackupInspectorJUnitTest {
   
   /**
    * Tests that the parser works with a full backup restore script.
-   * @throws Exception
    */
   @Test
-  public void testFullBackupScript() 
-  throws Exception {
+  public void testFullBackupScript() throws Exception {
     BackupInspector inspector = BackupInspector.createInspector(fullBackupDir);
     assertFalse(inspector.isIncremental());
     assertTrue(inspector.getIncrementalOplogFileNames().isEmpty());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/query/TemporaryResultSetFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/query/TemporaryResultSetFactoryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/query/TemporaryResultSetFactoryJUnitTest.java
index 3940a82..adfbad4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/query/TemporaryResultSetFactoryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/query/TemporaryResultSetFactoryJUnitTest.java
@@ -16,32 +16,29 @@
  */
 package com.gemstone.gemfire.internal.cache.persistence.query;
 
+import static org.junit.Assert.*;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator;
-import com.gemstone.gemfire.internal.cache.persistence.query.IndexMap;
 import com.gemstone.gemfire.internal.cache.persistence.query.IndexMap.IndexEntry;
 import com.gemstone.gemfire.internal.cache.persistence.query.mock.IndexMapImpl;
 import com.gemstone.gemfire.internal.cache.persistence.query.mock.NaturalComparator;
 import com.gemstone.gemfire.internal.cache.persistence.query.mock.Pair;
 import com.gemstone.gemfire.internal.cache.persistence.query.mock.PairComparator;
-import com.gemstone.gemfire.internal.cache.persistence.query.ResultBag;
-import com.gemstone.gemfire.internal.cache.persistence.query.ResultSet;
-import com.gemstone.gemfire.internal.cache.persistence.query.TemporaryResultSetFactory;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class TemporaryResultSetFactoryJUnitTest extends TestCase {
+public class TemporaryResultSetFactoryJUnitTest {
 
+  @Test
   public void testSortedResultSet() {
     ResultSet set = new TemporaryResultSetFactory().getSortedResultSet(null, false);
     set.add(1);
@@ -51,9 +48,9 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     set.add(2);
     
     assertItrEquals(set.iterator(), 1,2, 3, 4);
-    
   }
-  
+
+  @Test
   public void testSortedResultBag() {
     ResultBag set = new TemporaryResultSetFactory().getSortedResultBag(null, false);
     set.add(1);
@@ -61,11 +58,11 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     set.add(4);
     set.add(3);
     set.add(2);
-    
-    
+
     assertItrEquals(set.iterator(), 1,2, 2,3, 4);
   }
-  
+
+  @Test
   public void testResultList() {
     ResultList set = new TemporaryResultSetFactory().getResultList();
     set.add(1);
@@ -73,12 +70,12 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     set.add(4);
     set.add(3);
     set.add(2);
-    
-    
+
     assertItrEquals(set.iterator(), 1,2, 4,3, 2);
     assertItrEquals(set.iterator(2), 4,3, 2);
   }
-  
+
+  @Test
   public void testIndexMap() {
     IndexMap map = new IndexMapImpl();
     TreeMap expected = new TreeMap(new PairComparator(new NaturalComparator(), new NaturalComparator()));
@@ -87,7 +84,6 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     put("i4", "r4", "v4", map, expected);
     put("i2", "r5", "v5", map, expected);
     
-    
     assertItrEquals(map.keyIterator(), "r1", "r2", "r5", "r4");
     assertItrEquals(map.keyIterator("i2", true, "i3", true), "r2", "r5");
     assertItrEquals(map.keyIterator("i2", true, "i2", true), "r2", "r5");
@@ -97,16 +93,12 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     assertEntryEquals(map.iterator("i2", true, "i4", true), expected.tailMap(new Pair("i1", "r2")));
   }
 
-  private void put(String ikey, String rkey, String value, IndexMap map,
-      Map expected) {
+  private void put(String ikey, String rkey, String value, IndexMap map, Map expected) {
     map.put(ikey, rkey, value);
     expected.put(new Pair(ikey, rkey), value);
-    
   }
 
-  private void assertItrEquals(
-      CloseableIterator<CachedDeserializable> iterator, Object ... values) {
-    
+  private void assertItrEquals(CloseableIterator<CachedDeserializable> iterator, Object ... values) {
     ArrayList actual = new ArrayList();
     
     while(iterator.hasNext()) {
@@ -114,12 +106,9 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     }
     
     assertEquals(Arrays.asList(values), actual);
-    
   }
   
-  private void assertEntryEquals(
-      CloseableIterator<IndexEntry> closeableIterator, Map expected) {
-    
+  private void assertEntryEquals(CloseableIterator<IndexEntry> closeableIterator, Map expected) {
     LinkedHashMap actual = new LinkedHashMap();
     
     while(closeableIterator.hasNext()) {
@@ -131,6 +120,5 @@ public class TemporaryResultSetFactoryJUnitTest extends TestCase {
     }
     
     assertEquals(expected, actual);
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/region/entry/RegionEntryFactoryBuilderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/region/entry/RegionEntryFactoryBuilderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/region/entry/RegionEntryFactoryBuilderJUnitTest.java
index 09fde8a..fcbd140 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/region/entry/RegionEntryFactoryBuilderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/region/entry/RegionEntryFactoryBuilderJUnitTest.java
@@ -16,7 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache.region.entry;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.junit.Assert.*;
+
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
 import org.junit.Before;
@@ -24,7 +25,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
-import static org.junit.Assert.assertEquals;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 @RunWith(JUnitParamsRunner.class)
@@ -77,9 +78,7 @@ public class RegionEntryFactoryBuilderJUnitTest {
       "VersionedStatsDiskLRURegionEntryHeapFactory,true,true,true,true,false",
       "VersionedStatsDiskLRURegionEntryOffHeapFactory,true,true,true,true,true"
   })
-  public void testRegionEntryFactoryUnitTest(String factoryName, boolean enableStats, boolean enableLRU, boolean enableDisk,
-      boolean enableVersioning, boolean enableOffHeap) {
-    assertEquals(factoryName,
-        regionEntryFactoryBuilder.getRegionEntryFactoryOrNull(enableStats, enableLRU, enableDisk, enableVersioning, enableOffHeap).getClass().getSimpleName());
+  public void testRegionEntryFactoryUnitTest(String factoryName, boolean enableStats, boolean enableLRU, boolean enableDisk,boolean enableVersioning, boolean enableOffHeap) {
+    assertEquals(factoryName, regionEntryFactoryBuilder.getRegionEntryFactoryOrNull(enableStats, enableLRU, enableDisk, enableVersioning, enableOffHeap).getClass().getSimpleName());
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
index 40d741e..3b0157f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
@@ -112,7 +112,7 @@ public class Bug37805DUnitTest extends DistributedTestCase{
       }
     }
     //assertNull(rootRegions);
-    //assertEquals(0,((Collection)CacheServerTestUtil.getCache().rootRegions()).size());
+    //assertIndexDetailsEquals(0,((Collection)CacheServerTestUtil.getCache().rootRegions()).size());
   }
   
   private Pool getClientPool(String host, int server1Port,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
index 52f416a..f03f5fa 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
@@ -478,12 +478,12 @@ public class CacheServerTransactionsDUnitTest extends DistributedTestCase
     if (server.equals("server1")) {
       r1.invalidate(k1);
       assertNull(r1.getEntry(k1).getValue());
-      //assertEquals(r1.getEntry(k2).getValue(), server1_k2);
+      //assertIndexDetailsEquals(r1.getEntry(k2).getValue(), server1_k2);
     }
     else if (server.equals("server2")) {
       r1.invalidate(k1);
       assertNull(r1.getEntry(k1).getValue());
-      //assertEquals(r1.getEntry(k2).getValue(), server2_k3);
+      //assertIndexDetailsEquals(r1.getEntry(k2).getValue(), server2_k3);
     }
   }
 
@@ -495,12 +495,12 @@ public class CacheServerTransactionsDUnitTest extends DistributedTestCase
     if (server.equals("server1")) {
       r1.destroy(k1);
       assertNull(r1.getEntry(k1));
-      //assertEquals(r1.getEntry(k2).getValue(), server1_k2);
+      //assertIndexDetailsEquals(r1.getEntry(k2).getValue(), server1_k2);
     }
     else if (server.equals("server2")) {
       r1.destroy(k1);
       assertNull(r1.getEntry(k1));
-      //assertEquals(r1.getEntry(k2).getValue(), server2_k3);
+      //assertIndexDetailsEquals(r1.getEntry(k2).getValue(), server2_k3);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
index 42ff29a..5953b8a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
@@ -222,8 +222,8 @@ public class ClientHealthMonitorJUnitTest
 //     }
 //     connection1.putObject("region1", "key-1", "value-2", new EventID(new byte[] {1},1,3), null);
 //     this.system.getLogWriter().info("did put 2");
-//     assertEquals(1, s.getInt("currentClients"));
-//     assertEquals(1, s.getInt("currentClientConnections"));
+//     assertIndexDetailsEquals(1, s.getInt("currentClients"));
+//     assertIndexDetailsEquals(1, s.getInt("currentClientConnections"));
 
 //     // now lets see what happens when we close our connection
 //     // note we use a nasty close which just closes the socket instead
@@ -238,8 +238,8 @@ public class ClientHealthMonitorJUnitTest
 //                                       + s.getInt("currentClients")
 //                                       + " currentClientConnections="
 //                                       + s.getInt("currentClientConnections"));
-//       assertEquals(0, s.getInt("currentClients"));
-//       assertEquals(0, s.getInt("currentClientConnections"));
+//       assertIndexDetailsEquals(0, s.getInt("currentClients"));
+//       assertIndexDetailsEquals(0, s.getInt("currentClientConnections"));
 //     }
   }
  public void addExceptions() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
index 5415e86..d5111bc 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
@@ -623,7 +623,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
     };
     Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
     
-    // assertEquals(region1.getEntry(k1).getValue(), k1);
+    // assertIndexDetailsEquals(region1.getEntry(k1).getValue(), k1);
     wc = new WaitCriterion() {
       String excuse;
       public boolean done() {
@@ -648,8 +648,8 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
     };
     Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
     
-    // assertEquals(region1.getEntry(k2).getValue(), k2);
-    // assertEquals(region2.getEntry(k1).getValue(), k1);
+    // assertIndexDetailsEquals(region1.getEntry(k2).getValue(), k2);
+    // assertIndexDetailsEquals(region2.getEntry(k1).getValue(), k1);
     wc = new WaitCriterion() {
       String excuse;
       public boolean done() {
@@ -662,7 +662,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
     };
     Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
     
-    // assertEquals(region2.getEntry(k2).getValue(), k2);
+    // assertIndexDetailsEquals(region2.getEntry(k2).getValue(), k2);
   }
 
   /**
@@ -1103,7 +1103,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       assertNull(cache.getRegion(Region.SEPARATOR + regionName));
        verifyCacheClientProxyOnServer();
 
-      //assertEquals(1, bs.getAcceptor().getCacheClientNotifier().getClientProxies().size());
+      //assertIndexDetailsEquals(1, bs.getAcceptor().getCacheClientNotifier().getClientProxies().size());
     }
     catch (Exception ex) {
       ex.printStackTrace();
@@ -1215,7 +1215,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       };
       Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      // assertEquals(k1, r1.getEntry(k1).getValue());
+      // assertIndexDetailsEquals(k1, r1.getEntry(k1).getValue());
       wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
@@ -1228,7 +1228,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       };
       Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      // assertEquals(k2, r1.getEntry(k2).getValue());
+      // assertIndexDetailsEquals(k2, r1.getEntry(k2).getValue());
       wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
@@ -1241,7 +1241,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       };
       Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      // assertEquals(server_k1, r2.getEntry(k1).getValue());
+      // assertIndexDetailsEquals(server_k1, r2.getEntry(k1).getValue());
       wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
@@ -1254,7 +1254,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       };
       Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      // assertEquals(server_k2, r2.getEntry(k2).getValue());
+      // assertIndexDetailsEquals(server_k2, r2.getEntry(k2).getValue());
     }
     catch (Exception ex) {
       Assert.fail("failed while verifyUpdates()", ex);
@@ -1358,7 +1358,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       };
       Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      // assertEquals(server_k1, r2.getEntry(k1).getValue());
+      // assertIndexDetailsEquals(server_k1, r2.getEntry(k1).getValue());
       wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
@@ -1371,7 +1371,7 @@ public class ClientServerMiscDUnitTest extends CacheTestCase
       };
       Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      // assertEquals(server_k2, r2.getEntry(k2).getValue());
+      // assertIndexDetailsEquals(server_k2, r2.getEntry(k2).getValue());
     }
     catch (Exception ex) {
       Assert.fail("failed while verifyUpdatesOnRegion2()", ex);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
index 2aed3a7..d62addf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
@@ -553,7 +553,7 @@ public class ConflationDUnitTest extends DistributedTestCase
       }
     };
     Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-    // assertEquals("creates", 2, counterCreate);
+    // assertIndexDetailsEquals("creates", 2, counterCreate);
     
     ev = new WaitCriterion() {
       public boolean done() {
@@ -566,7 +566,7 @@ public class ConflationDUnitTest extends DistributedTestCase
     };
     Wait.waitForCriterion(ev, 60 * 1000, 200, true);
     
-    // assertEquals("destroys", 2, counterDestroy);
+    // assertIndexDetailsEquals("destroys", 2, counterDestroy);
     // assertTrue("updates", 20000 >= counterUpdate);
     ev = new WaitCriterion() {
       public boolean done() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
index 90bb911..bd4f94a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
@@ -537,7 +537,7 @@ public class DurableClientReconnectDUnitTest extends DistributedTestCase
     // Verify that it is durable and its properties are correct
     assertTrue(proxy.isDurable());
     assertEquals("DurableClientReconnectDUnitTest_client", proxy.getDurableId());
-//    assertEquals(60, proxy.getDurableTimeout());
+//    assertIndexDetailsEquals(60, proxy.getDurableTimeout());
     }catch (Exception e){
       Assert.fail("test failed due to" , e);
     }    

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/FilterProfileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/FilterProfileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/FilterProfileJUnitTest.java
index 1314d42..5d53dd2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/FilterProfileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/FilterProfileJUnitTest.java
@@ -16,10 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.util.Collections;
 
@@ -36,8 +34,8 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class FilterProfileJUnitTest {
 
-  LocalRegion mockRegion;
-  FilterProfile fprofile;
+  private LocalRegion mockRegion;
+  private FilterProfile fprofile;
   
   @Before
   public void setUp() {
@@ -48,7 +46,6 @@ public class FilterProfileJUnitTest {
     fprofile = new FilterProfile(mockRegion);
   }
 
-  
   @Test
   public void testUnregisterKey() {
     unregisterKey(false);
@@ -102,8 +99,7 @@ public class FilterProfileJUnitTest {
     assertTrue(isClientInterested);
     fprofile.unregisterClientInterest(clientId, "Object1234", InterestType.KEY);
     fprofile.unregisterClientInterest(clientId, "Object4567", InterestType.KEY);
-    assertFalse("still has this interest: " + fprofile.getKeysOfInterestFor(clientId),
-        fprofile.hasKeysOfInterestFor(clientId, inv));
+    assertFalse("still has this interest: " + fprofile.getKeysOfInterestFor(clientId), fprofile.hasKeysOfInterestFor(clientId, inv));
   }
 
   @Test
@@ -134,7 +130,6 @@ public class FilterProfileJUnitTest {
     assertFalse(fprofile.hasKeysOfInterestFor(clientId, inv));
   }
 
-  
   @Test
   public void testUnregisterRegex() {
     unregisterRegex(false);
@@ -194,7 +189,6 @@ public class FilterProfileJUnitTest {
     assertFalse(fprofile.hasRegexInterestFor(clientId, inv));
   }
 
-
   @Test
   public void testUnregisterAllKeys() {
     unregisterAllKeys(false);
@@ -237,7 +231,7 @@ public class FilterProfileJUnitTest {
     unregisterFilterClass(inv, true);
   }
 
-  public void unregisterFilterClass(boolean inv, boolean twoClients) {
+  private void unregisterFilterClass(boolean inv, boolean twoClients) {
     String clientId = "client";
     fprofile.registerClientInterest(clientId, "com.gemstone.gemfire.internal.cache.tier.sockets.TestFilter", InterestType.FILTER_CLASS, inv);
     if (twoClients) {
@@ -264,7 +258,7 @@ public class FilterProfileJUnitTest {
     unregisterAllFilterClass(inv, true);
   }
 
-  public void unregisterAllFilterClass(boolean inv, boolean twoClients) {
+  private void unregisterAllFilterClass(boolean inv, boolean twoClients) {
     String clientId = "client";
     fprofile.registerClientInterest(clientId, "com.gemstone.gemfire.internal.cache.tier.sockets.TestFilter", InterestType.FILTER_CLASS, inv);
     if (twoClients) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
index 693ead4..13f2527 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
@@ -382,7 +382,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase {
   }
 
 //  public static void processException() {
-//    assertEquals(conn.getServer().getPort(), PORT1.intValue());
+//    assertIndexDetailsEquals(conn.getServer().getPort(), PORT1.intValue());
 //    try {
 //      pool.processException(new Exception("dummy"), conn);
 //      //Thread.sleep(10000); // why sleep?
@@ -516,7 +516,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase {
     {
     try {
       Cache c = CacheFactory.getAnyInstance();
-      // assertEquals("More than one BridgeServer", 1,
+      // assertIndexDetailsEquals("More than one BridgeServer", 1,
       // c.getCacheServers().size());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
index 74370c7..66527d0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
@@ -269,7 +269,7 @@ public class InterestListFailoverDUnitTest extends DistributedTestCase
       final String key1 = "key-1";
       assertNotNull(r);
       // Verify that 'key-1' was updated
-      // assertEquals("vm2-key-1", r.getEntry("key-1").getValue());
+      // assertIndexDetailsEquals("vm2-key-1", r.getEntry("key-1").getValue());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
         public boolean done() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
index 8caf3f6..9f05aa7 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
@@ -17,7 +17,6 @@
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
 import static org.junit.Assert.*;
-import static org.mockito.Matchers.*;
 import static org.mockito.Mockito.*;
 
 import java.net.Socket;
@@ -33,11 +32,11 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class MessageJUnitTest {
 
-  Message message;
-  Socket mockSocket;
-  MessageStats mockStats;
-  ByteBuffer msgBuffer;
-  ServerConnection mockServerConnection;
+  private Message message;
+  private Socket mockSocket;
+  private MessageStats mockStats;
+  private ByteBuffer msgBuffer;
+  private ServerConnection mockServerConnection;
   
   @Before
   public void setUp() throws Exception {
@@ -81,11 +80,10 @@ public class MessageJUnitTest {
     message.setParts(parts);
     try {
       message.send();
+      fail("expected an exception but none was thrown");
     } catch (MessageTooLargeException e) {
       assertTrue(e.getMessage().contains("exceeds maximum integer value"));
-      return;
     }
-    fail("expected an exception but none was thrown");
   }
   
   @Test
@@ -98,14 +96,12 @@ public class MessageJUnitTest {
     message.setParts(parts);
     try {
       message.send();
+      fail("expected an exception but none was thrown");
     } catch (MessageTooLargeException e) {
       assertFalse(e.getMessage().contains("exceeds maximum integer value"));
-      return;
     }
-    fail("expected an exception but none was thrown");
   }
-  
-  
+
   // TODO many more tests are needed
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ObjectPartListJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ObjectPartListJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ObjectPartListJUnitTest.java
index ef6bfcb..0805e2a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ObjectPartListJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ObjectPartListJUnitTest.java
@@ -16,24 +16,23 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.List;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.CopyHelper;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
-public class ObjectPartListJUnitTest extends TestCase {
-  
-  public void testValueAsObject() throws IOException {
+public class ObjectPartListJUnitTest {
+
+  @Test
+  public void testValueAsObject() throws Exception {
     VersionedObjectList list = new VersionedObjectList(100, false, false);
     byte[] normalBytes = "value1".getBytes();
     list.addObjectPart("key", normalBytes , false, null);
@@ -69,8 +68,9 @@ public class ObjectPartListJUnitTest extends TestCase {
     assertEquals(new TestException("hello"), values.get(3));
     assertNull(values.get(4));
   }
-  
-  public void testValueAsObjectByteArray() throws IOException, ClassNotFoundException {
+
+  @Test
+  public void testValueAsObjectByteArray() throws Exception {
     ObjectPartList list = new VersionedObjectList(100, false, false, true);
     byte[] normalBytes = "value1".getBytes();
     list.addObjectPart("key", normalBytes , false, null);
@@ -108,10 +108,8 @@ public class ObjectPartListJUnitTest extends TestCase {
 
     public TestException(String message) {
       super(message);
-      // TODO Auto-generated constructor stub
     }
-    
-    
+
     @Override
     public boolean equals(Object o) {
       if(!(o instanceof TestException)) {
@@ -122,8 +120,5 @@ public class ObjectPartListJUnitTest extends TestCase {
       }
       return true;
     }
-    
-    
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
index 65d6448..f1fba88 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
@@ -65,8 +65,8 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       //pause(5000);      
       verifyLiveAndRedundantServers(3, 0);
       verifyOrderOfEndpoints();
-      //assertEquals(1, pool.getRedundantNames().size());
-      // assertEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
 //      pause(10 * 1000);
 //      assertFalse(pool.getCurrentServerNames().contains(SERVER3));
       WaitCriterion wc = new WaitCriterion() {
@@ -114,8 +114,8 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       // pause(5000);
       verifyLiveAndRedundantServers(3, 0);
       verifyOrderOfEndpoints();
-      //assertEquals(1, pool.getRedundantNames().size());
-      //assertEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
 //      pause(10 * 1000);
 //      assertFalse(pool.getCurrentServerNames().contains(SERVER1));
       WaitCriterion wc = new WaitCriterion() {
@@ -153,7 +153,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       assertEquals(1, pool.getRedundantNames().size());
       //assertTrue(pool.getRedundantNames().contains(SERVER1));      
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.stopServer());
       //pause(5000);
@@ -162,11 +162,11 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
       // assertTrue(proxy.getDeadServers().contains(SERVER3));
-      //assertEquals(2, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
-      //assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -193,7 +193,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER4));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -202,11 +202,11 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
       // assertTrue(proxy.getDeadServers().contains(SERVER3));
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -230,7 +230,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER4));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -241,11 +241,11 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
       // assertTrue(proxy.getDeadServers().contains(SERVER3));
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -270,7 +270,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER4));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -280,11 +280,11 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
       // assertTrue(proxy.getDeadServers().contains(SERVER3));
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -308,7 +308,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -318,11 +318,11 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
       // assertTrue(proxy.getDeadServers().contains(SERVER3));
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
-      // assertEquals(3, pool.getConnectedServerCount());
-     // assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+     // assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -347,20 +347,20 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       assertEquals(1, pool.getRedundantNames().size());
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server1.invoke(() -> RedundancyLevelTestBase.stopServer());
       //pause(5000);
       verifyDeadServers(1);
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
-      //assertEquals(2, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       //      assertTrue(pool.getRedundantNames()
       //          .contains(SERVER1));
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.verifyInterestRegistration());
-      //assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -387,20 +387,20 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       assertEquals(1, pool.getRedundantNames().size());
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server1.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
       verifyDeadServers(1);
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.verifyInterestRegistration());
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -425,7 +425,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       assertEquals(1, pool.getRedundantNames().size());
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server1.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -434,13 +434,13 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyDeadServers(1);
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.verifyInterestRegistration());
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -465,7 +465,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       assertEquals(1, pool.getRedundantNames().size());
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server1.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -473,14 +473,14 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyDeadServers(1);
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       verifyOrderOfEndpoints();
       //server1.invoke(RedundancyLevelTestBase.class,
       //    "verifyInterestRegistration");
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -505,7 +505,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       assertEquals(1, pool.getRedundantNames().size());
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server1.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -517,13 +517,13 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
       verifyDeadServers(1);
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER1));
       verifyOrderOfEndpoints();
       server2.invoke(() -> RedundancyLevelTestBase.verifyInterestRegistration());
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart2DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart2DUnitTest.java
index 4692699..0499847 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart2DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart2DUnitTest.java
@@ -67,7 +67,7 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
 
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server0.invoke(() -> RedundancyLevelTestBase.stopServer());
       //pause(5000);
@@ -75,12 +75,12 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
-      //assertEquals(2, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       //assertTrue(pool.getRedundantNames()
       //    .contains(SERVER2));
       assertTrue(pool.getPrimaryName().equals(SERVER2));
-      //assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -108,7 +108,7 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       waitConnectedServers(4);
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server0.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -116,12 +116,12 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER2));
       assertTrue(pool.getPrimaryName().equals(SERVER2));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -147,7 +147,7 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       waitConnectedServers(4);
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server0.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -157,12 +157,12 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER2));
       assertTrue(pool.getPrimaryName().equals(SERVER2));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -188,7 +188,7 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       waitConnectedServers(4);
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server0.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -197,12 +197,12 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER2));
       assertTrue(pool.getPrimaryName().equals(SERVER2));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -228,7 +228,7 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       waitConnectedServers(4);
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
-      //assertEquals(0, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(0, proxy.getDeadServers().size());
       verifyOrderOfEndpoints();
       server0.invoke(() -> RedundancyLevelTestBase.stopServer());
       // pause(5000);
@@ -237,12 +237,12 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       verifyRedundantServersContain(SERVER3);
       verifyLiveAndRedundantServers(3, 1);
       verifyOrderOfEndpoints();
-      // assertEquals(2, pool.getRedundantNames().size());
+      // assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // assertTrue(pool.getRedundantNames()
       // .contains(SERVER2));
       assertTrue(pool.getPrimaryName().equals(SERVER2));
-      // assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(1, proxy.getDeadServers().size());
+      // assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, proxy.getDeadServers().size());
     }
     catch (Exception ex) {
       Assert.fail(
@@ -273,7 +273,7 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(5000);
       verifyLiveAndRedundantServers(2, 1);
       verifyOrderOfEndpoints();
-      //assertEquals(2, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       //Not Sure
       //assertTrue(pool.getPrimaryName().equals(SERVER2));
       server2.invoke(() -> RedundancyLevelTestBase.verifyInterestRegistration());
@@ -311,19 +311,19 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       server1.invoke(() -> RedundancyLevelTestBase.stopServer());
       server2.invoke(() -> RedundancyLevelTestBase.stopServer());
       verifyDeadServers(2);
-      //assertEquals(2, pool.getConnectedServerCount());
-      //assertEquals(2, proxy.getDeadServers().size());
+      //assertIndexDetailsEquals(2, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(2, proxy.getDeadServers().size());
       //pause(10000);
       verifyLiveAndRedundantServers(2, 1);
       verifyOrderOfEndpoints();
-      //assertEquals(2, pool.getRedundantNames().size());      
+      //assertIndexDetailsEquals(2, pool.getRedundantNames().size());
       // bring up one server.
       server1.invoke(() -> RedundancyLevelTestBase.startServer());
       //pause(10000);
       verifyLiveAndRedundantServers(3, 2);
       verifyOrderOfEndpoints();
-      //assertEquals(3, pool.getRedundantNames().size());
-      //assertEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(3, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       verifyRedundantServersContain(SERVER2);
       verifyRedundantServersContain(SERVER4);
@@ -334,8 +334,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);
       Wait.pause(1000);
       verifyOrderOfEndpoints();
-      //assertEquals(3, pool.getRedundantNames().size());
-      //assertEquals(4, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(3, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(4, pool.getConnectedServerCount());
       server2.invoke(() -> RedundancyLevelTestBase.verifyNoCCP());
     }
     catch (Exception ex) {
@@ -364,8 +364,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);      
       verifyLiveAndRedundantServers(2, 1);
       verifyOrderOfEndpoints();
-      //assertEquals(2, pool.getRedundantNames().size());
-      //assertEquals(2, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(2, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(2, pool.getConnectedServerCount());
       assertTrue(pool.getRedundantNames().contains(SERVER4));
       assertTrue(pool.getPrimaryName().equals(SERVER1));
       assertFalse(pool.getRedundantNames().contains(SERVER3));
@@ -375,8 +375,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);
       verifyLiveAndRedundantServers(3, 2);
       verifyOrderOfEndpoints();
-      //assertEquals(3, pool.getConnectedServerCount());
-      //assertEquals(3, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(3, pool.getRedundantNames().size());
       assertTrue(pool.getRedundantNames().contains(SERVER4));
       assertTrue(pool.getRedundantNames().contains(SERVER3));
       assertTrue(pool.getPrimaryName().equals(SERVER1));
@@ -387,8 +387,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);
       Wait.pause(1000);
       verifyOrderOfEndpoints();
-      //assertEquals(3, pool.getRedundantNames().size());
-      //assertEquals(4, pool.getConnectedServerCount());      
+      //assertIndexDetailsEquals(3, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(4, pool.getConnectedServerCount());
       assertTrue(pool.getRedundantNames().contains(SERVER4));
       assertTrue(pool.getRedundantNames().contains(SERVER3));
       assertTrue(pool.getPrimaryName().equals(SERVER1));
@@ -418,8 +418,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);      
       verifyLiveAndRedundantServers(3, 2);
       verifyOrderOfEndpoints();
-      //assertEquals(3, pool.getRedundantNames().size());
-      //assertEquals(3, pool.getConnectedServerCount());      
+      //assertIndexDetailsEquals(3, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
       assertTrue(pool.getRedundantNames().contains(SERVER3));
       assertTrue(pool.getRedundantNames().contains(SERVER4));
       assertTrue(pool.getPrimaryName().equals(SERVER1));
@@ -428,8 +428,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       server1.invoke(() -> RedundancyLevelTestBase.startServer());
       Wait.pause(1000);
       verifyOrderOfEndpoints();
-      //assertEquals(3, pool.getRedundantNames().size());
-      //assertEquals(4, pool.getConnectedServerCount());      
+      //assertIndexDetailsEquals(3, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(4, pool.getConnectedServerCount());
       assertTrue(pool.getRedundantNames().contains(SERVER3));
       assertTrue(pool.getRedundantNames().contains(SERVER4));
       assertTrue(pool.getPrimaryName().equals(SERVER1));
@@ -458,8 +458,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);
       verifyLiveAndRedundantServers(3, 2);
       verifyOrderOfEndpoints();
-      //assertEquals(1, pool.getRedundantNames().size());
-      //assertEquals(3, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(3, pool.getConnectedServerCount());
       assertFalse(pool.getRedundantNames().contains(SERVER1));
       assertFalse(pool.getRedundantNames().contains(SERVER3));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
@@ -470,8 +470,8 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
       //pause(10000);
       verifyLiveAndRedundantServers(4, 3);
       verifyOrderOfEndpoints();
-      //assertEquals(1, pool.getRedundantNames().size());
-      //assertEquals(4, pool.getConnectedServerCount());
+      //assertIndexDetailsEquals(1, pool.getRedundantNames().size());
+      //assertIndexDetailsEquals(4, pool.getConnectedServerCount());
       //assertTrue(pool.getRedundantNames()
       //    .contains(SERVER1));
       assertTrue(pool.getRedundantNames().contains(SERVER2));
@@ -495,12 +495,12 @@ public class RedundancyLevelPart2DUnitTest extends RedundancyLevelTestBase
   {
     try {
       createClientCache(getServerHostName(Host.getHost(0)), PORT1, PORT2, PORT3, PORT4, 1);
-      assertEquals(1, proxy.getRedundantServers().size());
-      assertEquals(PORT3, proxy.acquireConnection().getEndpoint().getPort());
-      assertEquals(PORT4, proxy.acquireConnection().getEndpoint().getPort());
-      assertEquals(PORT1, proxy.acquireConnection().getEndpoint().getPort());
-      assertEquals(PORT2, proxy.acquireConnection().getEndpoint().getPort());
-      assertEquals(PORT3, proxy.acquireConnection().getEndpoint().getPort());
+      assertIndexDetailsEquals(1, proxy.getRedundantServers().size());
+      assertIndexDetailsEquals(PORT3, proxy.acquireConnection().getEndpoint().getPort());
+      assertIndexDetailsEquals(PORT4, proxy.acquireConnection().getEndpoint().getPort());
+      assertIndexDetailsEquals(PORT1, proxy.acquireConnection().getEndpoint().getPort());
+      assertIndexDetailsEquals(PORT2, proxy.acquireConnection().getEndpoint().getPort());
+      assertIndexDetailsEquals(PORT3, proxy.acquireConnection().getEndpoint().getPort());
     }
     catch (Exception ex) {
       ex.printStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
index 17e1e2a..285d200 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
@@ -156,7 +156,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   public static void verifyDispatcherIsAlive()
   {
     try {
-//      assertEquals("More than one BridgeServer", 1, cache.getCacheServers()
+//      assertIndexDetailsEquals("More than one BridgeServer", 1, cache.getCacheServers()
 //          .size());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
@@ -215,7 +215,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   public static void verifyDispatcherIsNotAlive()
   {
     try {
-      // assertEquals("More than one BridgeServer", 1,
+      // assertIndexDetailsEquals("More than one BridgeServer", 1,
       // cache.getCacheServers().size());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
index 328a796..9159ad7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CommitCommandTest.java
@@ -16,10 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
+import static org.mockito.Mockito.*;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -41,12 +38,9 @@ public class CommitCommandTest {
 	 * No NPE should be thrown from the {@link CommitCommand#writeCommitResponse(com.gemstone.gemfire.internal.cache.TXCommitMessage, Message, ServerConnection)}
 	 * if the response message is null as it is the case when JTA
 	 * transaction is rolled back with TX_SYNCHRONIZATION AFTER_COMPLETION STATUS_ROLLEDBACK 
-	 * @throws IOException 
-	 * 
 	 */
 	@Test
-	public void testWriteNullResponse() throws IOException {
-		
+	public void testWriteNullResponse() throws Exception {
 		Cache cache = mock(Cache.class);
 		Message origMsg = mock(Message.class);
 		ServerConnection servConn = mock(ServerConnection.class);
@@ -55,7 +49,5 @@ public class CommitCommandTest {
 		when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
 		
 		CommitCommand.writeCommitResponse(null, origMsg, servConn);
-		
 	}
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RVVExceptionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RVVExceptionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RVVExceptionJUnitTest.java
index 62a12df..4935524 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RVVExceptionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RVVExceptionJUnitTest.java
@@ -16,18 +16,17 @@
  */
 package com.gemstone.gemfire.internal.cache.versions;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class RVVExceptionJUnitTest extends TestCase {
+public class RVVExceptionJUnitTest {
 
-  public RVVExceptionJUnitTest() {
-  }
-  
+  @Test
   public void testRVVExceptionB() {
     RVVExceptionB ex = new RVVExceptionB(5, 10);
     ex.add(8);
@@ -35,14 +34,13 @@ public class RVVExceptionJUnitTest extends TestCase {
     assertEquals(8, ex.getHighestReceivedVersion());
     ex.add(5);
     assertEquals(8, ex.getHighestReceivedVersion());
-    
   }
 
+  @Test
   public void testRVVExceptionT() {
     RVVExceptionT ex = new RVVExceptionT(5, 10);
     ex.add(8);
     ex.add(6);
     assertEquals(8, ex.getHighestReceivedVersion());
-
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionHolder2JUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionHolder2JUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionHolder2JUnitTest.java
index af78f79..20be1b7 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionHolder2JUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionHolder2JUnitTest.java
@@ -18,8 +18,6 @@ package com.gemstone.gemfire.internal.cache.versions;
 
 import static org.junit.Assert.*;
 
-import java.util.Iterator;
-
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -43,7 +41,6 @@ public class RegionVersionHolder2JUnitTest {
     assertEquals(0, h.getExceptionCount());
   }
 
-
   @Test
   public void testRecordZeroDoesNothingWithBitSet() {
     recordZeroDoesNothing(true);
@@ -161,7 +158,7 @@ public class RegionVersionHolder2JUnitTest {
     assertTrue(h.isSpecialException(e, h));
     h.recordVersion(2l);
     // BUG: the exception is not removed
-//    assertEquals("unexpected RVV exception : " + h, 0, h.getExceptionCount());
+//    assertIndexDetailsEquals("unexpected RVV exception : " + h, 0, h.getExceptionCount());
   }
   
   private void createSpecialException(RegionVersionHolder h) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVectorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVectorJUnitTest.java
index df85998..82f2003 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVectorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVectorJUnitTest.java
@@ -16,17 +16,17 @@
  */
 package com.gemstone.gemfire.internal.cache.versions;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.io.IOException;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import junit.framework.TestCase;
-
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.DataSerializer;
@@ -39,8 +39,9 @@ import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class RegionVersionVectorJUnitTest extends TestCase {
+public class RegionVersionVectorJUnitTest {
 
+  @Test
   public void testExceptionsWithContains() {
     DiskStoreID ownerId = new DiskStoreID(0, 0);
     DiskStoreID id1 = new DiskStoreID(0, 1);
@@ -53,6 +54,7 @@ public class RegionVersionVectorJUnitTest extends TestCase {
   }
   
   @SuppressWarnings({ "unchecked", "rawtypes" })
+  @Test
   public void testRegionVersionVectors() throws Exception {
     // this is just a quick set of unit tests for basic RVV functionality
     
@@ -266,10 +268,10 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     assertTrue(rv1.contains(server2, bitSetRollPoint));
     assertTrue(rv1.contains(server2, bitSetRollPoint+1));
     assertFalse(rv1.contains(server2, bitSetRollPoint+2));
-  
   }
-  
-  public void testRVVSerialization() throws IOException, ClassNotFoundException {
+
+  @Test
+  public void testRVVSerialization() throws Exception {
     DiskStoreID ownerId = new DiskStoreID(0, 0);
     DiskStoreID id1 = new DiskStoreID(0, 1);
     DiskStoreID id2 = new DiskStoreID(1, 0);
@@ -300,6 +302,7 @@ public class RegionVersionVectorJUnitTest extends TestCase {
   /**
    * Test that we can copy the member to version map correctly.
    */
+  @Test
   public void testCopyMemberToVersion() {
     DiskStoreID id0 = new DiskStoreID(0, 0);
     DiskStoreID id1 = new DiskStoreID(0, 1);
@@ -331,7 +334,8 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     assertTrue(rvv1.dominates(rvv0));
     assertFalse(rvv0.dominates(rvv1));
   }
-  
+
+  @Test
   public void testSpecialException() {
     DiskStoreID id0 = new DiskStoreID(0, 0);
     DiskStoreID id1 = new DiskStoreID(0, 1);
@@ -357,7 +361,8 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     assertTrue(rvv1.dominates(rvv0));
     assertTrue(rvv0.dominates(rvv1));
   }
-  
+
+  @Test
   public void test48066_1() {
     DiskStoreID id0 = new DiskStoreID(0, 0);
     DiskRegionVersionVector rvv0 = new DiskRegionVersionVector(id0);
@@ -384,7 +389,8 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     System.out.println("after record 7, rvv2="+rvv2.fullToString());
     assertEquals(7, rvv2.getCurrentVersion());
   }
-  
+
+  @Test
   public void test48066_2() {
     DiskStoreID id0 = new DiskStoreID(0, 0);
     DiskRegionVersionVector rvv0 = new DiskRegionVersionVector(id0);
@@ -415,8 +421,8 @@ public class RegionVersionVectorJUnitTest extends TestCase {
   /**
    * Test for bug 47023. Make sure recordGCVersion works
    * correctly and doesn't generate exceptions for the local member.
-   * 
    */
+  @Test
   public void testRecordGCVersion() {
     DiskStoreID id0 = new DiskStoreID(0, 0);
     DiskStoreID id1 = new DiskStoreID(0, 1);
@@ -432,7 +438,6 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     rvv0.recordVersion(id1, 1);
     rvv0.recordVersion(id1, 3);
     rvv0.recordVersion(id1, 5);
-    
 
     //Assert that the exceptions are present
     {
@@ -470,9 +475,9 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     
     //exceptions greater than the GC version should still be there.
     assertFalse(holder1.contains(4));
-    
   }
 
+  @Test
   public void testRemoveOldVersions() {
     DiskStoreID id0 = new DiskStoreID(0, 0);
     DiskStoreID id1 = new DiskStoreID(0, 1);
@@ -511,7 +516,8 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     assertEquals("expected exceptions to be erased for " + rvv.fullToString(),
         rvv.getExceptionCount(id2), 0);
   }
-  
+
+  @Test
   public void testRegionVersionInTags() {
     VMVersionTag tag = new VMVersionTag();
     long version = 0x8080000000L;
@@ -519,10 +525,7 @@ public class RegionVersionVectorJUnitTest extends TestCase {
     assertEquals("failed test for bug #48576", version, tag.getRegionVersion());
   }
 
-
-  
-  private void doExceptionsWithContains(DiskStoreID id,
-      DiskRegionVersionVector rvv) {
+  private void doExceptionsWithContains(DiskStoreID id, DiskRegionVersionVector rvv) {
     rvv.recordVersion(id, 10);
     
     //Make sure we have exceptions from 0-10

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
index a7daf98..c85be1e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
@@ -17,6 +17,7 @@
 package com.gemstone.gemfire.internal.cache.wan.parallel;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -38,8 +39,6 @@ import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
 import com.gemstone.gemfire.internal.cache.wan.parallel.ParallelGatewaySenderQueue.MetaRegionFactory;
 import com.gemstone.gemfire.internal.cache.wan.parallel.ParallelGatewaySenderQueue.ParallelGatewaySenderQueueMetaRegion;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import static org.mockito.Mockito.*;
-
 
 @Category(UnitTest.class)
 public class ParallelGatewaySenderQueueJUnitTest {
@@ -62,7 +61,7 @@ public class ParallelGatewaySenderQueueJUnitTest {
   }
 
   @Test
-  public void testLocalSize() throws TimeoutException, RegionExistsException, ClassNotFoundException, IOException {
+  public void testLocalSize() throws Exception {
     ParallelGatewaySenderQueueMetaRegion mockMetaRegion = mock(ParallelGatewaySenderQueueMetaRegion.class);
     PartitionedRegionDataStore dataStore = mock(PartitionedRegionDataStore.class);
     when(mockMetaRegion.getDataStore()).thenReturn(dataStore);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractEntityResolverTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractEntityResolverTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractEntityResolverTest.java
index f42d52c..9d11715 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractEntityResolverTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractEntityResolverTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.cache.xmlcache;
 
 import com.gemstone.gemfire.internal.ClassPathLoader;
@@ -77,12 +76,10 @@ public abstract class AbstractEntityResolverTest {
    * Resolve the cache.xml XSD using the {@link PivotalEntityResolver}. Verifies
    * that the META-INF/schemas files are correctly found.
    * 
-   * @throws SAXException
-   * @throws IOException
    * @since 8.1
    */
   @Test
-  public void testResolveEntity() throws SAXException, IOException {
+  public void testResolveEntity() throws Exception {
     final InputSource inputSource = getEntityResolver().resolveEntity(null, getSystemId());
     assertNotNull(inputSource);
     assertEquals(getSystemId(), inputSource.getSystemId());
@@ -93,12 +90,10 @@ public abstract class AbstractEntityResolverTest {
    * <code>null</code> <code>systemId</code>. Asserts that returns to
    * <code>null<code>.
    * 
-   * @throws SAXException
-   * @throws IOException
    * @since 8.1
    */
   @Test
-  public void testResolveEntityNullSystemId() throws SAXException, IOException {
+  public void testResolveEntityNullSystemId() throws SAXException, Exception {
     final String systemId = null;
     final InputSource inputSource = getEntityResolver().resolveEntity(null, systemId);
     assertNull(inputSource);
@@ -109,12 +104,10 @@ public abstract class AbstractEntityResolverTest {
    * <code>"--not-a-valid-system-id--"</code> <code>systemId</code>, which is
    * not in the Pivotal namespace.. Asserts that returns to <code>null<code>.
    * 
-   * @throws SAXException
-   * @throws IOException
    * @since 8.1
    */
   @Test
-  public void testResolveEntityUnkownSystemId() throws SAXException, IOException {
+  public void testResolveEntityUnkownSystemId() throws Exception {
     final String systemId = "--not-a-valid-system-id--";
     final InputSource inputSource = getEntityResolver().resolveEntity(null, systemId);
     assertNull(inputSource);
@@ -126,12 +119,10 @@ public abstract class AbstractEntityResolverTest {
    * <code>systemId</code>, which should not be found. Asserts that returns to
    * <code>null<code>.
    * 
-   * @throws SAXException
-   * @throws IOException
    * @since 8.1
    */
   @Test
-  public void testResolveEntityNotFoundSystemId() throws SAXException, IOException {
+  public void testResolveEntityNotFoundSystemId() throws Exception {
     final String systemId = "http://schema.pivotal.io/this/should/be/not/found.xsd";
     final InputSource inputSource = getEntityResolver().resolveEntity(null, systemId);
     assertNull(inputSource);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractXmlParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractXmlParserJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractXmlParserJUnitTest.java
index 6148da1..8e9e2a4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractXmlParserJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/xmlcache/AbstractXmlParserJUnitTest.java
@@ -31,7 +31,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link AbstractXmlParser}.
- * 
  *
  * @since 8.1
  */



[08/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
index f1574bc..9249afd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
@@ -16,52 +16,76 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
 import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Stack;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.Vector;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.CanonicalInstantiator;
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.Instantiator;
-import com.gemstone.gemfire.CanonicalInstantiator;
 import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.InternalInstantiator;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import java.io.*;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import junit.framework.*;
-
-import org.junit.experimental.categories.Category;
-
 /**
  * Tests the functionality of the {@link DataSerializable} class.
  *
  * @since 3.0
  */
 @Category(UnitTest.class)
-public class DataSerializableJUnitTest extends TestCase
-  implements Serializable {
+public class DataSerializableJUnitTest implements Serializable {
 
   /** A <code>ByteArrayOutputStream</code> that data is serialized to */
   private transient ByteArrayOutputStream baos;
 
-  public DataSerializableJUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Helper methods
+  @Rule
+  public transient TestName testName = new TestName();
 
   /**
    * Creates a new <code>ByteArrayOutputStream</code> for this test to
    * work with.
    */
+  @Before
   public void setUp() {
     this.baos = new ByteArrayOutputStream();
   }
@@ -73,14 +97,14 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Returns a <code>DataOutput</code> to write to
    */
-  protected DataOutputStream getDataOutput() {
+  private DataOutputStream getDataOutput() {
     return new DataOutputStream(this.baos);
   }
 
   /**
    * Returns a <code>DataInput</code> to read from
    */
-  protected DataInput getDataInput() {
+  private DataInput getDataInput() {
     // changed this to use ByteBufferInputStream to give us better
     // test coverage of this class.
     ByteBuffer bb = ByteBuffer.wrap(this.baos.toByteArray());
@@ -88,7 +112,7 @@ public class DataSerializableJUnitTest extends TestCase
     return bbis;
   }
 
-  protected DataInputStream getDataInputStream() {
+  private DataInputStream getDataInputStream() {
     ByteBuffer bb = ByteBuffer.wrap(this.baos.toByteArray());
     ByteBufferInputStream bbis = new ByteBufferInputStream(bb);
     return new DataInputStream(bbis);
@@ -97,27 +121,17 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Returns a random number generator
    */
-  protected Random getRandom() {
-    long seed =
-      Long.getLong("SEED", System.currentTimeMillis()).longValue();
-    System.out.println("SEED for " + this.getName() + ": " + seed);
+  private Random getRandom() {
+    long seed = Long.getLong("SEED", System.currentTimeMillis()).longValue();
+    System.out.println("SEED for " + this.testName.getMethodName() + ": " + seed);
     return new Random(seed);
   }
 
-  protected static void fail(String s, Throwable t) {
-    StringWriter sw = new StringWriter();
-    PrintWriter pw = new PrintWriter(sw, true);
-    pw.println(s);
-    t.printStackTrace(pw);
-    fail(sw.toString());
-  }
-
-  ////////  Test methods
-
   /**
    * Tests data serializing a {@link Class}
    */
-  public void testClass() throws IOException, ClassNotFoundException {
+  @Test
+  public void testClass() throws Exception {
     Class c = this.getClass();
 
     DataOutputStream out = getDataOutput();
@@ -131,11 +145,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Class} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testClassObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testClassObject() throws Exception {
     Class c = this.getClass();
 
     DataOutputStream out = getDataOutput();
@@ -147,9 +160,8 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals(c, c2);
   }
 
-  public void testBigInteger()
-  throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testBigInteger() throws Exception {
     BigInteger o = new BigInteger("12345678901234567890");
 
     DataOutputStream out = getDataOutput();
@@ -160,9 +172,9 @@ public class DataSerializableJUnitTest extends TestCase
     BigInteger o2 = (BigInteger) DataSerializer.readObject(in);
     assertEquals(o, o2);
   }
-  public void testBigDecimal()
-  throws IOException, ClassNotFoundException {
 
+  @Test
+  public void testBigDecimal() throws Exception {
     BigDecimal o = new BigDecimal("1234567890.1234567890");
 
     DataOutputStream out = getDataOutput();
@@ -173,9 +185,9 @@ public class DataSerializableJUnitTest extends TestCase
     BigDecimal o2 = (BigDecimal) DataSerializer.readObject(in);
     assertEquals(o, o2);
   }
-  public void testUUID()
-  throws IOException, ClassNotFoundException {
 
+  @Test
+  public void testUUID() throws Exception {
     UUID o = UUID.randomUUID();
 
     DataOutputStream out = getDataOutput();
@@ -186,9 +198,9 @@ public class DataSerializableJUnitTest extends TestCase
     UUID o2 = (UUID) DataSerializer.readObject(in);
     assertEquals(o, o2);
   }
-  public void testTimestamp()
-  throws IOException, ClassNotFoundException {
 
+  @Test
+  public void testTimestamp() throws Exception {
     Timestamp o = new Timestamp(new Date().getTime() + 79);
 
     DataOutputStream out = getDataOutput();
@@ -203,7 +215,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Date}
    */
-  public void testDate() throws IOException {
+  @Test
+  public void testDate() throws Exception {
     Date date = new Date();
 
     DataOutputStream out = getDataOutput();
@@ -217,11 +230,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Date} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testDateObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testDateObject() throws Exception {
     Date date = new Date();
 
     DataOutputStream out = getDataOutput();
@@ -236,7 +248,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link File}
    */
-  public void testFile() throws IOException {
+  @Test
+  public void testFile() throws Exception {
     File file = new File(System.getProperty("user.dir"));
 
     DataOutputStream out = getDataOutput();
@@ -250,11 +263,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link File} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testFileObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testFileObject() throws Exception {
     File file = new File(System.getProperty("user.dir"));
 
     DataOutputStream out = getDataOutput();
@@ -269,7 +281,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link InetAddress}
    */
-  public void testInetAddress() throws IOException {
+  @Test
+  public void testInetAddress() throws Exception {
     InetAddress address = InetAddress.getLocalHost();
 
     DataOutputStream out = getDataOutput();
@@ -277,18 +290,17 @@ public class DataSerializableJUnitTest extends TestCase
     out.flush();
 
     DataInput in = getDataInput();
-    InetAddress address2 = 
+    InetAddress address2 =
       DataSerializer.readInetAddress(in);
     assertEquals(address, address2);
   }
 
   /**
    * Tests data serializing a {@link InetAddress} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testInetAddressObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testInetAddressObject() throws Exception {
     InetAddress address = InetAddress.getLocalHost();
 
     DataOutputStream out = getDataOutput();
@@ -303,11 +315,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing <code>null</code> using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testNullObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testNullObject() throws Exception {
     Object value = null;
 
     DataOutputStream out = getDataOutput();
@@ -320,8 +331,9 @@ public class DataSerializableJUnitTest extends TestCase
   }
 
   /**
-   * Tests data serializing a non-<code>null</code> {@link String} 
+   * Tests data serializing a non-<code>null</code> {@link String}
    */
+  @Test
   public void testString() throws Exception {
     String value = "Hello";
 
@@ -337,6 +349,7 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals(value, value2);
   }
 
+  @Test
   public void testUtfString() throws Exception {
     String value = "Hello" + Character.MIN_VALUE + Character.MAX_VALUE;
 
@@ -352,7 +365,7 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals(value, value2);
   }
 
-  
+  @Test
   public void testBigString() throws Exception {
     StringBuffer sb = new StringBuffer(100000);
     for (int i=0; i < 100000; i++) {
@@ -371,10 +384,11 @@ public class DataSerializableJUnitTest extends TestCase
     value2 = (String)DataSerializer.readObject(in);
     assertEquals(value, value2);
   }
-  
+
   /**
-   * * Tests data serializing a non-<code>null</code> {@link String} longer than 64k.
+   * Tests data serializing a non-<code>null</code> {@link String} longer than 64k.
    */
+  @Test
   public void testBigUtfString() throws Exception {
     StringBuffer sb = new StringBuffer(100000);
     for (int i=0; i < 100000; i++) {
@@ -399,9 +413,10 @@ public class DataSerializableJUnitTest extends TestCase
   }
 
   /**
-   * Tests data serializing a non-ascii {@link String} 
-   */   
-  public void testNonAsciiString() throws IOException {
+   * Tests data serializing a non-ascii {@link String}
+   */
+  @Test
+  public void testNonAsciiString() throws Exception {
     basicTestString("Hello1" + '\u0000');
     setUp();
     basicTestString("Hello2" + '\u0080');
@@ -421,16 +436,19 @@ public class DataSerializableJUnitTest extends TestCase
   }
 
   /**
-   * Tests data serializing a <code>null</code> {@link String} 
+   * Tests data serializing a <code>null</code> {@link String}
    */
-//   public void testNullString() throws IOException {
-//     basicTestString(null);
-//   }
+  @Ignore("for unknown reason")
+  @Test
+   public void testNullString() throws Exception {
+     basicTestString(null);
+   }
 
   /**
    * Tests data serializing a {@link Boolean}
    */
-  public void testBoolean() throws IOException {
+  @Test
+  public void testBoolean() throws Exception {
     Boolean value = new Boolean(getRandom().nextInt() % 2 == 0);
 
     DataOutputStream out = getDataOutput();
@@ -444,10 +462,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Boolean} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testBooleanObject()
-    throws IOException, ClassNotFoundException {
+  @Test
+  public void testBooleanObject() throws Exception {
 
     Boolean value = new Boolean(getRandom().nextInt() % 2 == 0);
 
@@ -460,6 +478,7 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals(value, value2);
   }
 
+  @Test
   public void testWriteObjectAsByteArray() throws Exception {
     // make sure recursive calls to WriteObjectAsByteArray work to test bug 38194
     Object v = new WOABA();
@@ -479,7 +498,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  static class WOABA implements DataSerializable {
+  private static class WOABA implements DataSerializable {
     private byte[] deserialized;
     private WOABA2 f = new WOABA2();
     public WOABA() {
@@ -503,7 +522,8 @@ public class DataSerializableJUnitTest extends TestCase
       this.deserialized = DataSerializer.readByteArray(in);
     }
   }
-  static class WOABA2 implements DataSerializable {
+
+  private  static class WOABA2 implements DataSerializable {
     private byte[] deserialized;
     private String f = "foobar";
     public WOABA2() {
@@ -517,11 +537,12 @@ public class DataSerializableJUnitTest extends TestCase
       this.deserialized = DataSerializer.readByteArray(in);
     }
   }
-  
+
   /**
    * Tests data serializing a {@link Character}
    */
-  public void testCharacter() throws IOException {
+  @Test
+  public void testCharacter() throws Exception {
     char c = (char) ('A' + getRandom().nextInt('Z' - 'A'));
     Character value = new Character(c);
 
@@ -536,11 +557,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Character} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testCharacterObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testCharacterObject() throws Exception {
     char c = (char) ('A' + getRandom().nextInt('Z' - 'A'));
     Character value = new Character(c);
 
@@ -556,7 +576,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Byte}
    */
-  public void testByte() throws IOException {
+  @Test
+  public void testByte() throws Exception {
     Byte value = new Byte((byte) getRandom().nextInt());
 
     DataOutputStream out = getDataOutput();
@@ -570,11 +591,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Byte} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testByteObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testByteObject() throws Exception {
     Byte value = new Byte((byte) getRandom().nextInt());
 
     DataOutputStream out = getDataOutput();
@@ -589,6 +609,7 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Short}
    */
+  @Test
   public void testShort() throws IOException {
     Short value = new Short((short) getRandom().nextInt());
 
@@ -603,8 +624,9 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Short} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
+  @Test
   public void testShortObject()
     throws IOException, ClassNotFoundException {
 
@@ -622,7 +644,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Integer}
    */
-  public void testInteger() throws IOException {
+  @Test
+  public void testInteger() throws Exception {
     Integer value = new Integer(getRandom().nextInt());
 
     DataOutputStream out = getDataOutput();
@@ -636,11 +659,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Integer} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testIntegerObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testIntegerObject() throws Exception {
     Integer value = new Integer(getRandom().nextInt());
 
     DataOutputStream out = getDataOutput();
@@ -655,7 +677,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Long}
    */
-  public void testLong() throws IOException {
+  @Test
+  public void testLong() throws Exception {
     Long value = new Long(getRandom().nextLong());
 
     DataOutputStream out = getDataOutput();
@@ -669,11 +692,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Long} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testLongObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testLongObject() throws Exception {
     Long value = new Long(getRandom().nextLong());
 
     DataOutputStream out = getDataOutput();
@@ -688,7 +710,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Float}
    */
-  public void testFloat() throws IOException {
+  @Test
+  public void testFloat() throws Exception {
     Float value = new Float(getRandom().nextFloat());
 
     DataOutputStream out = getDataOutput();
@@ -702,11 +725,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Float} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testFloatObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testFloatObject() throws Exception {
     Float value = new Float(getRandom().nextFloat());
 
     DataOutputStream out = getDataOutput();
@@ -721,7 +743,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a {@link Double}
    */
-  public void testDouble() throws IOException {
+  @Test
+  public void testDouble() throws Exception {
     Double value = new Double(getRandom().nextDouble());
 
     DataOutputStream out = getDataOutput();
@@ -735,11 +758,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a {@link Double} using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testDoubleObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testDoubleObject() throws Exception {
     Double value = new Double(getRandom().nextDouble());
 
     DataOutputStream out = getDataOutput();
@@ -754,9 +776,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>byte</code> array
    */
-  public void testByteArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testByteArray() throws Exception {
     byte[] array = new byte[] { (byte) 4, (byte) 5, (byte) 6 };
 
     DataOutputStream out = getDataOutput();
@@ -778,11 +799,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>byte</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testByteArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testByteArrayObject() throws Exception {
     byte[] array = new byte[] { (byte) 4, (byte) 5, (byte) 6 };
 
     DataOutputStream out = getDataOutput();
@@ -801,9 +821,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>short</code> array
    */
-  public void testShortArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testShortArray() throws Exception {
     short[] array = new short[] { (short) 4, (short) 5, (short) 6 };
 
     DataOutputStream out = getDataOutput();
@@ -821,11 +840,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>short</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testShortArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testShortArrayObject() throws Exception {
     short[] array = new short[] { (short) 4, (short) 5, (short) 6 };
 
     DataOutputStream out = getDataOutput();
@@ -844,9 +862,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>String</code> array
    */
-  public void testStringArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testStringArray() throws Exception {
     Random random = getRandom();
 
     String[] array =
@@ -871,9 +888,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing a <code>String</code> array that contains
    * a <code>null</code> <code>String</code>.
    */
-  public void testStringArrayWithNull()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testStringArrayWithNull() throws Exception {
     Random random = getRandom();
 
     String[] array =
@@ -896,11 +912,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>String</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testStringArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testStringArrayObject() throws Exception {
     Random random = getRandom();
 
     String[] array =
@@ -924,9 +939,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>int</code> array
    */
-  public void testIntArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testIntArray() throws Exception {
     int[] array = new int[] {  4,  5, 6 };
 
     DataOutputStream out = getDataOutput();
@@ -944,11 +958,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>int</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testIntArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testIntArrayObject() throws Exception {
     int[] array = new int[] { 4, 5, 6 };
 
     DataOutputStream out = getDataOutput();
@@ -967,9 +980,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>long</code> array
    */
-  public void testLongArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testLongArray() throws Exception {
     long[] array = new long[] { 4, 5, 6 };
 
     DataOutputStream out = getDataOutput();
@@ -987,11 +999,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>long</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testLongArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testLongArrayObject() throws Exception {
     long[] array = new long[] { 4, 5, 6 };
 
     DataOutputStream out = getDataOutput();
@@ -1010,9 +1021,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>float</code> array
    */
-  public void testFloatArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testFloatArray() throws Exception {
     float[] array =
       new float[] { (float) 4.0, (float) 5.0, (float) 6.0 };
 
@@ -1031,11 +1041,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>float</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testFloatArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testFloatArrayObject() throws Exception {
     float[] array =
       new float[] { (float) 4.0, (float) 5.0, (float) 6.0 };
 
@@ -1055,9 +1064,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>double</code> array
    */
-  public void testDoubleArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testDoubleArray() throws Exception {
     double[] array =
       new double[] { 4.0, 5.0, 6.0 };
 
@@ -1076,11 +1084,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>double</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testDoubleArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testDoubleArrayObject() throws Exception {
     double[] array =
       new double[] { 4.0, 5.0, 6.0 };
 
@@ -1100,9 +1107,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing a <code>Object</code> array
    */
-  public void testObjectArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testObjectArray() throws Exception {
     Random random = getRandom();
     SerializableImpl[] array = new SerializableImpl[] {
       new SerializableImpl(random), new SerializableImpl(random),
@@ -1125,11 +1131,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>Object</code> array using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testObjectArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testObjectArrayObject() throws Exception {
     Random random = getRandom();
     SerializableImpl[] array = new SerializableImpl[] {
       new SerializableImpl(random), new SerializableImpl(random),
@@ -1154,9 +1159,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests serializing an object that is {@link
    * DataSerializableJUnitTest.SerializableImpl not specially cased}.
    */
-  public void testUnspecialObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testUnspecialObject() throws Exception {
     Object o = new SerializableImpl(getRandom());
 
     DataOutputStream out = getDataOutput();
@@ -1170,11 +1174,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests serializing an object that implements {@link
-   * DataSerializable} 
+   * DataSerializable}
    */
-  public void testDataSerializable()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testDataSerializable() throws Exception {
     DataSerializable ds = new DataSerializableImpl(getRandom());
 
     DataOutputStream out = getDataOutput();
@@ -1189,11 +1192,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests serializing an object that implements {@link
-   * DataSerializable} 
+   * DataSerializable}
    */
-  public void testVersionedDataSerializable()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testVersionedDataSerializable() throws Exception {
     VersionedDataSerializableImpl ds = new VersionedDataSerializableImpl(getRandom());
 
     VersionedDataOutputStream v = new VersionedDataOutputStream(this.baos, Version.GFE_70);
@@ -1213,9 +1215,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests writing a {@link com.gemstone.gemfire.DataSerializable.Replaceable} object
    */
-  public void testReplaceable()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testReplaceable() throws Exception {
     Object o = new ReplaceableImpl();
 
     DataOutputStream out = getDataOutput();
@@ -1230,8 +1231,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link ArrayList}
    */
-  public void testArrayList()
-    throws IOException, ClassNotFoundException {
+  @Test
+  public void testArrayList() throws Exception {
     tryArrayList(-1);
     tryArrayList(50);
     tryArrayList(0x100);
@@ -1261,9 +1262,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link ArrayList} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testArrayListObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testArrayListObject() throws Exception {
     Random random = getRandom();
     ArrayList list = new ArrayList();
     int size = random.nextInt(50);
@@ -1284,9 +1284,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link HashSet}
    */
-  public void testHashSet()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testHashSet() throws Exception {
     Random random = getRandom();
     HashSet set = new HashSet();
     int size = random.nextInt(50);
@@ -1307,9 +1306,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link HashSet} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testHashSetObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testHashSetObject() throws Exception {
     Random random = getRandom();
     HashSet set = new HashSet();
     int size = random.nextInt(50);
@@ -1330,9 +1328,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link TreeSet}
    */
-  public void testTreeSet()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTreeSet() throws Exception {
     Random random = getRandom();
     TreeSet set = new TreeSet();
     int size = random.nextInt(50);
@@ -1356,9 +1353,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link TreeSet}
    */
-  public void testTreeSetWithComparator()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTreeSetWithComparator() throws Exception {
     Random random = getRandom();
     int size = random.nextInt(50);
     TreeSet set = new TreeSet(new MyComparator(size));
@@ -1384,9 +1380,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link TreeSet} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testTreeSetObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTreeSetObject() throws Exception {
     Random random = getRandom();
     TreeSet set = new TreeSet();
     int size = random.nextInt(50);
@@ -1407,9 +1402,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link HashMap}
    */
-  public void testHashMap()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testHashMap() throws Exception {
     Random random = getRandom();
     HashMap map = new HashMap();
     int size = random.nextInt(50);
@@ -1432,9 +1426,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link HashMap} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testHashMapObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testHashMapObject() throws Exception {
     Random random = getRandom();
     HashMap map = new HashMap();
     int size = random.nextInt(50);
@@ -1457,9 +1450,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link TreeMap}
    */
-  public void testTreeMap()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTreeMap() throws Exception {
     Random random = getRandom();
     TreeMap map = new TreeMap();
     int size = random.nextInt(50);
@@ -1485,9 +1477,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link TreeMap}
    */
-  public void testTreeMapWithComparator()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTreeMapWithComparator() throws Exception {
     Random random = getRandom();
     int size = random.nextInt(50);
     TreeMap map = new TreeMap(new MyComparator(size));
@@ -1527,14 +1518,13 @@ public class DataSerializableJUnitTest extends TestCase
       return false;
     }
   }
-  
+
   /**
    * Tests data serializing an {@link TreeMap} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testTreeMapObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTreeMapObject() throws Exception {
     Random random = getRandom();
     TreeMap map = new TreeMap();
     int size = random.nextInt(50);
@@ -1557,9 +1547,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link LinkedHashSet}
    */
-  public void testLinkedHashSet()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testLinkedHashSet() throws Exception {
     Random random = getRandom();
     LinkedHashSet set = new LinkedHashSet();
     int size = random.nextInt(50);
@@ -1580,9 +1569,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link LinkedHashSet} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testLinkedHashSetObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testLinkedHashSetObject() throws Exception {
     Random random = getRandom();
     LinkedHashSet set = new LinkedHashSet();
     int size = random.nextInt(50);
@@ -1603,9 +1591,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link Hashtable}
    */
-  public void testHashtable()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testHashtable() throws Exception {
     Random random = getRandom();
     Hashtable map = new Hashtable();
     int size = random.nextInt(50);
@@ -1628,9 +1615,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link Hashtable} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testHashtableObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testHashtableObject() throws Exception {
     Random random = getRandom();
     Hashtable map = new Hashtable();
     int size = random.nextInt(50);
@@ -1653,9 +1639,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link IdentityHashMap}
    */
-  public void testIdentityHashMap()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testIdentityHashMap() throws Exception {
     Random random = getRandom();
     IdentityHashMap map = new IdentityHashMap();
     int size = random.nextInt(50);
@@ -1678,9 +1663,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link IdentityHashMap} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testIdentityHashMapObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testIdentityHashMapObject() throws Exception {
     Random random = getRandom();
     IdentityHashMap map = new IdentityHashMap();
     int size = random.nextInt(50);
@@ -1703,9 +1687,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link Vector}
    */
-  public void testVector()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testVector() throws Exception {
     Random random = getRandom();
     Vector list = new Vector();
     int size = random.nextInt(50);
@@ -1726,9 +1709,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link Vector} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testVectorObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testVectorObject() throws Exception {
     Random random = getRandom();
     Vector list = new Vector();
     int size = random.nextInt(50);
@@ -1749,9 +1731,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests data serializing an {@link Stack}
    */
-  public void testStack()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testStack() throws Exception {
     Random random = getRandom();
     Stack list = new Stack();
     int size = random.nextInt(50);
@@ -1772,9 +1753,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing an {@link Stack} using {@link
    * DataSerializer#writeObject}.
    */
-  public void testStackObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testStackObject() throws Exception {
     Random random = getRandom();
     Stack list = new Stack();
     int size = random.nextInt(50);
@@ -1796,9 +1776,8 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests data serializing {@link TimeUnit}s using {@link
    * DataSerializer#writeObject}.
    */
-  public void testTimeUnitObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testTimeUnitObject() throws Exception {
     DataOutputStream out = getDataOutput();
     for (TimeUnit v: TimeUnit.values()) {
       DataSerializer.writeObject(v, out, false /* no java serialization allowed */);
@@ -1811,9 +1790,8 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  public void testProperties()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testProperties() throws Exception {
     DataOutputStream out = getDataOutput();
     DataSerializer.writeProperties(new Properties(), out);
     DataSerializer.writeProperties(null, out);
@@ -1842,6 +1820,7 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that registering a <code>Serializer</code> with id 0 throws
    * an exception.
    */
+  @Test
   public void testSerializerZero() {
     try {
       DataSerializer.register(DS0.class);
@@ -1851,7 +1830,8 @@ public class DataSerializableJUnitTest extends TestCase
       // pass...
     }
   }
-  static class DS0 extends DataSerializerImpl {
+
+  private static class DS0 extends DataSerializerImpl {
     public int getId() {
       return 0;
     }
@@ -1864,6 +1844,7 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that registering two <code>Serializer</code>s with the same
    * id throws an exception.
    */
+  @Test
   public void testRegisterTwoSerializers() {
     byte id = (byte) 42;
     DataSerializer.register(DS42.class);
@@ -1891,6 +1872,7 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that an <code>IOException</code> is thrown when the
    * serializer for an object cannot be found.
    */
+  @Test
   public void testNoDeSerializer() throws Exception {
     Random random = new Random();
 
@@ -1913,13 +1895,14 @@ public class DataSerializableJUnitTest extends TestCase
     } finally {
       InternalDataSerializer.GetMarker.WAIT_MS = savVal;
     }
-     
+
   }
 
   /**
    * Tests that a late-registering <code>DataSerializable</code>
    * indeed causes a waiting readObject() method to be notified.
    */
+  @Test
   public void testLateDeSerializer() throws Exception {
     Random random = new Random();
 
@@ -1972,6 +1955,7 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that a late-registering <code>Instantiator</code>
    * indeed causes a waiting readObject() method to be notified.
    */
+  @Test
   public void testLateInstantiator() throws Exception {
     Random random = new Random();
 
@@ -2028,6 +2012,7 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Tests that a custom serializer is consulted
    */
+  @Test
   public void testCustomSerializer() throws Exception {
     Random random = new Random();
 
@@ -2051,6 +2036,7 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that the appropriate exceptions are thrown by {@link
    * Instantiator#register} when given bad input.
    */
+  @Test
   public void testInstantiatorExceptions() {
 
     try {
@@ -2073,21 +2059,6 @@ public class DataSerializableJUnitTest extends TestCase
       // pass...
     }
 
-    // With 1.5's strong typing this code no longer compiles (which is good!)
-//     try {
-//       Instantiator inst = new
-//         Instantiator(SerializableImpl.class, (byte) 42) {
-//           public DataSerializable newInstance() {
-//             return null;
-//           }
-//         };
-//       Instantiator.register(inst);
-//       fail("Should have thrown an IllegalArgumentException");
-
-//     } catch (IllegalArgumentException ex) {
-//       // pass...
-//     }
-
     Instantiator.register(new
                           Instantiator(DataSerializableImpl.class, (byte) 42) {
         public DataSerializable newInstance() {
@@ -2131,6 +2102,7 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that an <code>Instantiator</code> is invoked at the
    * appropriate times.
    */
+  @Test
   public void testInstantiator() throws Exception {
     final boolean[] wasInvoked = new boolean[] { false };
     Instantiator.register(new
@@ -2158,6 +2130,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
+  @Test
   public void testInstantiator2() throws Exception {
     final boolean[] wasInvoked = new boolean[] { false };
     Instantiator.register(new
@@ -2185,6 +2158,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
+  @Test
   public void testInstantiator4() throws Exception {
     final boolean[] wasInvoked = new boolean[] { false };
     Instantiator.register(new
@@ -2212,7 +2186,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  static class Class_testInstantiator extends DataSerializerImpl {
+  private static class Class_testInstantiator extends DataSerializerImpl {
     public static Class supClass;
 
     public int getId() {
@@ -2230,11 +2204,12 @@ public class DataSerializableJUnitTest extends TestCase
       return false;
     }
   }
-  
+
   /**
    * Tests that an <code>CanonicalInstantiator</code> is invoked at the
    * appropriate times.
    */
+  @Test
   public void testCanonicalInstantiator() throws Exception {
     final boolean[] wasInvoked = new boolean[] { false };
     Instantiator.register(new
@@ -2262,11 +2237,13 @@ public class DataSerializableJUnitTest extends TestCase
       InternalInstantiator.unregister(CanonicalDataSerializableImpl.class, (byte) 45);
     }
   }
+
   /**
    * Tests that only one serializer is invoked when a serializer
    * specifies its supported classes.
    * Alos tests UDDS1.
    */
+  @Test
   public void testSupportedClasses() throws Exception {
     DataSerializer ds1 = DataSerializer.register(Class_testSupportedClasses1.class);
     int id = ds1.getId();
@@ -2293,6 +2270,7 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Make sure a user defined ds with an id of 2 bytes works.
    */
+  @Test
   public void testUDDS2() throws Exception {
     DataSerializer ds2 = DataSerializer.register(Class_testSupportedClasses3.class);
     int id2 = ds2.getId();
@@ -2315,6 +2293,7 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * Make sure a user defined ds with an id of42 bytes works.
    */
+  @Test
   public void testUDDS4() throws Exception {
     DataSerializer ds2 = DataSerializer.register(Class_testSupportedClasses4.class);
     int id2 = ds2.getId();
@@ -2334,7 +2313,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  static class Class_testSupportedClasses1 extends DataSerializerImpl {
+  private static class Class_testSupportedClasses1 extends DataSerializerImpl {
     public int getId() {
       return 29;
     }
@@ -2343,7 +2322,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
     public boolean toData(Object o, DataOutput out)
       throws IOException {
-      
+
       if (o instanceof NonDataSerializable) {
         fail("toData() should not be invoked with a " +
              "NonDataSerializable");
@@ -2351,8 +2330,8 @@ public class DataSerializableJUnitTest extends TestCase
       return false;
     }
   }
-  
-  static class Class_testSupportedClasses2
+
+  private static class Class_testSupportedClasses2
     extends NonDataSerializable.NonDSSerializer {
     public static boolean wasInvoked = false;
     public static boolean toDataInvoked = false;
@@ -2376,7 +2355,7 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  static class Class_testSupportedClasses3
+  private static class Class_testSupportedClasses3
     extends NonDataSerializable.NonDSSerializer {
     public static boolean wasInvoked = false;
     public static boolean toDataInvoked = false;
@@ -2399,23 +2378,31 @@ public class DataSerializableJUnitTest extends TestCase
       return super.fromData(in);
     }
   }
-  static class Class_testSupportedClasses4
-    extends NonDataSerializable.NonDSSerializer {
+
+  private static class Class_testSupportedClasses4 extends NonDataSerializable.NonDSSerializer {
+
     public static boolean wasInvoked = false;
     public static boolean toDataInvoked = false;
     public static boolean fromDataInvoked = false;
+
     public int getId() {
       return 1000000;
     }
+
+    @Override
     public Class[] getSupportedClasses() {
       wasInvoked = true;
       return super.getSupportedClasses();
     }
+
+    @Override
     public boolean toData(Object o, DataOutput out)
       throws IOException {
       toDataInvoked = true;
       return super.toData(o, out);
     }
+
+    @Override
     public Object fromData(DataInput in)
       throws IOException, ClassNotFoundException {
       fromDataInvoked = true;
@@ -2429,9 +2416,10 @@ public class DataSerializableJUnitTest extends TestCase
    *
    * This test is disabled due a bug in JaCoCo 0.6.2 agent while
    * handling stackoverflow exceptions during tests.
-   *
    */
-  public void disabled_testCyclicalObjectGraph() throws IOException {
+  @Ignore("disabled due a bug in JaCoCo 0.6.2 agent while handling stackoverflow exceptions")
+  @Test
+  public void testCyclicalObjectGraph() throws Exception {
     Link link1 = new Link(1);
     Link link2 = new Link(2);
     link1.next = link2;
@@ -2458,13 +2446,14 @@ public class DataSerializableJUnitTest extends TestCase
     } finally {
       SystemFailureTestHook.setExpectedFailureClass(null);
     }
-    
+
   }
 
   /**
    * Tests that data serializing the same object through two different
    * reference paths does not preserve referential integrity.
    */
+  @Test
   public void testReferentialIntegrity() throws Exception {
     Link top = new Link(1);
     Link left = new Link(2);
@@ -2499,20 +2488,23 @@ public class DataSerializableJUnitTest extends TestCase
    * Tests that <code>RegistrationListener</code>s are invoked at the
    * proper times.
    */
+  @Test
   public void testRegistrationListeners() {
     final DataSerializer[] array = new DataSerializer[2];
 
     TestRegistrationListener l1 = new TestRegistrationListener() {
+        @Override
         public void newDataSerializer2(DataSerializer ds) {
           array[0] = ds;
         }
       };
     TestRegistrationListener l2 = new TestRegistrationListener() {
+        @Override
         public void newDataSerializer2(DataSerializer ds) {
           array[1] = ds;
         }
       };
-    
+
     InternalDataSerializer.addRegistrationListener(l1);
     InternalDataSerializer.addRegistrationListener(l2);
 
@@ -2535,22 +2527,25 @@ public class DataSerializableJUnitTest extends TestCase
     Class c = DataSerializableImpl.class;
     id = (byte) 100;
     final Instantiator inst0 = new Instantiator(c, id) {
+        @Override
         public DataSerializable newInstance() {
           return new DataSerializableImpl();
         }
       };
 
     TestRegistrationListener l3 = new TestRegistrationListener() {
+        @Override
         public void newInstantiator2(Instantiator inst) {
           assertEquals(inst0, inst);
         }
       };
     TestRegistrationListener l4 = new TestRegistrationListener() {
+        @Override
         public void newInstantiator2(Instantiator inst) {
           assertEquals(inst0, inst);
         }
       };
-    
+
     InternalDataSerializer.addRegistrationListener(l3);
     InternalDataSerializer.addRegistrationListener(l4);
 
@@ -2566,15 +2561,20 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  static class DS42 extends DataSerializerImpl {
+  private static class DS42 extends DataSerializerImpl {
+
+    @Override
     public int getId() {
       return 42;
     }
+
+    @Override
     public Class[] getSupportedClasses() {
       return new Class[]{DS42.class};
     }
   }
 
+  @Test
   public void testIllegalSupportedClasses() {
     tryToSupport(String.class);
     tryToSupport(java.net.InetAddress.class);
@@ -2615,6 +2615,7 @@ public class DataSerializableJUnitTest extends TestCase
     tryToSupport(String[].class);
     tryToSupport(Object[].class);
   }
+
   private void tryToSupport(final Class c) {
     illegalClass = c;
     try {
@@ -2630,8 +2631,8 @@ public class DataSerializableJUnitTest extends TestCase
   }
 
   protected static Class illegalClass = null;
-  
-  public static class IllegalDS extends DataSerializerImpl {
+
+  private static class IllegalDS extends DataSerializerImpl {
     public IllegalDS() {
     }
     public int getId() {
@@ -2641,6 +2642,7 @@ public class DataSerializableJUnitTest extends TestCase
       return new Class[]{illegalClass};
     }
   }
+
   /**
    * Data serializes and then data de-serializes the given object and
    * asserts that the class of object the pre- and post- data
@@ -2664,9 +2666,8 @@ public class DataSerializableJUnitTest extends TestCase
    *
    * @since 4.0
    */
-  public void testSubclasses()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testSubclasses() throws Exception {
     checkClass(new Date());
     checkClass(new Date(){ });
 
@@ -2715,7 +2716,8 @@ public class DataSerializableJUnitTest extends TestCase
    * {@link StatArchiveWriter#readCompactValue}. Also added test for
    * ByteBufferInputStream#readUnsigned* methods (bug #41197).
    */
-  public void testStatArchiveCompactValueSerialization() throws IOException {
+  @Test
+  public void testStatArchiveCompactValueSerialization() throws Exception {
     // test all combos of valueToTest and + and -offsets
     long[] valuesToTest = new long[] { 0, Byte.MAX_VALUE, Byte.MIN_VALUE,
         Short.MAX_VALUE, Short.MIN_VALUE, Integer.MAX_VALUE, Integer.MIN_VALUE,
@@ -2778,15 +2780,13 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-  ///////////////////////  Inner Classes  ///////////////////////
-
   /**
    * A <code>DataSerializer</code> that provides default
    * implementations of its methods.
    */
-  static abstract class DataSerializerImpl extends DataSerializer {
-    public DataSerializerImpl() {
+  private static abstract class DataSerializerImpl extends DataSerializer {
 
+    public DataSerializerImpl() {
     }
 
     public boolean toData(Object o, DataOutput out)
@@ -2806,7 +2806,8 @@ public class DataSerializableJUnitTest extends TestCase
    * A class that implements {@link Serializable} and has fields of
    * each type.
    */
-  public static class SerializableImpl implements Serializable {
+  private static class SerializableImpl implements Serializable {
+
     protected byte byteField;
     protected short shortField;
     protected int intField;
@@ -2830,8 +2831,6 @@ public class DataSerializableJUnitTest extends TestCase
     protected int unsignedByteField;
     protected int unsignedShortField;
 
-    //////////////////////  Constructors  //////////////////////
-
     /**
      * Creates a new <code>SerializableImpl</code> whose contents
      * is randomly generated.
@@ -2857,7 +2856,7 @@ public class DataSerializableJUnitTest extends TestCase
 
       this.unsignedByteField = random.nextInt(256);
       this.unsignedShortField = random.nextInt(65536);
-      
+
       int length = random.nextInt(100);
       StringBuffer sb = new StringBuffer();
       for (int i = 0; i < length; i++) {
@@ -2877,12 +2876,11 @@ public class DataSerializableJUnitTest extends TestCase
 
     }
 
-    //////////////////////  Instance Methods  //////////////////////
-
     /**
      * Two <code>SerializableImpl</code>s are equal if their
      * contents are equal.
      */
+    @Override
     public boolean equals(Object o) {
       if (!(o instanceof SerializableImpl)) {
         return false;
@@ -2908,13 +2906,14 @@ public class DataSerializableJUnitTest extends TestCase
         this.booleanFieldPrim == other.booleanFieldPrim &&
         this.unsignedByteField == other.unsignedByteField &&
         this.unsignedShortField == other.unsignedShortField &&
-        (this.stringField == null || 
+        (this.stringField == null ||
          this.stringField.equals(other.stringField)) &&
         (this.objectField == null ||
          this.objectField.equals(other.objectField)) &&
         true;
     }
 
+    @Override
     public String toString() {
       StringBuffer sb = new StringBuffer();
       sb.append(this.getClass().getName());
@@ -2952,14 +2951,15 @@ public class DataSerializableJUnitTest extends TestCase
       sb.append(this.objectField);
 
       return sb.toString();
-    } 
+    }
   }
 
   /**
    * A class that implements {@link DataSerializable}
+   *
+   * Also used by DataTypeJUnitTest
    */
-  public static class DataSerializableImpl extends SerializableImpl
-    implements DataSerializable {
+  public static class DataSerializableImpl extends SerializableImpl implements DataSerializable {
 
     /**
      * Creates a new <code>DataSerializableImpl</code> whose contents
@@ -2977,6 +2977,7 @@ public class DataSerializableJUnitTest extends TestCase
       super();
     }
 
+    @Override
     public void toData(DataOutput out) throws IOException {
       DataSerializer.writeByte(new Byte(this.byteField), out);
       DataSerializer.writeShort(new Short(this.shortField), out);
@@ -3003,6 +3004,7 @@ public class DataSerializableJUnitTest extends TestCase
       DataSerializer.writeObject(this.objectField, out);
     }
 
+    @Override
     public void fromData(DataInput in)
       throws IOException, ClassNotFoundException {
 
@@ -3032,44 +3034,52 @@ public class DataSerializableJUnitTest extends TestCase
     }
 
   }
-  
-  public static class VersionedDataSerializableImpl extends DataSerializableImpl implements VersionedDataSerializable {
-    public Version[] getSerializationVersions() { return new Version[] { Version.GFE_71 }; }
+
+  private static class VersionedDataSerializableImpl extends DataSerializableImpl implements VersionedDataSerializable {
+
+    @Override
+    public Version[] getSerializationVersions() {
+      return new Version[] { Version.GFE_71 };
+    }
 
     transient boolean preMethodInvoked;
 
-    public VersionedDataSerializableImpl() { }
-    
+    public VersionedDataSerializableImpl() {
+    }
+
     public VersionedDataSerializableImpl(Random random) {
       super(random);
     }
-    
+
     public void toDataPre_GFE_7_1_0_0(DataOutput out) throws IOException {
       this.preMethodInvoked = true;
       toData(out);
     }
-    
+
     public void fromDataPre_GFE_7_1_0_0(DataInput in) throws IOException, ClassNotFoundException {
       this.preMethodInvoked = true;
       fromData(in);
     }
-    
+
     public boolean preMethodInvoked() {
       return this.preMethodInvoked;
     }
   }
 
-  public static class CanonicalDataSerializableImpl extends SerializableImpl
-    implements DataSerializable {
+  private static class CanonicalDataSerializableImpl extends SerializableImpl implements DataSerializable {
+
     private static final byte SINGLETON_BYTE = 23;
     private static final CanonicalDataSerializableImpl singleton = new CanonicalDataSerializableImpl(new Random());
+
     public static CanonicalDataSerializableImpl create() {
         return singleton;
     }
+
     public static CanonicalDataSerializableImpl create(byte b) {
       assertEquals(SINGLETON_BYTE, b);
       return singleton;
     }
+
     /**
      * Creates a new <code>CanonicalDataSerializableImpl</code> whose contents
      * is randomly generated.
@@ -3078,10 +3088,12 @@ public class DataSerializableJUnitTest extends TestCase
       super(random);
     }
 
+    @Override
     public void toData(DataOutput out) throws IOException {
       out.writeByte(SINGLETON_BYTE);
     }
 
+    @Override
     public void fromData(DataInput in)
       throws IOException, ClassNotFoundException {
     }
@@ -3090,11 +3102,11 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * A class that replaces itself with an <code>Integer</code> when
-   * written. 
+   * written.
    */
-  static class ReplaceableImpl
-    implements DataSerializable.Replaceable {
+  private static class ReplaceableImpl implements DataSerializable.Replaceable {
 
+    @Override
     public Object replace() throws IOException {
       return new Integer(42);
     }
@@ -3106,7 +3118,8 @@ public class DataSerializableJUnitTest extends TestCase
    *
    * @since 3.5
    */
-  public static class NonDataSerializable {
+  private static class NonDataSerializable {
+
     protected int intValue;
     protected double doubleValue;
     protected String stringValue;
@@ -3124,7 +3137,6 @@ public class DataSerializableJUnitTest extends TestCase
     }
 
     protected NonDataSerializable() {
-
     }
 
     public boolean equals(Object o) {
@@ -3146,27 +3158,28 @@ public class DataSerializableJUnitTest extends TestCase
         ;
     }
 
-
     /**
      * A <code>Serializer</code> that data serializes instances of
-     * <code>NonDataSerializable</code>. 
+     * <code>NonDataSerializable</code>.
      */
-    public static class NonDSSerializer
-      extends DataSerializer {
+    public static class NonDSSerializer extends DataSerializer {
 
       private static final byte CLASS_ID = (byte) 100;
 
       public NonDSSerializer() {
-
       }
 
+      @Override
       public int getId() {
         return CLASS_ID;
       }
+
+      @Override
       public Class[] getSupportedClasses() {
         return new Class[] {NonDataSerializable.class};
       }
 
+      @Override
       public boolean toData(Object o, DataOutput out)
         throws IOException {
 
@@ -3188,6 +3201,7 @@ public class DataSerializableJUnitTest extends TestCase
         }
       }
 
+      @Override
       public Object fromData(DataInput in)
         throws IOException, ClassNotFoundException {
 
@@ -3209,7 +3223,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * A much more simple class to be data serialized
    */
-  public static class IntWrapper {
+  private static class IntWrapper {
+
     public int intValue;
 
     public static final byte CLASS_ID = (byte) 42;
@@ -3222,6 +3237,7 @@ public class DataSerializableJUnitTest extends TestCase
       this.intValue = intValue;
     }
 
+    @Override
     public boolean equals(Object o) {
       if (o instanceof IntWrapper) {
         IntWrapper other = (IntWrapper) o;
@@ -3236,29 +3252,33 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * A <code>DataSerializable</code> int wrapper
    */
-  public static class DSIntWrapper extends IntWrapper
-    implements DataSerializable {
+  private static class DSIntWrapper extends IntWrapper implements DataSerializable {
 
     public DSIntWrapper(Random random) {
       super(random);
     }
 
+    @Override
     public void toData(DataOutput out) throws IOException {
       out.writeInt(this.intValue);
     }
 
+    @Override
     public void fromData(DataInput in)
       throws IOException, ClassNotFoundException {
 
       this.intValue = in.readInt();
     }
   }
-  
-  public static class SerializableIntWrapper implements Serializable {
+
+  private static class SerializableIntWrapper implements Serializable {
+
     private int data;
+
     public SerializableIntWrapper(int intValue) {
       this.data = intValue;
     }
+
     @Override
     public int hashCode() {
       final int prime = 31;
@@ -3266,6 +3286,7 @@ public class DataSerializableJUnitTest extends TestCase
       result = prime * result + data;
       return result;
     }
+
     @Override
     public boolean equals(Object obj) {
       if (this == obj)
@@ -3284,7 +3305,8 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * A node in an object chain
    */
-  static class Link implements DataSerializable, Serializable {
+  private static class Link implements DataSerializable, Serializable {
+
     private int id;
     Link next;
     Link next2;
@@ -3294,15 +3316,16 @@ public class DataSerializableJUnitTest extends TestCase
     }
 
     public Link() {
-
     }
 
+    @Override
     public void toData(DataOutput out) throws IOException {
       out.writeInt(this.id);
       DataSerializer.writeObject(this.next, out);
       DataSerializer.writeObject(this.next2, out);
     }
 
+    @Override
     public void fromData(DataInput in)
       throws IOException, ClassNotFoundException {
       this.id = in.readInt();
@@ -3314,8 +3337,7 @@ public class DataSerializableJUnitTest extends TestCase
   /**
    * A <code>RegistrationListener</code> used for testing
    */
-  static class TestRegistrationListener
-    implements InternalDataSerializer.RegistrationListener {
+  private static class TestRegistrationListener implements InternalDataSerializer.RegistrationListener {
 
     /** Was this listener invoked? */
     private boolean invoked = false;
@@ -3323,8 +3345,6 @@ public class DataSerializableJUnitTest extends TestCase
     /** An error thrown in a callback */
     private Throwable callbackError = null;
 
-    //////////////////////  Instance Methods  //////////////////////
-
     /**
      * Returns wether or not one of this listener methods was invoked.
      * Before returning, the <code>invoked</code> flag is cleared.
@@ -3335,7 +3355,7 @@ public class DataSerializableJUnitTest extends TestCase
       this.invoked = false;
       return value;
     }
-  
+
     private void checkForError() {
       if (this.callbackError != null) {
         AssertionError error =
@@ -3345,12 +3365,13 @@ public class DataSerializableJUnitTest extends TestCase
       }
     }
 
+    @Override
     public final void newDataSerializer(DataSerializer ds) {
       this.invoked = true;
       try {
         newDataSerializer2(ds);
 
-      } 
+      }
       catch (VirtualMachineError e) {
         SystemFailure.initiateFailure(e);
         throw e;
@@ -3365,12 +3386,13 @@ public class DataSerializableJUnitTest extends TestCase
       throw new UnsupportedOperationException(s);
     }
 
+    @Override
     public final void newInstantiator(Instantiator instantiator) {
       this.invoked = true;
       try {
         newInstantiator2(instantiator);
 
-      } 
+      }
       catch (VirtualMachineError e) {
         SystemFailure.initiateFailure(e);
         throw e;
@@ -3391,11 +3413,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>byte[][]</code> using {@link
-   * DataSerializer#writeObject}. 
+   * DataSerializer#writeObject}.
    */
-  public void testByteArrayArrayObject()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testByteArrayArrayObject() throws Exception {
     byte[] ar0 = new byte[] { (byte) 1, (byte) 2, (byte) 3 };
     byte[] ar1 = new byte[] { (byte) 4, (byte) 5, (byte) 6 };
     byte[] ar2 = new byte[] { (byte) 7, (byte) 8, (byte) 9 };
@@ -3422,11 +3443,10 @@ public class DataSerializableJUnitTest extends TestCase
 
   /**
    * Tests data serializing a <code>byte[][]</code> using {@link
-   * DataSerializer#writeObjectArray}. 
+   * DataSerializer#writeObjectArray}.
    */
-  public void testByteArrayArray()
-    throws IOException, ClassNotFoundException {
-
+  @Test
+  public void testByteArrayArray() throws Exception {
     byte[] ar0 = new byte[] { (byte) 1, (byte) 2, (byte) 3 };
     byte[] ar1 = new byte[] { (byte) 4, (byte) 5, (byte) 6 };
     byte[] ar2 = new byte[] { (byte) 7, (byte) 8, (byte) 9 };
@@ -3452,8 +3472,8 @@ public class DataSerializableJUnitTest extends TestCase
   }
 
   // see bug 41721
-  public void testArrayMinShortLength()
-    throws IOException, ClassNotFoundException {
+  @Test
+  public void testArrayMinShortLength() throws Exception {
     DataOutputStream out = getDataOutput();
     DataSerializer.writeByteArray(new byte[0x8000], out);
     out.flush();
@@ -3462,8 +3482,9 @@ public class DataSerializableJUnitTest extends TestCase
     byte[] array = DataSerializer.readByteArray(in);
     assertEquals(0x8000, array.length);
   }
-  public void testArrayMaxShortLength()
-    throws IOException, ClassNotFoundException {
+
+  @Test
+  public void testArrayMaxShortLength() throws Exception {
     DataOutputStream out = getDataOutput();
     DataSerializer.writeByteArray(new byte[0xFFFF], out);
     out.flush();
@@ -3478,6 +3499,7 @@ public class DataSerializableJUnitTest extends TestCase
    * is > 0xFFFF, but who's utf-8 encoded length is < 0xFFFF
    * See bug 40932.
    */
+  @Test
   public void testStringEncodingLengthCrossesBoundry() throws Exception {
     StringBuffer sb = new StringBuffer(0xFFFF);
     for (int i=0; i < 0xFFFF; i++) {
@@ -3501,16 +3523,18 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals(value, value2);
   }
 
-  enum DAY_OF_WEEK implements PdxSerializerObject {
+  private enum DAY_OF_WEEK implements PdxSerializerObject {
     MON, TUE, WED, THU, FRI, SAT, SUN
   }
-  enum MONTH implements PdxSerializerObject {
+
+  private enum MONTH implements PdxSerializerObject {
     JAN, FEB, MAR
   }
-  
+
   /**
    * Tests Dataserializing an Enum
    */
+  @Test
   public void testEnum() throws Exception {
     DAY_OF_WEEK e = DAY_OF_WEEK.SUN;
     MONTH m = MONTH.FEB;
@@ -3543,6 +3567,7 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals(m, m2);
   }
 
+  @Test
   public void testObjectEnum() throws Exception {
     final String propName = "DataSerializer.DEBUG";
     System.setProperty(propName, "true");
@@ -3553,7 +3578,7 @@ public class DataSerializableJUnitTest extends TestCase
       DataSerializer.writeObject(e, out);
       DataSerializer.writeObject(m, out);
       out.flush();
-  
+
       DataInput in = getDataInput();
       DAY_OF_WEEK e2 = (DAY_OF_WEEK)DataSerializer.readObject(in);
       MONTH m2 = (MONTH)DataSerializer.readObject(in);
@@ -3565,14 +3590,15 @@ public class DataSerializableJUnitTest extends TestCase
       System.getProperties().remove(propName);
     }
   }
-  
+
   /**
    * Usually the DataInput instance passed to DataSerializer.readObject is an instance of InputStream.
    * Make sure that an object that uses standard java serialization can be written and read from a
    * DataInput that is not an instance of InputStream.
    * See bug 47249.
    */
-  public void testOddDataInput() throws IOException, ClassNotFoundException {
+  @Test
+  public void testOddDataInput() throws Exception {
     SerializableIntWrapper o = new SerializableIntWrapper(-1);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataSerializer.writeObject(o, new DataOutputStream(baos));
@@ -3581,7 +3607,8 @@ public class DataSerializableJUnitTest extends TestCase
     assertEquals (o, o2);
   }
   
-  public static class OddDataInput implements DataInput {
+  private static class OddDataInput implements DataInput {
+
     private ByteBufferInputStream bbis;
 
     public OddDataInput(ByteBuffer bb) {
@@ -3664,7 +3691,6 @@ public class DataSerializableJUnitTest extends TestCase
     }
   }
 
-
-  class Foo {
+  private class Foo {
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/FileUtilJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/FileUtilJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/FileUtilJUnitTest.java
index 31211c6..473c180 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/FileUtilJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/FileUtilJUnitTest.java
@@ -30,12 +30,9 @@ import java.io.IOException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-/**
- *
- */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FileUtilJUnitTest {
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/GemFireVersionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/GemFireVersionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/GemFireVersionJUnitTest.java
index 60995bb..fc4ab1b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/GemFireVersionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/GemFireVersionJUnitTest.java
@@ -16,30 +16,32 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 /**
  * This test prints out the version information obtained from the
  * {@link GemFireVersion} class.  It provides a record of what version
  * of GemFire (and the JDK) was used to run the unit tests.
  */
 @Category(UnitTest.class)
-public class GemFireVersionJUnitTest extends TestCase {
+public class GemFireVersionJUnitTest {
 
   /**
    * Prints both the GemFire version info and the system properties.
    * We have to print both 
    */
+  @Test
   public void testPrintInfo() {
-	ByteArrayOutputStream baos = new ByteArrayOutputStream();
-	PrintStream ps = new PrintStream(baos);
+	  ByteArrayOutputStream baos = new ByteArrayOutputStream();
+	  PrintStream ps = new PrintStream(baos);
     GemFireVersion.print(ps);
     final String versionOutput = baos.toString();
     System.out.println(versionOutput);
@@ -50,6 +52,7 @@ public class GemFireVersionJUnitTest extends TestCase {
     assertTrue(versionOutput.contains("Running on:"));
   }
 
+  @Test
   public void testMajorMinorVersions() {
     assertEquals(1, GemFireVersion.getMajorVersion("1.0.3"));
     assertEquals(33, GemFireVersion.getMajorVersion("33.0.3"));
@@ -77,7 +80,8 @@ public class GemFireVersionJUnitTest extends TestCase {
     assertTrue("7.0.2.13 should be == 7.0.2.13", GemFireVersion.compareVersions("7.0.2.13", "7.0.2.13", true) == 0);
     assertTrue("7.0.2.15 should be > 7.0.2.13", GemFireVersion.compareVersions("7.0.2.14", "7.0.2.13", true) > 0);
   }
-  
+
+  @Test
   public void testVersionClass() throws Exception {
     compare(Version.GFE_662, Version.GFE_66);
     compare(Version.GFE_6622, Version.GFE_662);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/HeapDataOutputStreamJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/HeapDataOutputStreamJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/HeapDataOutputStreamJUnitTest.java
index 966da83..f723d4c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/HeapDataOutputStreamJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/HeapDataOutputStreamJUnitTest.java
@@ -16,13 +16,14 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
-import junit.framework.TestCase;
-
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.DataSerializer;
@@ -35,11 +36,11 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * TODO right now this just tests the new
  * write(ByteBuffer) method. We might want
  * to add some unit tests for the existing methods.
- *
  */
 @Category(UnitTest.class)
-public class HeapDataOutputStreamJUnitTest extends TestCase {
-  
+public class HeapDataOutputStreamJUnitTest {
+
+  @Test
   public void testWriteByteBuffer() {
     HeapDataOutputStream out = new HeapDataOutputStream(64, Version.CURRENT);
     
@@ -56,8 +57,8 @@ public class HeapDataOutputStreamJUnitTest extends TestCase {
     
     assertEquals(new String(bytes) , new String(actual));
   }
-  
 
+  @Test
   public void testWriteByteBufferCopyUseBuffer() {
     ByteBuffer buf = ByteBuffer.allocate(32);
     HeapDataOutputStream out = new HeapDataOutputStream(buf, Version.CURRENT, true);
@@ -95,6 +96,7 @@ public class HeapDataOutputStreamJUnitTest extends TestCase {
     assertEquals(expected, tmp);
   }
 
+  @Test
   public void testWriteByteBufferNoCopyUseBuffer() {
     ByteBuffer buf = ByteBuffer.allocate(32);
     HeapDataOutputStream out = new HeapDataOutputStream(buf, Version.CURRENT, true);
@@ -135,7 +137,8 @@ public class HeapDataOutputStreamJUnitTest extends TestCase {
     ByteBuffer expected = ByteBuffer.wrap(expectedBytes);
     assertEquals(expected, tmp);
   }
-  
+
+  @Test
   public void testWriteJavaSerializeNoCopy() throws IOException, ClassNotFoundException {
     byte[] bytes = new byte[2000];
     for (int i = 0; i < bytes.length; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/LineWrapUnitJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/LineWrapUnitJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/LineWrapUnitJUnitTest.java
index 93d55dc..39baa16 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/LineWrapUnitJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/LineWrapUnitJUnitTest.java
@@ -16,21 +16,20 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.util.Arrays;
 import java.util.List;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
 @Category(UnitTest.class)
-public class LineWrapUnitJUnitTest extends TestCase {
-  
+public class LineWrapUnitJUnitTest {
+
+  @Test
   public void test() {
     String test = new String("aaa aaaaa  aaa aaaa");
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/NanoTimerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/NanoTimerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/NanoTimerJUnitTest.java
index 6ecd9f2..b017e25 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/NanoTimerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/NanoTimerJUnitTest.java
@@ -22,7 +22,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.NanoTimer.TimeService;
-import com.gemstone.gemfire.internal.util.StopWatch;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
@@ -33,21 +32,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class NanoTimerJUnitTest {
   
-  /**
-   * Simple deterministic clock. Any time you want
-   * your clock to tick call incTime.
-   */
-  private class TestTimeService implements TimeService {
-    private long now;
-    public void incTime() {
-      this.now++;
-    }
-    @Override
-    public long getTime() {
-      return this.now;
-    }
-  }
-
   @Test
   public void testMillisToNanos() {
     assertEquals(0, NanoTimer.millisToNanos(0));
@@ -130,4 +114,19 @@ public class NanoTimerJUnitTest {
     assertTrue(timer.getTimeSinceReset() < timer.getTimeSinceConstruction());
     assertTrue(timer.getTimeSinceReset() <= ts.getTime() - timer.getLastResetTime());
   }
+
+  /**
+   * Simple deterministic clock. Any time you want
+   * your clock to tick call incTime.
+   */
+  private class TestTimeService implements TimeService {
+    private long now;
+    public void incTime() {
+      this.now++;
+    }
+    @Override
+    public long getTime() {
+      return this.now;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/ObjIdMapJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ObjIdMapJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ObjIdMapJUnitTest.java
index 5ce3926..ce9b997 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/ObjIdMapJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/ObjIdMapJUnitTest.java
@@ -16,27 +16,28 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.lang.ref.WeakReference;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.*;
-
 /**
  * This class tests the functionality of the {@link ObjIdMap} class.
  */
 @Category(UnitTest.class)
-public class ObjIdMapJUnitTest extends TestCase {
-
-  public ObjIdMapJUnitTest(String name) {
-    super(name);
-  }
-
-  ////////  Test methods
+public class ObjIdMapJUnitTest {
 
+  @Test
   public void testSimplePut() {
     ObjIdMap map = new ObjIdMap();
     int key = 4;
@@ -45,12 +46,14 @@ public class ObjIdMapJUnitTest extends TestCase {
     assertSame(value, map.get(key));
   }
 
+  @Test
   public void testGetNotThere() {
     ObjIdMap map = new ObjIdMap();
     int key = 4;
     assertSame(null, map.get(key));
   }
 
+  @Test
   public void testSimpleContainsKey() {
     ObjIdMap map = new ObjIdMap();
     int key = 4;
@@ -59,6 +62,7 @@ public class ObjIdMapJUnitTest extends TestCase {
     assertTrue(map.containsKey(key));
   }
 
+  @Test
   public void testSimpleRemove() {
     ObjIdMap map = new ObjIdMap();
     int key = 4;
@@ -67,6 +71,7 @@ public class ObjIdMapJUnitTest extends TestCase {
     assertSame(value, map.remove(key));
   }
 
+  @Test
   public void testSimpleValues() {
     ObjIdMap map = new ObjIdMap();
     for (int i = 0; i < 20; i++) {
@@ -88,6 +93,7 @@ public class ObjIdMapJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testRandomMap() {
     final ObjIdMap map = new ObjIdMap();
     final int size = 1000;
@@ -157,8 +163,8 @@ public class ObjIdMapJUnitTest extends TestCase {
     }
   }
 
+  @Test
   public void testRandomGrowRemoveRelease() {
-
     ObjIdMap map = new ObjIdMap();
     Random random = new Random(System.currentTimeMillis());
     List saver = new ArrayList();
@@ -229,6 +235,7 @@ public class ObjIdMapJUnitTest extends TestCase {
     );
   }
 
+  @Test
   public void testIterator() {
     int size = 10;
     ObjIdMap map = new ObjIdMap();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/OneTaskOnlyDecoratorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/OneTaskOnlyDecoratorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/OneTaskOnlyDecoratorJUnitTest.java
index dc71a85..f57a8fb 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/OneTaskOnlyDecoratorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/OneTaskOnlyDecoratorJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
@@ -23,24 +25,20 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
-/**
- *
- */
-@SuppressWarnings("synthetic-access")
 @Category(UnitTest.class)
-public class OneTaskOnlyDecoratorJUnitTest extends TestCase {
+public class OneTaskOnlyDecoratorJUnitTest {
   
   /**
    * Test to make sure we only execute the task once
    * no matter how many times we schedule it. 
    */
-  public void testExecuteOnlyOnce() throws InterruptedException {
+  @Test
+  public void testExecuteOnlyOnce() throws Exception {
     ScheduledExecutorService ex = Executors.newScheduledThreadPool(1);
    
     MyConflationListener listener = new MyConflationListener();
@@ -80,7 +78,8 @@ public class OneTaskOnlyDecoratorJUnitTest extends TestCase {
    * Test to make sure we reschedule the task for execution 
    * if it has already in progress.
    */
-  public void testReschedule() throws InterruptedException {
+  @Test
+  public void testReschedule() throws Exception {
     ScheduledExecutorService ex = Executors.newScheduledThreadPool(1);
     OneTaskOnlyExecutor decorator = new OneTaskOnlyExecutor(ex);
     
@@ -121,7 +120,8 @@ public class OneTaskOnlyDecoratorJUnitTest extends TestCase {
    * Test to make sure we reschedule the task for execution 
    * if the new requested execution is earlier than the previous one
    */
-  public void testRescheduleForEarlierTime() throws InterruptedException {
+  @Test
+  public void testRescheduleForEarlierTime() throws Exception {
     ScheduledExecutorService ex = Executors.newScheduledThreadPool(1);
     MyConflationListener listener = new MyConflationListener();
     OneTaskOnlyExecutor decorator = new OneTaskOnlyExecutor(ex, listener);
@@ -149,10 +149,10 @@ public class OneTaskOnlyDecoratorJUnitTest extends TestCase {
     assertTrue(elapsed < TimeUnit.SECONDS.toNanos(120));
   }
 
-
   private static class MyConflationListener extends OneTaskOnlyExecutor.ConflatedTaskListenerAdapter {
     private int dropCount;
 
+    @Override
     public void taskDropped() {
       dropCount++;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/internal/PutAllOperationContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/PutAllOperationContextJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/PutAllOperationContextJUnitTest.java
index 835cada..b2ca5c4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/PutAllOperationContextJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/PutAllOperationContextJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal;
 
+import static org.junit.Assert.*;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -24,6 +26,7 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.operations.PutAllOperationContext;
@@ -32,11 +35,10 @@ import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
 import com.gemstone.gemfire.internal.cache.Token;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
-
 @Category(UnitTest.class)
-public class PutAllOperationContextJUnitTest extends TestCase {
-  
+public class PutAllOperationContextJUnitTest {
+
+  @Test
   public void testIllegalMapMods() {
     LinkedHashMap<String, String> m = new LinkedHashMap<>();
     m.put("1", "1");
@@ -91,6 +93,7 @@ public class PutAllOperationContextJUnitTest extends TestCase {
   /**
    * Make sure that we do not expose the internal Token.INVALID to customers
    */
+  @Test
   public void testInvalidToken() {
     LinkedHashMap<String, Object> m = new LinkedHashMap<>();
     m.put("INVALID_TOKEN", Token.INVALID);
@@ -111,9 +114,11 @@ public class PutAllOperationContextJUnitTest extends TestCase {
     assertEquals(null, me.getValue());
     assertEquals(Token.INVALID, m.get("INVALID_TOKEN"));
   }
+
   /**
    * Make sure that we do not expose the internal CachedDeserializable to customers
    */
+  @Test
   public void testCachedDeserializable() {
     LinkedHashMap<String, Object> m = new LinkedHashMap<>();
     Object v = Integer.valueOf(99);
@@ -143,6 +148,7 @@ public class PutAllOperationContextJUnitTest extends TestCase {
     assertEquals(hm.hashCode(), opMap.hashCode());
   }
 
+  @Test
   public void testLegalMapMods() {
     LinkedHashMap<String, String> m = new LinkedHashMap<>();
     m.put("1", "1");


[14/16] incubator-geode git commit: Updating and fixing tests

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/OpExecutorImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/OpExecutorImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/OpExecutorImplJUnitTest.java
index 8171c67..6e370cd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/OpExecutorImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/OpExecutorImplJUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.client.internal;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -28,8 +30,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledExecutorService;
 
-import junit.framework.TestCase;
-
+import org.junit.Before;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.CancelCriterion;
@@ -46,11 +48,9 @@ import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
-public class OpExecutorImplJUnitTest extends TestCase {
+public class OpExecutorImplJUnitTest {
+
   DummyManager manager;
   private LogWriter logger;
   private DummyEndpointManager endpointManager;
@@ -65,7 +65,8 @@ public class OpExecutorImplJUnitTest extends TestCase {
   protected int getPrimary;
   protected int getBackups;
   private CancelCriterion cancelCriterion;
-  
+
+  @Before
   public void setUp() {
     this.logger = new LocalLogWriter(InternalLogWriter.FINEST_LEVEL, System.out);
     this.endpointManager = new DummyEndpointManager();
@@ -74,22 +75,23 @@ public class OpExecutorImplJUnitTest extends TestCase {
     riTracker = new RegisterInterestTracker();
     cancelCriterion = new CancelCriterion() {
 
+      @Override
       public String cancelInProgress() {
         return null;
       }
 
+      @Override
       public RuntimeException generateCancelledException(Throwable e) {
         return null;
       }
     };
   }
   
-  public void tearDown() throws InterruptedException {
-  }
-  
+  @Test
   public void testExecute() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 3, 10, false, cancelCriterion, null);
     Object result = exec.execute(new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         return "hello";
       }
@@ -108,6 +110,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
     try {
     result = exec.execute(new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         throw new SocketTimeoutException();
       }
@@ -130,6 +133,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
     try {
       result = exec.execute(new Op() {
+        @Override
         public Object attempt(Connection cnx) throws Exception {
           throw new ServerOperationException("Something didn't work");
         }
@@ -151,6 +155,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
     try {
       result = exec.execute(new Op() {
+        @Override
         public Object attempt(Connection cnx) throws Exception {
           throw new IOException("Something didn't work");
         }
@@ -179,13 +184,15 @@ public class OpExecutorImplJUnitTest extends TestCase {
     getPrimary = 0;
     getBackups = 0;
   }
-  
+
+  @Test
   public void testExecuteOncePerServer() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, -1, 10, false, cancelCriterion, null);
     
     manager.numServers = 5;
     try {
       exec.execute(new Op() {
+        @Override
         public Object attempt(Connection cnx) throws Exception {
           throw new IOException("Something didn't work");
         }
@@ -204,13 +211,15 @@ public class OpExecutorImplJUnitTest extends TestCase {
     assertEquals(6, invalidateConnections);
     assertEquals(6, serverCrashes);
   }
-  
+
+  @Test
   public void testRetryFailedServers() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager, queueManager, endpointManager, riTracker, 10, 10, false, cancelCriterion, null);
     
     manager.numServers = 5;
     try {
       exec.execute(new Op() {
+        @Override
         public Object attempt(Connection cnx) throws Exception {
           throw new IOException("Something didn't work");
         }
@@ -230,10 +239,12 @@ public class OpExecutorImplJUnitTest extends TestCase {
     assertEquals(11, serverCrashes);
   }
 
+  @Test
   public void testExecuteOn() throws Exception {
     OpExecutorImpl exec = new OpExecutorImpl(manager,queueManager, endpointManager, riTracker, 3, 10, false, cancelCriterion, null);
     ServerLocation server = new ServerLocation("localhost", -1);
     Object result = exec.executeOn(server, new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         return "hello";
       }
@@ -252,6 +263,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
     try {
     result = exec.executeOn(server, new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         throw new SocketTimeoutException();
       }
@@ -273,6 +285,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
     try {
       result = exec.executeOn(server,new Op() {
+        @Override
         public Object attempt(Connection cnx) throws Exception {
           throw new ServerOperationException("Something didn't work");
         }
@@ -301,6 +314,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
       logger.info(addExpected);
       try {
         result = exec.executeOn(server,new Op() {
+          @Override
             public Object attempt(Connection cnx) throws Exception {
               throw new Exception("Something didn't work");
             }
@@ -321,10 +335,12 @@ public class OpExecutorImplJUnitTest extends TestCase {
     assertEquals(1, invalidateConnections);
     assertEquals(1, serverCrashes);
   }
-  
+
+  @Test
   public void testExecuteOnAllQueueServers() {
     OpExecutorImpl exec = new OpExecutorImpl(manager,queueManager, endpointManager, riTracker, 3, 10, false, cancelCriterion, null);
     exec.executeOnAllQueueServers(new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         return "hello";
       }
@@ -342,6 +358,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
     queueManager.backups = 3;
     exec.executeOnAllQueueServers(new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         throw new SocketTimeoutException();
       }
@@ -361,6 +378,7 @@ public class OpExecutorImplJUnitTest extends TestCase {
     queueManager.backups = 3;
     Object result = exec.executeOnQueuesAndReturnPrimaryResult(new Op() {
       int i = 0;
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         i++;
         if(i < 15) {
@@ -382,10 +400,12 @@ public class OpExecutorImplJUnitTest extends TestCase {
     
   }
 
+  @Test
   public void testThreadLocalConnection() {
     OpExecutorImpl exec = new OpExecutorImpl(manager,queueManager, endpointManager, riTracker, 3, 10, true, cancelCriterion, null);
     ServerLocation server = new ServerLocation("localhost", -1);
     Op op = new Op() {
+      @Override
       public Object attempt(Connection cnx) throws Exception {
         //do nothing
         return cnx;
@@ -417,21 +437,19 @@ public class OpExecutorImplJUnitTest extends TestCase {
     assertEquals(0, returns);
   }
   
-  public class DummyManager implements ConnectionManager {
+  private class DummyManager implements ConnectionManager {
+
     protected int numServers  = Integer.MAX_VALUE;
     private int currentServer = 0;
 
-    
     public DummyManager() {
     }
-    
-    
 
+    @Override
     public void emergencyClose() {
     }
 
-
-
+    @Override
     public Connection borrowConnection(long aquireTimeout) {
       borrows++;
       return new DummyConnection(new ServerLocation("localhost", currentServer++ % numServers));
@@ -440,180 +458,217 @@ public class OpExecutorImplJUnitTest extends TestCase {
     /* (non-Javadoc)
      * @see com.gemstone.gemfire.cache.client.internal.pooling.ConnectionManager#borrowConnection(com.gemstone.gemfire.distributed.internal.ServerLocation, long)
      */
+    @Override
     public Connection borrowConnection(ServerLocation server, long aquireTimeout,boolean onlyUseExistingCnx) {
       borrows++;
       return new DummyConnection(server);
     }
 
+    @Override
     public void close(boolean keepAlive) {
-      
-    }
-
-    public Map getEndpointMap() {
-      return null;
     }
 
+    @Override
     public void returnConnection(Connection connection) {
       returns++;
       
     }
+
+    @Override
     public void returnConnection(Connection connection, boolean accessed) {
       returns++;
       
     }
 
+    @Override
     public void start(ScheduledExecutorService backgroundProcessor) {
     }
 
-    public Connection exchangeConnection(Connection conn, Set excludedServers,
-        long aquireTimeout) {
+    @Override
+    public Connection exchangeConnection(Connection conn, Set excludedServers, long aquireTimeout) {
       if(excludedServers.size() >= numServers) {
         throw new NoAvailableServersException();
       }
       exchanges++;
       return new DummyConnection(new ServerLocation("localhost", currentServer++ % numServers));
     }
+
+    @Override
     public int getConnectionCount() {
       return 0;
     }
+
+    @Override
     public Connection getConnection(Connection conn) {
       return conn;
     }
-    public void activate(Connection conn) {}
-    public void passivate(Connection conn, boolean accessed) {}
+
+    @Override
+    public void activate(Connection conn) {
+    }
+
+    @Override
+    public void passivate(Connection conn, boolean accessed) {
+    }
   }
   
-  public class DummyConnection implements Connection {
+  private class DummyConnection implements Connection {
     
     private ServerLocation server;
 
     public DummyConnection(ServerLocation serverLocation) {
       this.server = serverLocation;
     }
+
+    @Override
     public void close(boolean keepAlive) throws Exception {
     }
+
+    @Override
     public void destroy() {
       invalidateConnections++;
     }
+
+    @Override
     public boolean isDestroyed() {
       return false;
     }
+
+    @Override
     public ByteBuffer getCommBuffer() {
       return null;
     }
 
+    @Override
     public ServerLocation getServer() {
       return server;
     }
 
+    @Override
     public Socket getSocket() {
       return null;
     }
 
+    @Override
     public ConnectionStats getStats() {
       return null;
     }
-    
+
+    @Override
     public int getDistributedSystemId() {
       return 0;
     }
 
-
+    @Override
     public Endpoint getEndpoint() {
       return new Endpoint(null,null,null,null, null);
     }
 
-    public void setEndpoint(Endpoint endpoint) {
-    }
-
+    @Override
     public ServerQueueStatus getQueueStatus() {
       return null;
     }
 
+    @Override
     public Object execute(Op op) throws Exception {
       return op.attempt(this);
     }
-    
+
+    @Override
     public void emergencyClose() {
     }
-    
+
+    @Override
     public short getWanSiteVersion(){
       return -1;
     }
-    
+
+    @Override
     public void setWanSiteVersion(short wanSiteVersion){
     }
+
+    @Override
     public InputStream getInputStream() {
       return null;
     }
+
+    @Override
     public OutputStream getOutputStream() {
       return null;
-    } 
+    }
+
+    @Override
     public void setConnectionID(long id) {
     }
+
+    @Override
     public long getConnectionID() {
       return 0;
     }
   }
   
-    
-  public class DummyEndpointManager implements EndpointManager {
-
-    
+  private class DummyEndpointManager implements EndpointManager {
 
+    @Override
     public void addListener(EndpointListener listener) {
     }
 
+    @Override
     public void close() {
     }
 
+    @Override
     public Endpoint referenceEndpoint(ServerLocation server, DistributedMember memberId) {
       return null;
     }
 
+    @Override
     public Map getEndpointMap() {
       return null;
     }
 
+    @Override
     public void removeListener(EndpointListener listener) {
-      
     }
 
+    @Override
     public void serverCrashed(Endpoint endpoint) {
       serverCrashes++;
     }
+
+    @Override
     public int getConnectedServerCount() {
       return 0;
     }
 
-    public void fireEndpointNowInUse(Endpoint endpoint) {
-      // TODO Auto-generated method stub
-      
-    }
-
+    @Override
     public Map getAllStats() {
       return null;
     }
 
+    @Override
     public String getPoolName() {
       return null;
     }
   }
   
-  public class DummyQueueManager implements QueueManager {
+  private class DummyQueueManager implements QueueManager {
+
     int backups = 0;
     int currentServer = 0;
+
     public QueueConnections getAllConnectionsNoWait() {
       return getAllConnections();
     }
-    
+
+    @Override
     public void emergencyClose() {
     }
 
-
-
+    @Override
     public QueueConnections getAllConnections() {
       return new QueueConnections() {
+        @Override
         public List getBackups() {
           getBackups++;
           ArrayList result = new ArrayList(backups);
@@ -622,41 +677,47 @@ public class OpExecutorImplJUnitTest extends TestCase {
           }
           return result;
         }
+        @Override
         public Connection getPrimary() {
           getPrimary++;
           return new DummyConnection(new ServerLocation("localhost", currentServer++));
         }
+        @Override
         public QueueConnectionImpl getConnection(Endpoint ep) {
           return null;
         }
       };
     }
 
+    @Override
     public void close(boolean keepAlive) {
     }
 
+    @Override
     public void start(ScheduledExecutorService background) {
     }
-    
-    
+
+    @Override
     public QueueState getState() {
       return null;
     }
 
+    @Override
     public InternalPool getPool() {
       return null;
     }
 
+    @Override
     public void readyForEvents(InternalDistributedSystem system) {
     }
-    
+
+    @Override
     public InternalLogWriter getSecurityLogger() {
       return null;
     }
 
+    @Override
     public void checkEndpoint(ClientUpdater qc, Endpoint endpoint) {
-      // TODO Auto-generated method stub
-      
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ServerBlackListJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ServerBlackListJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ServerBlackListJUnitTest.java
index 8a9d7a1..e9e50c2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ServerBlackListJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ServerBlackListJUnitTest.java
@@ -28,21 +28,17 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.admin.DistributedSystemConfig;
 import com.gemstone.gemfire.cache.client.internal.ServerBlackList.BlackListListenerAdapter;
 import com.gemstone.gemfire.cache.client.internal.ServerBlackList.FailureTracker;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.util.StopWatch;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- *
- */
 @Category(UnitTest.class)
 public class ServerBlackListJUnitTest {
   
   private ScheduledExecutorService background;
-  protected ServerBlackList blackList;
+  private ServerBlackList blackList;
 
   @Before
   public void setUp()  throws Exception {
@@ -67,12 +63,8 @@ public class ServerBlackListJUnitTest {
     assertEquals(Collections.singleton(location1),  blackList.getBadServers());
     
     boolean done = false;
-    try {
-      for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 10000; done = (blackList.getBadServers().size() == 0)) {
-        Thread.sleep(200);
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
+    for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 10000; done = (blackList.getBadServers().size() == 0)) {
+      Thread.sleep(200);
     }
     assertTrue("blackList still has bad servers", done);
     
@@ -81,15 +73,16 @@ public class ServerBlackListJUnitTest {
 
   @Test
   public void testListener()  throws Exception {
-    
     final AtomicInteger adds = new AtomicInteger();
     final AtomicInteger removes = new AtomicInteger();
     blackList.addListener(new BlackListListenerAdapter() {
 
+      @Override
       public void serverAdded(ServerLocation location) {
         adds.incrementAndGet();
       }
 
+      @Override
       public void serverRemoved(ServerLocation location) {
         removes.incrementAndGet();
       }
@@ -107,12 +100,8 @@ public class ServerBlackListJUnitTest {
     assertEquals(0, removes.get());
     
     boolean done = false;
-    try {
-      for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 10000; done = (removes.get() != 0)) {
-        Thread.sleep(200);
-      }
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
+    for (StopWatch time = new StopWatch(true); !done && time.elapsedTimeMillis() < 10000; done = (removes.get() != 0)) {
+      Thread.sleep(200);
     }
     assertTrue("removes still empty", done);
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/locator/LocatorStatusResponseJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/locator/LocatorStatusResponseJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/locator/LocatorStatusResponseJUnitTest.java
index f7a960b..165dcd4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/locator/LocatorStatusResponseJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/locator/LocatorStatusResponseJUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.cache.client.internal.locator;
 
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
index 6360d59..fc06afd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
@@ -448,8 +448,8 @@ public class ConnectionManagerJUnitTest {
     
 //     //wait to make sure checked out connections aren't timed out
 //     Thread.sleep(idleTimeout + 100);
-//     Assert.assertEquals(5,factory.creates);
-//     Assert.assertEquals(0,factory.destroys);
+//     Assert.assertIndexDetailsEquals(5,factory.creates);
+//     Assert.assertIndexDetailsEquals(0,factory.destroys);
     
 //     manager.returnConnection(conn1);
 //     manager.returnConnection(conn2);
@@ -468,8 +468,8 @@ public class ConnectionManagerJUnitTest {
 //     long elapsed = System.currentTimeMillis() - start;
 //     Assert.assertTrue(elapsed > idleTimeout);
     
-//     Assert.assertEquals(5,factory.creates);
-//     Assert.assertEquals(3,factory.destroys);
+//     Assert.assertIndexDetailsEquals(5,factory.creates);
+//     Assert.assertIndexDetailsEquals(3,factory.destroys);
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
index 9f57fc0..b8d5008 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
@@ -474,7 +474,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
                     primaryCount++;
                   }
                 }
-                //assertEquals(memberSizes[membersIdx] * (1024* 1024), 
+                //assertIndexDetailsEquals(memberSizes[membersIdx] * (1024* 1024),
                 //             totalBucketBytes);
                 assertEquals(memberPrimaryCounts[membersIdx], primaryCount);
                 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
index 7b265f9..8df44ea 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
@@ -16,24 +16,13 @@
  */
 package com.gemstone.gemfire.cache.operations;
 
-import static org.junit.Assert.*;
-import static org.junit.matchers.JUnitMatchers.*;
-
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.util.Properties;
 
-import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.matchers.JUnitMatchers;
-import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Cache;
@@ -42,30 +31,11 @@ import com.gemstone.gemfire.pdx.PdxInstance;
 import com.gemstone.gemfire.pdx.PdxReader;
 import com.gemstone.gemfire.pdx.PdxSerializable;
 import com.gemstone.gemfire.pdx.PdxWriter;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class PutOperationContextJUnitTest {
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-  
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-  }
-
-  @Before
-  public void setUp() throws Exception {
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
   @Test
   public void testGetSerializedValue() throws IOException {
     {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
index 8efec6e..9ebbbf3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
@@ -16,57 +16,24 @@
  */
 package com.gemstone.gemfire.cache.operations.internal;
 
-import static org.junit.Assert.*;
-import static org.junit.matchers.JUnitMatchers.*;
-
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.util.Properties;
 
-import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.matchers.JUnitMatchers;
-import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.operations.PutOperationContextJUnitTest;
 import com.gemstone.gemfire.pdx.PdxInstance;
-import com.gemstone.gemfire.pdx.PdxReader;
-import com.gemstone.gemfire.pdx.PdxSerializable;
-import com.gemstone.gemfire.pdx.PdxWriter;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class GetOperationContextImplJUnitTest {
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-  
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-  }
-
-  @Before
-  public void setUp() throws Exception {
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
   @Test
   public void testGetSerializedValue() throws IOException {
     {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/partition/PartitionRegionHelperDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/partition/PartitionRegionHelperDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/partition/PartitionRegionHelperDUnitTest.java
index d09c6e0..6910089 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/partition/PartitionRegionHelperDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/partition/PartitionRegionHelperDUnitTest.java
@@ -115,7 +115,7 @@ public class PartitionRegionHelperDUnitTest extends CacheTestCase {
           //when we get to the last bucket, which should be primary on that node, we
           //don't even put a copy of the bucket on that node
           // See bug #40470
-//          assertEquals(4, member.getPrimaryCount());
+//          assertIndexDetailsEquals(4, member.getPrimaryCount());
         }
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/CacheUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/CacheUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/CacheUtils.java
index 88d0e30..78ec8b6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/CacheUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/CacheUtils.java
@@ -361,7 +361,7 @@ public class CacheUtils {
           if (p1 instanceof Struct) { 
             Object[] values1 = ((Struct)p1).getFieldValues(); 
             Object[] values2 = ((Struct)p2).getFieldValues(); 
-            //test.assertEquals(values1.length, values2.length); 
+            //test.assertIndexDetailsEquals(values1.length, values2.length);
             if(values1.length != values2.length) { 
               ok = false; 
               break outer; 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
index 934798c..0498a79 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
@@ -1,19 +1,19 @@
-/*
- * 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.
- */
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package com.gemstone.gemfire.cache.query.dunit;
 
 import java.io.IOException;
@@ -60,16 +60,16 @@ import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator;
-import com.gemstone.gemfire.pdx.internal.PdxString;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.Invoke;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
-import com.gemstone.gemfire.test.dunit.SerializableCallable;
-import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.pdx.internal.PdxString;
+import com.gemstone.gemfire.test.dunit.Assert;
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 
 public class PdxStringQueryDUnitTest extends CacheTestCase{
@@ -82,8 +82,8 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
   private final String rootRegionName = "root";
   private final String regionName = "PdxTest";
   private final String regName = "/" + rootRegionName + "/" + regionName;
-  private final static int orderByQueryIndex = 11;
-  private final static int [] groupByQueryIndex = new int[]{7, 8, 9,10};
+  private final static int orderByQueryIndex = 11;
+  private final static int [] groupByQueryIndex = new int[]{7, 8, 9,10};
   
   private final String[] queryString = new String[] { 
       "SELECT pos.secId FROM " + regName + " p, p.positions.values pos WHERE pos.secId LIKE '%L'",//0
@@ -92,10 +92,10 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       "SELECT pos.secId FROM " + regName + " p, p.positions.values pos WHERE pos.secId > 'APPL' and pos.secId < 'SUN'",//3
       "select pos.secId from " + regName + " p, p.positions.values pos where pos.secId  IN SET ('YHOO', 'VMW')",//4
       "select pos.secId from " + regName + " p, p.positions.values pos where NOT (pos.secId = 'VMW')",//5
-      "select pos.secId from " + regName + " p, p.positions.values pos where NOT (pos.secId IN SET('SUN', 'ORCL')) ",//6
-      "select pos.secId , count(pos.id) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",//7
-      "select pos.secId , sum(pos.id) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",//8,
-      "select pos.secId , count(distinct pos.secId) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",//9
+      "select pos.secId from " + regName + " p, p.positions.values pos where NOT (pos.secId IN SET('SUN', 'ORCL')) ",//6
+      "select pos.secId , count(pos.id) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",//7
+      "select pos.secId , sum(pos.id) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",//8,
+      "select pos.secId , count(distinct pos.secId) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",//9
       "select  count(distinct pos.secId) from " + regName + " p, p.positions.values pos where  pos.secId > 'APPL' ",//10
       "SELECT distinct pos.secId FROM " + regName + " p, p.positions.values pos order by pos.secId",//11
       "SELECT distinct pos.secId FROM " + regName + " p, p.positions.values pos WHERE p.ID > 1 order by pos.secId limit 5",//12
@@ -108,10 +108,10 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       "SELECT pos.secIdIndexed FROM " + regName + " p, p.positions.values pos WHERE pos.secIdIndexed > 'APPL' and pos.secIdIndexed < 'SUN'",//3
       "select pos.secIdIndexed from " + regName + " p, p.positions.values pos where pos.secIdIndexed  IN SET ('YHOO', 'VMW')",//4
       "select pos.secIdIndexed from " + regName + " p, p.positions.values pos where NOT (pos.secIdIndexed = 'VMW')",//5
-      "select pos.secIdIndexed from " + regName + " p, p.positions.values pos where NOT (pos.secIdIndexed IN SET('SUN', 'ORCL')) ",//6
-      "select pos.secIdIndexed , count(pos.id) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL' group by pos.secIdIndexed ",//7
-      "select pos.secIdIndexed , sum(pos.id) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL' group by pos.secIdIndexed ",//8
-      "select pos.secIdIndexed , count(distinct pos.secIdIndexed) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL' group by pos.secIdIndexed ",//9
+      "select pos.secIdIndexed from " + regName + " p, p.positions.values pos where NOT (pos.secIdIndexed IN SET('SUN', 'ORCL')) ",//6
+      "select pos.secIdIndexed , count(pos.id) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL' group by pos.secIdIndexed ",//7
+      "select pos.secIdIndexed , sum(pos.id) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL' group by pos.secIdIndexed ",//8
+      "select pos.secIdIndexed , count(distinct pos.secIdIndexed) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL' group by pos.secIdIndexed ",//9
       "select  count(distinct pos.secIdIndexed) from " + regName + " p, p.positions.values pos where  pos.secIdIndexed > 'APPL'  ",//10
       "SELECT distinct pos.secIdIndexed FROM " + regName + " p, p.positions.values pos order by pos.secIdIndexed",//11
       "SELECT distinct pos.secIdIndexed FROM " + regName + " p, p.positions.values pos WHERE p.ID > 1 order by pos.secIdIndexed limit 5",//12
@@ -930,8 +930,8 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
 
             if(i < orderByQueryIndex){
               // Compare local and remote query results.
-              if (!compareResultsOfWithAndWithoutIndex(rs)){
-                LogWriterUtils.getLogWriter().info("result0="+rs[0][0].asList());
+              if (!compareResultsOfWithAndWithoutIndex(rs)){
+                LogWriterUtils.getLogWriter().info("result0="+rs[0][0].asList());
                 LogWriterUtils.getLogWriter().info("result1="+rs[0][1].asList());
                fail("Local and Remote Query Results are not matching for query :" + queryString[i]);  
               }
@@ -1417,8 +1417,8 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
                    
             if(i < orderByQueryIndex){
               // Compare local and remote query results.
-              if (!compareResultsOfWithAndWithoutIndex(rs)){
-                LogWriterUtils.getLogWriter().info("result0="+rs[0][0].asList());
+              if (!compareResultsOfWithAndWithoutIndex(rs)){
+                LogWriterUtils.getLogWriter().info("result0="+rs[0][0].asList());
                 LogWriterUtils.getLogWriter().info("result1="+rs[0][1].asList());
                fail("Local and Remote Query Results are not matching for query :" + queryString[i]);  
               }
@@ -1689,38 +1689,38 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
     }
   }
   
-  private void checkForPdxString(List results, String query) {
-    boolean isGroupByQuery = false;
-    for (int i : groupByQueryIndex) {
-      if (query.equals(queryString[i]) || query.equals(queryString2[i])) {
-        isGroupByQuery = true;
-        break;
-      }
-    }
-    for (Object o : results) {
-      if (o instanceof Struct) {
-        if (!isGroupByQuery) {
-          Object o1 = ((Struct) o).getFieldValues()[0];
-          Object o2 = ((Struct) o).getFieldValues()[1];
-          if (!(o1 instanceof String)) {
-            fail("Returned instance of " + o1.getClass()
-                + " instead of String for query: " + query);
-          }
-
-          if (!(o2 instanceof String)) {
-            fail("Returned instance of " + o2.getClass()
-                + " instead of String for query: " + query);
-          }
-        }
-      } else {
-        if (!isGroupByQuery) {
-          if (!(o instanceof String)) {
-            fail("Returned instance of " + o.getClass()
-                + " instead of String for query: " + query);
-          }
-        }
-      }
-    }
+  private void checkForPdxString(List results, String query) {
+    boolean isGroupByQuery = false;
+    for (int i : groupByQueryIndex) {
+      if (query.equals(queryString[i]) || query.equals(queryString2[i])) {
+        isGroupByQuery = true;
+        break;
+      }
+    }
+    for (Object o : results) {
+      if (o instanceof Struct) {
+        if (!isGroupByQuery) {
+          Object o1 = ((Struct) o).getFieldValues()[0];
+          Object o2 = ((Struct) o).getFieldValues()[1];
+          if (!(o1 instanceof String)) {
+            fail("Returned instance of " + o1.getClass()
+                + " instead of String for query: " + query);
+          }
+
+          if (!(o2 instanceof String)) {
+            fail("Returned instance of " + o2.getClass()
+                + " instead of String for query: " + query);
+          }
+        }
+      } else {
+        if (!isGroupByQuery) {
+          if (!(o instanceof String)) {
+            fail("Returned instance of " + o.getClass()
+                + " instead of String for query: " + query);
+          }
+        }
+      }
+    }
   }
 
   public boolean compareResultsOfWithAndWithoutIndex(SelectResults[][] r ) { 
@@ -1759,7 +1759,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
           if (p1 instanceof Struct) { 
             Object[] values1 = ((Struct)p1).getFieldValues(); 
             Object[] values2 = ((Struct)p2).getFieldValues(); 
-            //test.assertEquals(values1.length, values2.length); 
+            //test.assertIndexDetailsEquals(values1.length, values2.length);
             if(values1.length != values2.length) { 
               ok = false; 
               break outer; 
@@ -1966,7 +1966,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
     vm.invoke(new CacheSerializableRunnable("createPool :" + poolName) {
       public void run2() throws CacheException {
         // Create Cache.
-        getLonerSystem();
+        getLonerSystem();
         IgnoredException.addIgnoredException("Connection refused");
         getCache();        
         PoolFactory cpf = PoolManager.createFactory();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
index 4e94e8b..a87c243 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
@@ -567,7 +567,7 @@ public class QueryUsingPoolDUnitTest extends CacheTestCase {
         Assert.fail("Failed executing " + queryString, e);
       }
       LogWriterUtils.getLogWriter().fine("size: " + results.size());
-      //assertEquals(numberOfEntries, results.size());
+      //assertIndexDetailsEquals(numberOfEntries, results.size());
       assertTrue(!results.getCollectionType().allowsDuplicates() && results.getCollectionType().getElementType().isStructType());
     });
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
index 6769a0e..9b8ae0c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
@@ -201,7 +201,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           } catch (Exception e) {
             fail("Failed executing " + queryString, e);
         }
-          assertEquals(100, results.size());
+          assertIndexDetailsEquals(100, results.size());
           assertTrue(results instanceof ResultsCollectionWrapper);
           IdComparator comparator = new IdComparator();
           Object[] resultsArray = results.toArray();
@@ -659,7 +659,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
             Assert.fail("Failed executing " + queryString, e);
           }
           LogWriterUtils.getLogWriter().fine("size: " + results.size());
-          //assertEquals(numberOfEntries, results.size());
+          //assertIndexDetailsEquals(numberOfEntries, results.size());
           assertTrue(!results.getCollectionType().allowsDuplicates() && results.getCollectionType().getElementType().isStructType());
         }
       });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/CountStarJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/CountStarJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/CountStarJUnitTest.java
index b6d7af0..7b28cae 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/CountStarJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/CountStarJUnitTest.java
@@ -216,7 +216,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -271,7 +271,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -325,7 +325,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -360,7 +360,7 @@ public class CountStarJUnitTest {
     //Verify Index Creation
     //assertNotNull(queryService.getIndex(region, "sampleIndex-1"));
     //assertNotNull(queryService.getIndex(region, "sampleIndex-2"));
-    //assertEquals(3, queryService.getIndexes().size());
+    //assertIndexDetailsEquals(3, queryService.getIndexes().size());
 
     //Run queries
     Query query1 = null;
@@ -380,7 +380,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -423,7 +423,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -465,7 +465,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query with indexes: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -582,7 +582,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarDistinctQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarDistinctQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();
@@ -619,7 +619,7 @@ public class CountStarJUnitTest {
         //Also verify with size of result2 to count
         assertEquals("COUNT(*) query result is wrong for query: " + queryStr , result2.size(), count);
         
-        //assertEquals("Query: "+ queryStr, countStarDistinctQueries.get(queryStr).intValue(), count);
+        //assertIndexDetailsEquals("Query: "+ queryStr, countStarDistinctQueries.get(queryStr).intValue(), count);
       }
     } catch (Exception e){
       e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IUMRMultiIndexesMultiRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IUMRMultiIndexesMultiRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IUMRMultiIndexesMultiRegionJUnitTest.java
index c12d820..f326fb9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IUMRMultiIndexesMultiRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IUMRMultiIndexesMultiRegionJUnitTest.java
@@ -137,7 +137,7 @@ public class IUMRMultiIndexesMultiRegionJUnitTest {
           if( !(indexUsed).equals("statusIndexPf1")){
             fail("<statusIndexPf1> was expected but found "+indexUsed);
           }
-          //assertEquals("statusIndexPf1",itr.next().toString());
+          //assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -231,7 +231,7 @@ public class IUMRMultiIndexesMultiRegionJUnitTest {
             fail("indices used do not match with those which are expected to be used" +
                 "<statusIndexPf1> and <empIdIndex> were expected but found " +itr.next());
           }
-          //assertEquals("statusIndexPf1",itr.next().toString());
+          //assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
       } catch (Exception e) {
@@ -843,7 +843,7 @@ public class IUMRMultiIndexesMultiRegionJUnitTest {
             if(  !(temp.equals("idIndexPf") || temp.equals("empIdIndexPf2") || temp.equals("statusIndexPf"))){
               fail("<idIndexPf> or <empIdIndexPf2>    was expected but found "+temp.toString());
             }
-            //assertEquals("statusIndexPf1",itr.next().toString());
+            //assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
           }
 
           if (i != 3 /* For join query without parenthesis */) {
@@ -945,7 +945,7 @@ public class IUMRMultiIndexesMultiRegionJUnitTest {
             if(  !(temp.equals("idIndexPf") || temp.equals("empIdIndexPf2"))){
               fail("<idIndexPf> or <empIdIndexPf2>    was expected but found "+temp.toString());
             }
-            //assertEquals("statusIndexPf1",itr.next().toString());
+            //assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
           }
 
           int indxs = observer.indexesUsed.size();
@@ -1054,7 +1054,7 @@ public class IUMRMultiIndexesMultiRegionJUnitTest {
               fail("<idIndexPf> or <empIdIndexPf2>    was expected but found "
                   + temp.toString());
             }
-            // assertEquals("statusIndexPf1",itr.next().toString());
+            // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
           }
 
           int indxs = observer.indexesUsed.size();
@@ -1232,7 +1232,7 @@ public class IUMRMultiIndexesMultiRegionJUnitTest {
             if(  !(temp.equals("ageIndexemp")||temp.equals("idIndexPf") || temp.equals("empIdIndexPf2") || temp.equals("statusIndexPf3"))){
               fail("<idIndexPf> or <empIdIndexPf2>    was expected but found "+temp.toString());
             }
-            //assertEquals("statusIndexPf1",itr.next().toString());
+            //assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
           }
 
           int indxs = observer.indexesUsed.size();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexMaintenanceAsynchJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexMaintenanceAsynchJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexMaintenanceAsynchJUnitTest.java
index 168bb57..6f70ecf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexMaintenanceAsynchJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/IndexMaintenanceAsynchJUnitTest.java
@@ -102,7 +102,7 @@ public class IndexMaintenanceAsynchJUnitTest {
   public void testAddEntry() throws Exception {
 
     new NewThread(region, index);
-    //assertEquals(5, stats.getNumberOfValues());
+    //assertIndexDetailsEquals(5, stats.getNumberOfValues());
     Thread.sleep(12000);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/JavaSerializationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/JavaSerializationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/JavaSerializationJUnitTest.java
index 0242e39..c300a1c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/JavaSerializationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/JavaSerializationJUnitTest.java
@@ -16,15 +16,16 @@
  */
 package com.gemstone.gemfire.cache.query.functional;
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import junit.framework.TestCase;
-
 import com.gemstone.gemfire.cache.query.QueryService;
 import com.gemstone.gemfire.cache.query.internal.StructImpl;
 import com.gemstone.gemfire.cache.query.internal.types.ObjectTypeImpl;
@@ -33,12 +34,9 @@ import com.gemstone.gemfire.cache.query.types.ObjectType;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class JavaSerializationJUnitTest extends TestCase {
-  
-  public JavaSerializationJUnitTest(String testName) {
-    super(testName);
-  }
-  
+public class JavaSerializationJUnitTest {
+
+  @Test
   public void testStructImplSerialization() throws Exception {
     String[] fieldNames = {"col1", "col2"};
     ObjectType[] fieldTypes = {new ObjectTypeImpl(Integer.class), new ObjectTypeImpl(String.class)};
@@ -47,7 +45,8 @@ public class JavaSerializationJUnitTest extends TestCase {
     StructImpl si = new StructImpl(type, values);
     verifyJavaSerialization(si);   
   }
-  
+
+  @Test
   public void testUndefinedSerialization() throws Exception {
     verifyJavaSerialization(QueryService.UNDEFINED);   
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/LimitClauseJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/LimitClauseJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/LimitClauseJUnitTest.java
index c8f08a1..5280dfb 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/LimitClauseJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/LimitClauseJUnitTest.java
@@ -1289,14 +1289,14 @@ public class LimitClauseJUnitTest {
       assertTrue(idIndex instanceof CompactRangeIndex);
       SelectResults resultsWithIndex = (SelectResults) query.execute();
 
-      assertEquals(resultsNoIndex.size(), resultsWithIndex.size());
-      assertEquals(limit, resultsWithIndex.size());
+      assertIndexDetailsEquals(resultsNoIndex.size(), resultsWithIndex.size());
+      assertIndexDetailsEquals(limit, resultsWithIndex.size());
       
       int expectedId = 6;
       Iterator iterator = resultsNoIndex.iterator();
       while (iterator.hasNext()) {
         Portfolio p = (Portfolio) iterator.next();
-        assertEquals(expectedId++, p.ID);
+        assertIndexDetailsEquals(expectedId++, p.ID);
       }
       
       //check order by results
@@ -1304,7 +1304,7 @@ public class LimitClauseJUnitTest {
       iterator = resultsWithIndex.iterator();
       while (iterator.hasNext()) {
         Portfolio p = (Portfolio) iterator.next();
-        assertEquals(expectedId++, p.ID);
+        assertIndexDetailsEquals(expectedId++, p.ID);
       }      
   }
  */ 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MiscJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MiscJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MiscJUnitTest.java
index cb2a5f1..4bd3258 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MiscJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MiscJUnitTest.java
@@ -320,7 +320,7 @@ public class MiscJUnitTest {
       j = 0;
       while ( itr.hasNext()) {
        Struct struct = (Struct)itr.next();
-       assertEquals(j++, ((Integer)struct.get("getID")).intValue());
+       assertIndexDetailsEquals(j++, ((Integer)struct.get("getID")).intValue());
       }*/      
     }
     catch (Exception e) {      
@@ -614,22 +614,22 @@ public class MiscJUnitTest {
     String qStr = "Select distinct * from /portfolios pf where pf.getID() > Integer.MIN_VALUE"; 
     Query q = qs.newQuery(qStr);
     SelectResults result = (SelectResults) q.execute();
-    assertEquals(result.size(),4);  
+    assertIndexDetailsEquals(result.size(),4);
     
     qStr = "Select distinct * from /portfolios pf where pf.getID() > Integer.MAX_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),0);
+    assertIndexDetailsEquals(result.size(),0);
     
     qStr = "Select distinct * from /portfolios pf where pf.getID() > \"Long\".MAX_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),0);
+    assertIndexDetailsEquals(result.size(),0);
     
     qStr = "Select distinct * from /portfolios pf where pf.getID() > \"Long\".MIN_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),4);
+    assertIndexDetailsEquals(result.size(),4);
     
     qStr = "Select distinct * from /portfolios pf where pf.getID() > integer.MIN_VALUE"; 
     q = qs.newQuery(qStr);
@@ -643,22 +643,22 @@ public class MiscJUnitTest {
     qStr = "Select distinct * from /portfolios pf where pf.intFunction(Integer.MIN_VALUE) = Integer.MIN_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),4);
+    assertIndexDetailsEquals(result.size(),4);
     
     qStr = "Select distinct * from /portfolios pf where pf.intFunction(Integer.MAX_VALUE) = Integer.MAX_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),4);
+    assertIndexDetailsEquals(result.size(),4);
     
     qStr = "Select distinct * from /portfolios pf where pf.longFunction( \"Long\".MAX_VALUE) = \"Long\".MAX_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),4);
+    assertIndexDetailsEquals(result.size(),4);
     
     qStr = "Select distinct * from /portfolios pf where pf.longFunction(\"Long\".MIN_VALUE) = \"Long\".MIN_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),4);
+    assertIndexDetailsEquals(result.size(),4);
       
     qStr = "Select distinct * from /portfolios pf where pf.Integer.MAX_VALUE = Integer.MAX_VALUE"; 
     q = qs.newQuery(qStr);
@@ -672,8 +672,8 @@ public class MiscJUnitTest {
     qStr = "Select distinct Integer.MAX_VALUE from /portfolios pf where Integer.MAX_VALUE = Integer.MAX_VALUE"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),1);
-    assertEquals(result.iterator().next(),Integer.valueOf(Integer.MAX_VALUE));
+    assertIndexDetailsEquals(result.size(),1);
+    assertIndexDetailsEquals(result.iterator().next(),Integer.valueOf(Integer.MAX_VALUE));
     
     
   }
@@ -687,27 +687,27 @@ public class MiscJUnitTest {
     String qStr = "Select distinct * from /tempdata td where td.Integer.MAX_VALUE > Integer.MAX_VALUE"; 
     Query q = qs.newQuery(qStr);
     SelectResults result = (SelectResults) q.execute();
-    assertEquals(result.size(),0);  
+    assertIndexDetailsEquals(result.size(),0);
     
     qStr = "Select distinct * from /tempdata td where td.Integer.MAX_VALUE = 2 "; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),1);
+    assertIndexDetailsEquals(result.size(),1);
        
     qStr = "Select distinct * from /tempdata td where Integer.MAX_VALUE = 2 "; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),0);
+    assertIndexDetailsEquals(result.size(),0);
     
     qStr = "Select distinct * from /tempdata td where Integer.MAX_VALUE.intValue = Integer.MAX_VALUE "; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),0);
+    assertIndexDetailsEquals(result.size(),0);
     
     qStr = "Select distinct * from /tempdata td where Integer.test = 'test'"; 
     q = qs.newQuery(qStr);
     result = (SelectResults) q.execute();
-    assertEquals(result.size(),1);
+    assertIndexDetailsEquals(result.size(),1);
   }  
   
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MultiRegionIndexUsageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MultiRegionIndexUsageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MultiRegionIndexUsageJUnitTest.java
index 4354b89..4b6071c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MultiRegionIndexUsageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/MultiRegionIndexUsageJUnitTest.java
@@ -206,8 +206,8 @@ public class MultiRegionIndexUsageJUnitTest {
       }
 
       // while(itr.hasNext()){
-      // assertEquals("villageName1", itr.next().toString());
-      // assertEquals("cityName2", itr.next().toString());
+      // assertIndexDetailsEquals("villageName1", itr.next().toString());
+      // assertIndexDetailsEquals("cityName2", itr.next().toString());
       // }
 
       CacheUtils.log("5555555555555555555555555555");
@@ -269,8 +269,8 @@ public class MultiRegionIndexUsageJUnitTest {
         }
       }
 
-      // assertEquals("villageName1", itr.next().toString());
-      // assertEquals("cityName3", itr.next().toString());
+      // assertIndexDetailsEquals("villageName1", itr.next().toString());
+      // assertIndexDetailsEquals("cityName3", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
 
@@ -404,8 +404,8 @@ public class MultiRegionIndexUsageJUnitTest {
         }
       }
 
-      // assertEquals("districtName3", itr.next().toString());
-      // assertEquals("stateName2", itr.next().toString());
+      // assertIndexDetailsEquals("districtName3", itr.next().toString());
+      // assertIndexDetailsEquals("stateName2", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
 
@@ -461,8 +461,8 @@ public class MultiRegionIndexUsageJUnitTest {
         }
       }
 
-      // assertEquals("cityName1", itr.next().toString());
-      // assertEquals("cityName2", itr.next().toString());
+      // assertIndexDetailsEquals("cityName1", itr.next().toString());
+      // assertIndexDetailsEquals("cityName2", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
       CacheUtils.log("------------- testCutDown1 end------------- ");
@@ -516,8 +516,8 @@ public class MultiRegionIndexUsageJUnitTest {
         }
       }
 
-      // assertEquals("stateName2", itr.next().toString());
-      // assertEquals("cityName1", itr.next().toString());
+      // assertIndexDetailsEquals("stateName2", itr.next().toString());
+      // assertIndexDetailsEquals("cityName1", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
       CacheUtils.log("------------- testSelectAsFromClause end------------- ");
@@ -571,9 +571,9 @@ public class MultiRegionIndexUsageJUnitTest {
         }
       }
 
-      // assertEquals("cityName3", itr.next().toString());
-      // assertEquals("cityName1", itr.next().toString());
-      // assertEquals("stateName1", itr.next().toString());
+      // assertIndexDetailsEquals("cityName3", itr.next().toString());
+      // assertIndexDetailsEquals("cityName1", itr.next().toString());
+      // assertIndexDetailsEquals("stateName1", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
       CacheUtils.log("------------- testSelectAsWhereClause end------------- ");
@@ -665,8 +665,8 @@ public class MultiRegionIndexUsageJUnitTest {
         }
       }
 
-      // assertEquals("cityName", itr.next().toString());
-      // assertEquals("stateName", itr.next().toString());
+      // assertIndexDetailsEquals("cityName", itr.next().toString());
+      // assertIndexDetailsEquals("stateName", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
       CacheUtils.log("------------- testFunctionUse2 end------------- ");
@@ -741,7 +741,7 @@ public class MultiRegionIndexUsageJUnitTest {
       QueryObserverHolder.setInstance(observer);
       rs[0][1] = (SelectResults) q.execute();
 
-      // assertEquals("districtName3", itr.next().toString());
+      // assertIndexDetailsEquals("districtName3", itr.next().toString());
 
       areResultsMatching(rs, new String[] { sqlStr });
       CacheUtils.log("------------- testFunctionUse4 end------------- ");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByReplicatedJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByReplicatedJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByReplicatedJUnitTest.java
index 9f8ef4f..7ddb6b4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByReplicatedJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByReplicatedJUnitTest.java
@@ -160,7 +160,7 @@ public class NonDistinctOrderByReplicatedJUnitTest extends NonDistinctOrderByTes
           if (!(indexUsed).equals("PKIDIndexPf1")) {
             fail("<PKIDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -259,7 +259,7 @@ public class NonDistinctOrderByReplicatedJUnitTest extends NonDistinctOrderByTes
           if (!(indexUsed).equals("PKIDIndexPf1")) {
             fail("<PKIDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByTestImplementation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByTestImplementation.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByTestImplementation.java
index 41e0413..4feadaf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByTestImplementation.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/NonDistinctOrderByTestImplementation.java
@@ -164,7 +164,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(itr.next().toString()).equals("IDIndexPf1")) {
             fail("<IDIndexPf1> was expected but found " + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -255,7 +255,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           limit = Integer.parseInt(queries[i].substring(indexLimit + 5).trim());
         }
         assertTrue(!limitQuery || rcw.size() <= limit);
-        // assertEquals("Set",rcw.getCollectionType().getSimpleClassName());
+        // assertIndexDetailsEquals("Set",rcw.getCollectionType().getSimpleClassName());
         if (!rcw.getCollectionType().isOrdered()) {
           fail("The collection type="+rcw.getCollectionType().getSimpleClassName()+"  is not ordered");
         }
@@ -269,7 +269,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(itr.next().toString()).equals("IDIndexPf1")) {
             fail("<IDIndexPf1> was expected but found " + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -368,7 +368,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(itr.next().toString()).equals("IDIndexPf1")) {
             fail("<IDIndexPf1> was expected but found " + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -468,7 +468,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(itr.next().toString()).equals("IDIndexPf1")) {
             fail("<IDIndexPf1> was expected but found " + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -571,7 +571,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(indexUsed).equals("IDIndexPf1")) {
             fail("<IDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -674,7 +674,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(indexUsed).equals("IDIndexPf1")) {
             fail("<IDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -775,7 +775,7 @@ public abstract class NonDistinctOrderByTestImplementation {
           if (!(indexUsed).equals("PKIDIndexPf1")) {
             fail("<PKIDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByReplicatedJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByReplicatedJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByReplicatedJUnitTest.java
index 84f9350..e1372e9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByReplicatedJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByReplicatedJUnitTest.java
@@ -491,7 +491,7 @@ public class OrderByReplicatedJUnitTest extends OrderByTestImplementation {
           if (!(indexUsed).equals("PKIDIndexPf1")) {
             fail("<PKIDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -624,7 +624,7 @@ public class OrderByReplicatedJUnitTest extends OrderByTestImplementation {
           if (!(indexUsed).equals("PKIDIndexPf1")) {
             fail("<PKIDIndexPf1> was expected but found " + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByTestImplementation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByTestImplementation.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByTestImplementation.java
index 71e2c6d..206346e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByTestImplementation.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/OrderByTestImplementation.java
@@ -164,7 +164,7 @@ public abstract class OrderByTestImplementation {
             fail("<IDIndexPf1> was expected but found "
                 + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -314,7 +314,7 @@ public abstract class OrderByTestImplementation {
           limit = Integer.parseInt(queries[i].substring(indexLimit+5).trim());
         }
         assertTrue(!limitQuery || rcw.size() <= limit);
-        //assertEquals("Set",rcw.getCollectionType().getSimpleClassName());
+        //assertIndexDetailsEquals("Set",rcw.getCollectionType().getSimpleClassName());
         String colType = rcw.getCollectionType().getSimpleClassName();
         if (!(colType.equals("Ordered") || colType.equals("LinkedHashSet"))) {
           fail("The collection type " + colType + " is not expexted");
@@ -329,7 +329,7 @@ public abstract class OrderByTestImplementation {
             fail("<IDIndexPf1> was expected but found "
                 + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -429,7 +429,7 @@ public abstract class OrderByTestImplementation {
             fail("<IDIndexPf1> was expected but found "
                 + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -512,7 +512,7 @@ public abstract class OrderByTestImplementation {
             fail("<IDIndexPf1> was expected but found "
                 + itr.next().toString());
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -615,7 +615,7 @@ public abstract class OrderByTestImplementation {
             fail("<IDIndexPf1> was expected but found "
                 + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -699,7 +699,7 @@ public abstract class OrderByTestImplementation {
             fail("<IDIndexPf1> was expected but found "
                 + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();
@@ -783,7 +783,7 @@ public abstract class OrderByTestImplementation {
             fail("<PKIDIndexPf1> was expected but found "
                 + indexUsed);
           }
-          // assertEquals("statusIndexPf1",itr.next().toString());
+          // assertIndexDetailsEquals("statusIndexPf1",itr.next().toString());
         }
 
         int indxs = observer.indexesUsed.size();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/PdxOrderByJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/PdxOrderByJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/PdxOrderByJUnitTest.java
index a1cbff4..2f0327c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/PdxOrderByJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/PdxOrderByJUnitTest.java
@@ -306,7 +306,7 @@ public class PdxOrderByJUnitTest {
           if (p1 instanceof Struct) {
             Object[] values1 = ((Struct) p1).getFieldValues();
             Object[] values2 = ((Struct) p2).getFieldValues();
-            // test.assertEquals(values1.length, values2.length);
+            // test.assertIndexDetailsEquals(values1.length, values2.length);
             if (values1.length != values2.length) {
               ok = false;
               break ;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef0a6243/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledAggregateFunctionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
index fc793bf..c7d83b3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/CompiledAggregateFunctionJUnitTest.java
@@ -16,13 +16,14 @@
  */
 package com.gemstone.gemfire.cache.query.internal;
 
+import static org.junit.Assert.*;
+
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.TestCase;
-
 import org.jmock.Mockery;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -45,20 +46,19 @@ import com.gemstone.gemfire.cache.query.internal.aggregate.SumDistinctPRQueryNod
 import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-/**
- * 
- *
- */
 @Category(UnitTest.class)
-public class CompiledAggregateFunctionJUnitTest extends TestCase {
+public class CompiledAggregateFunctionJUnitTest {
   
-  Mockery context = new Mockery();
-  private Cache cache = context.mock(Cache.class) ; 
-  private List bucketList = new ArrayList();
+  private Mockery context;
+  private Cache cache;
+  private List bucketList;
   
-  public CompiledAggregateFunctionJUnitTest(String testName) {
-    super(testName);
-    bucketList.add(Integer.valueOf(1)); 
+  @Before
+  public void setUp() throws Exception {
+    context = new Mockery();
+    cache = context.mock(Cache.class);
+    bucketList = new ArrayList();
+    bucketList.add(Integer.valueOf(1));
   }
 
   @Test
@@ -194,6 +194,4 @@ public class CompiledAggregateFunctionJUnitTest extends TestCase {
     MaxMin maxMin1 = (MaxMin) agg1;
     assertFalse(((Boolean) findMax.get(maxMin1)).booleanValue());
   }
-
-  
 }